You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2015/09/07 13:20:36 UTC

[01/28] incubator-kylin git commit: KYLIN-975 fix bug when user configures kylin.job.hive.database.for.intermediatetable

Repository: incubator-kylin
Updated Branches:
  refs/heads/2.x-staging [created] 175392c6e


KYLIN-975 fix bug when user configures kylin.job.hive.database.for.intermediatetable

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/2.x-staging
Commit: 08845a1ec278510df35bf822a9f4868ab896615a
Parents: bca588b
Author: gaodayue <ga...@meituan.com>
Authored: Sun Aug 30 22:56:32 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Aug 31 09:34:26 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/invertedindex/IIJobBuilder.java   | 10 ++++---
 .../apache/kylin/source/hive/HiveMRInput.java   | 28 +++++++++++++-------
 2 files changed, 25 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/08845a1e/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
index b18cc61..b4f69d6 100644
--- a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
@@ -57,7 +57,7 @@ public final class IIJobBuilder {
         IIJob result = initialJob(seg, "BUILD", submitter);
         final String jobId = result.getId();
         final IIJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(seg.getIIDesc());
-        final String intermediateHiveTableName = intermediateTableDesc.getTableName();
+        final String intermediateTableIdentity = getIntermediateTableIdentity(intermediateTableDesc);
         final String factDistinctColumnsPath = getIIDistinctColumnsPath(seg, jobId);
         final String iiRootPath = getJobWorkingDir(jobId) + "/" + seg.getIIInstance().getName() + "/";
         final String iiPath = iiRootPath + "*";
@@ -65,11 +65,11 @@ public final class IIJobBuilder {
         final AbstractExecutable intermediateHiveTableStep = createFlatHiveTableStep(intermediateTableDesc, jobId);
         result.addTask(intermediateHiveTableStep);
 
-        result.addTask(createFactDistinctColumnsStep(seg, intermediateHiveTableName, jobId, factDistinctColumnsPath));
+        result.addTask(createFactDistinctColumnsStep(seg, intermediateTableIdentity, jobId, factDistinctColumnsPath));
 
         result.addTask(createBuildDictionaryStep(seg, factDistinctColumnsPath));
 
-        result.addTask(createInvertedIndexStep(seg, intermediateHiveTableName, iiRootPath));
+        result.addTask(createInvertedIndexStep(seg, intermediateTableIdentity, iiRootPath));
 
         // create htable step
         result.addTask(createCreateHTableStep(seg));
@@ -223,4 +223,8 @@ public final class IIJobBuilder {
     private String getJobWorkingDir(String uuid) {
         return engineConfig.getHdfsWorkingDirectory() + "/" + "kylin-" + uuid;
     }
+
+    private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {
+        return engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + "." + intermediateTableDesc.getTableName();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/08845a1e/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index fefd76f..ec9b432 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -59,8 +59,12 @@ public class HiveMRInput implements IMRInput {
         final String dbName;
         final String tableName;
 
-        public HiveTableInputFormat(String hiveTable) {
-            String[] parts = HadoopUtil.parseHiveTableName(hiveTable);
+        /**
+         * Construct a HiveTableInputFormat to read hive table.
+         * @param fullQualifiedTableName "databaseName.tableName"
+         */
+        public HiveTableInputFormat(String fullQualifiedTableName) {
+            String[] parts = HadoopUtil.parseHiveTableName(fullQualifiedTableName);
             dbName = parts[0];
             tableName = parts[1];
         }
@@ -112,8 +116,9 @@ public class HiveMRInput implements IMRInput {
             }
 
             ShellExecutable step = new ShellExecutable();
-            StringBuffer buf = new StringBuffer();
+            StringBuilder buf = new StringBuilder();
             buf.append("hive -e \"");
+            buf.append(useDatabaseHql + "\n");
             buf.append(dropTableHql + "\n");
             buf.append(createTableHql + "\n");
             buf.append(insertDataHqls + "\n");
@@ -129,15 +134,18 @@ public class HiveMRInput implements IMRInput {
         public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
             GarbageCollectionStep step = new GarbageCollectionStep();
             step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
-            step.setOldHiveTable(flatHiveTableDesc.getTableName());
+            step.setIntermediateTableIdentity(getIntermediateTableIdentity());
             jobFlow.addTask(step);
         }
 
         @Override
         public IMRTableInputFormat getFlatTableInputFormat() {
-            return new HiveTableInputFormat(flatHiveTableDesc.getTableName());
+            return new HiveTableInputFormat(getIntermediateTableIdentity());
         }
 
+        private String getIntermediateTableIdentity() {
+            return conf.getConfig().getHiveDatabaseForIntermediateTable() + "." + flatHiveTableDesc.getTableName();
+        }
     }
 
     public static class GarbageCollectionStep extends AbstractExecutable {
@@ -146,9 +154,9 @@ public class HiveMRInput implements IMRInput {
         protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
             StringBuffer output = new StringBuffer();
 
-            final String hiveTable = this.getOldHiveTable();
+            final String hiveTable = this.getIntermediateTableIdentity();
             if (StringUtils.isNotEmpty(hiveTable)) {
-                final String dropSQL = "USE " + context.getConfig().getHiveDatabaseForIntermediateTable() + ";" + " DROP TABLE IF EXISTS  " + hiveTable + ";";
+                final String dropSQL = "DROP TABLE IF EXISTS  " + hiveTable + ";";
                 final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
                 ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
                 try {
@@ -164,11 +172,11 @@ public class HiveMRInput implements IMRInput {
             return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
         }
 
-        public void setOldHiveTable(String hiveTable) {
-            setParam("oldHiveTable", hiveTable);
+        public void setIntermediateTableIdentity(String tableIdentity) {
+            setParam("oldHiveTable", tableIdentity);
         }
 
-        private String getOldHiveTable() {
+        private String getIntermediateTableIdentity() {
             return getParam("oldHiveTable");
         }
     }


[14/28] incubator-kylin git commit: Kylin support spark 1.3.0 and above

Posted by ma...@apache.org.
Kylin support spark 1.3.0 and above


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

Branch: refs/heads/2.x-staging
Commit: eb0652a6ba25ddb69b5e48095bd9dd6c92136120
Parents: 8c56d17
Author: qianhao.zhou <qi...@ebay.com>
Authored: Mon Aug 31 15:16:12 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Fri Sep 4 21:46:20 2015 +0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/eb0652a6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0a83f56..e5ee107 100644
--- a/pom.xml
+++ b/pom.xml
@@ -72,7 +72,7 @@
         <commons-email.version>1.1</commons-email.version>
 
         <!-- Spark -->
-        <spark.version>1.4.1</spark.version>
+        <spark.version>1.3.0</spark.version>
 
 
         <!-- Utility -->


[25/28] incubator-kylin git commit: bug fix for ci

Posted by ma...@apache.org.
bug fix for ci


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

Branch: refs/heads/2.x-staging
Commit: 9f4b93b949d93bcb1ca0e4afd22e36d4016ccae3
Parents: 7797c2b
Author: honma <ho...@ebay.com>
Authored: Mon Sep 7 14:38:56 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Sep 7 14:38:56 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/model/DataModelDesc.java | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9f4b93b9/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
index 1c6ef62..66581c0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
@@ -104,7 +104,7 @@ public class DataModelDesc extends RootPersistentEntity {
     public String getFactTable() {
         return factTable;
     }
-    
+
     public TableDesc getFactTableDesc() {
         return factTableDesc;
     }
@@ -175,6 +175,7 @@ public class DataModelDesc extends RootPersistentEntity {
     }
 
     public void init(Map<String, TableDesc> tables) {
+        this.factTable = this.factTable.toUpperCase();
         this.factTableDesc = tables.get(this.factTable.toUpperCase());
         if (factTableDesc == null) {
             throw new IllegalStateException("Fact table does not exist:" + this.factTable);
@@ -207,7 +208,7 @@ public class DataModelDesc extends RootPersistentEntity {
 
             StringUtil.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
             StringUtil.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
-            
+
             // primary key
             String[] pks = join.getPrimaryKey();
             TblColRef[] pkCols = new TblColRef[pks.length];
@@ -221,7 +222,7 @@ public class DataModelDesc extends RootPersistentEntity {
                 pkCols[i] = colRef;
             }
             join.setPrimaryKeyColumns(pkCols);
-            
+
             // foreign key
             String[] fks = join.getForeignKey();
             TblColRef[] fkCols = new TblColRef[fks.length];
@@ -235,7 +236,7 @@ public class DataModelDesc extends RootPersistentEntity {
                 fkCols[i] = colRef;
             }
             join.setForeignKeyColumns(fkCols);
-            
+
             // Validate join in dimension
             if (pkCols.length != fkCols.length) {
                 throw new IllegalStateException("Primary keys(" + lookup.getTable() + ")" + Arrays.toString(pks) + " are not consistent with Foreign keys(" + this.getFactTable() + ") " + Arrays.toString(fks));


[17/28] incubator-kylin git commit: KYLIN-1002 add cleanup tools

Posted by ma...@apache.org.
KYLIN-1002 add cleanup tools


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

Branch: refs/heads/2.x-staging
Commit: cdc33f5b94fa58ac6b648d8e73f1e7dea9dc6d39
Parents: cc0ede3
Author: honma <ho...@ebay.com>
Authored: Sun Sep 6 15:31:30 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Sun Sep 6 15:31:39 2015 +0800

----------------------------------------------------------------------
 .../kylin/storage/hbase/util/HBaseClean.java    | 127 +++++++++++++++++++
 .../kylin/storage/hbase/util/HBaseUsage.java    |  70 ++++++++++
 2 files changed, 197 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cdc33f5b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
new file mode 100644
index 0000000..1d28283
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.storage.hbase.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+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.client.HBaseAdmin;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * clean hbase tables by tag
+ */
+public class HBaseClean extends AbstractHadoopJob {
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(true).withDescription("actually delete or not").create("delete");
+
+    @SuppressWarnings("static-access")
+    private static final Option OPTION_TAG = OptionBuilder.withArgName("tag").hasArg().isRequired(true).withDescription("the tag of HTable").create("tag");
+
+    protected static final Logger log = LoggerFactory.getLogger(HBaseClean.class);
+    boolean delete = false;
+    String tag = null;
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        log.info("----- jobs args: " + Arrays.toString(args));
+        try {
+            options.addOption(OPTION_DELETE);
+            options.addOption(OPTION_TAG);
+            parseOptions(options, args);
+
+            log.info("options: '" + getOptionsAsString() + "'");
+            
+            tag = getOptionValue(OPTION_TAG);
+            delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
+
+            cleanUp();
+
+            return 0;
+        } catch (Exception e) {
+            e.printStackTrace(System.err);
+            throw e;
+        }
+    }
+
+    private void cleanUp() {
+        try {
+            // get all kylin hbase tables
+            Configuration conf = HBaseConfiguration.create();
+            HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+            String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
+            HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
+            List<String> allTablesNeedToBeDropped = Lists.newArrayList();
+            for (HTableDescriptor desc : tableDescriptors) {
+                String host = desc.getValue(IRealizationConstants.HTableTag);
+                if (tag.equalsIgnoreCase(host)) {
+                    allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
+                }
+            }
+
+            if (delete) {
+                // drop tables
+                for (String htableName : allTablesNeedToBeDropped) {
+                    log.info("Deleting HBase table " + htableName);
+                    if (hbaseAdmin.tableExists(htableName)) {
+                        if (hbaseAdmin.isTableEnabled(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();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        int exitCode = ToolRunner.run(new HBaseClean(), args);
+        System.exit(exitCode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cdc33f5b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
new file mode 100644
index 0000000..ffb1e25
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.storage.hbase.util;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class HBaseUsage {
+
+    public static void main(String[] args) throws IOException {
+        show();
+    }
+
+    private static void show() throws IOException {
+        Map<String, List<String>> envs = Maps.newHashMap();
+
+        // get all kylin hbase tables
+        Configuration conf = HBaseConfiguration.create();
+        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
+        HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
+        for (HTableDescriptor desc : tableDescriptors) {
+            String host = desc.getValue(IRealizationConstants.HTableTag);
+            if (StringUtils.isEmpty(host)) {
+                add("unknown", desc.getNameAsString(), envs);
+            } else {
+                add(host, desc.getNameAsString(), envs);
+            }
+        }
+
+        for (Map.Entry<String, List<String>> entry : envs.entrySet()) {
+            System.out.println(entry.getKey() + " has htable count: " + entry.getValue().size());
+        }
+        hbaseAdmin.close();
+    }
+
+    private static void add(String tag, String tableName, Map<String, List<String>> envs) {
+        if (!envs.containsKey(tag)) {
+            envs.put(tag, Lists.<String> newArrayList());
+        }
+        envs.get(tag).add(tableName);
+    }
+}


[24/28] incubator-kylin git commit: KYLIN-983 Query sql offset keyword bug

Posted by ma...@apache.org.
KYLIN-983 Query sql offset keyword bug

Signed-off-by: honma <ho...@ebay.com>


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

Branch: refs/heads/2.x-staging
Commit: 7797c2bfbe03e1bd7604b8dd9834f6c2a702e8ae
Parents: b817dc2
Author: wangxiaoyu8 <wa...@jd.com>
Authored: Mon Sep 7 10:00:58 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Sep 7 10:49:08 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/storage/StorageContext.java |  9 +++++++++
 .../apache/kylin/query/relnode/OLAPLimitRel.java |  5 +++++
 .../kylin/query/test/ITKylinQueryTest.java       |  3 +--
 .../resources/query/sql_verifyCount/query04.sql  | 19 -------------------
 .../query/sql_verifyCount/query04.sql.disable    | 19 +++++++++++++++++++
 .../resources/query/sql_verifyCount/query05.sql  | 19 -------------------
 .../query/sql_verifyCount/query05.sql.disable    | 19 +++++++++++++++++++
 .../resources/query/sql_verifyCount/query06.sql  | 19 -------------------
 .../query/sql_verifyCount/query06.sql.disable    | 19 +++++++++++++++++++
 .../resources/query/sql_verifyCount/query07.sql  | 19 -------------------
 .../query/sql_verifyCount/query07.sql.disable    | 19 +++++++++++++++++++
 .../resources/query/sql_verifyCount/query08.sql  | 19 +++++++++++++++++++
 .../query/sql_verifyCount/query08.sql.expected   |  1 +
 .../cube/v1/SerializedHBaseTupleIterator.java    |  2 +-
 14 files changed, 112 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index 3d4f04b..de83aca 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -40,6 +40,7 @@ public class StorageContext {
     private String connUrl;
     private int threshold;
     private int limit;
+    private int offset;
     private boolean hasSort;
     private List<MeasureDesc> sortMeasures;
     private List<OrderEnum> sortOrders;
@@ -94,6 +95,14 @@ public class StorageContext {
         this.limit = l;
     }
 
+    public int getOffset() {
+        return offset;
+    }
+
+    public void setOffset(int offset) {
+        this.offset = offset;
+    }
+
     public void enableLimit() {
         this.enableLimit = true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
index a31cc21..60acd40 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
@@ -78,6 +78,11 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel {
         Number limitValue = (Number) (((RexLiteral) localFetch).getValue());
         int limit = limitValue.intValue();
         this.context.storageContext.setLimit(limit);
+        if(localOffset != null) {
+            Number offsetValue = (Number) (((RexLiteral) localOffset).getValue());
+            int offset = offsetValue.intValue();
+            this.context.storageContext.setOffset(offset);
+        }
     }
 
     private ColumnRowType buildColumnRowType() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
index a08d595..72d7c4a 100644
--- a/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/ITKylinQueryTest.java
@@ -127,9 +127,8 @@ public class ITKylinQueryTest extends KylinTestBase {
         execAndCompQuery("src/test/resources/query/sql", null, true);
     }
 
-    @Ignore
     @Test
-    public void testSimpleQuery() throws Exception {
+    public void testVerifyQuery() throws Exception {
         verifyResultRowCount("src/test/resources/query/sql_verifyCount");
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/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
deleted file mode 100644
index 9d3e409..0000000
--- a/query/src/test/resources/query/sql_verifyCount/query04.sql
+++ /dev/null
@@ -1,19 +0,0 @@
---
--- Licensed to the Apache Software Foundation (ASF) under one
--- or more contributor license agreements.  See the NOTICE file
--- distributed with this work for additional information
--- regarding copyright ownership.  The ASF licenses this file
--- to you under the Apache License, Version 2.0 (the
--- "License"); you may not use this file except in compliance
--- with the License.  You may obtain a copy of the License at
---
---     http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
---
-
-select * from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query04.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query04.sql.disable b/query/src/test/resources/query/sql_verifyCount/query04.sql.disable
new file mode 100644
index 0000000..9d3e409
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query04.sql.disable
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select * from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/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
deleted file mode 100644
index 753a85a..0000000
--- a/query/src/test/resources/query/sql_verifyCount/query05.sql
+++ /dev/null
@@ -1,19 +0,0 @@
---
--- Licensed to the Apache Software Foundation (ASF) under one
--- or more contributor license agreements.  See the NOTICE file
--- distributed with this work for additional information
--- regarding copyright ownership.  The ASF licenses this file
--- to you under the Apache License, Version 2.0 (the
--- "License"); you may not use this file except in compliance
--- with the License.  You may obtain a copy of the License at
---
---     http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
---
-
-select price from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query05.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query05.sql.disable b/query/src/test/resources/query/sql_verifyCount/query05.sql.disable
new file mode 100644
index 0000000..753a85a
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query05.sql.disable
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select price from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/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
deleted file mode 100644
index 39578a4..0000000
--- a/query/src/test/resources/query/sql_verifyCount/query06.sql
+++ /dev/null
@@ -1,19 +0,0 @@
---
--- Licensed to the Apache Software Foundation (ASF) under one
--- or more contributor license agreements.  See the NOTICE file
--- distributed with this work for additional information
--- regarding copyright ownership.  The ASF licenses this file
--- to you under the Apache License, Version 2.0 (the
--- "License"); you may not use this file except in compliance
--- with the License.  You may obtain a copy of the License at
---
---     http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
---
-
-select lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query06.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query06.sql.disable b/query/src/test/resources/query/sql_verifyCount/query06.sql.disable
new file mode 100644
index 0000000..39578a4
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query06.sql.disable
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/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
deleted file mode 100644
index 0afb493..0000000
--- a/query/src/test/resources/query/sql_verifyCount/query07.sql
+++ /dev/null
@@ -1,19 +0,0 @@
---
--- Licensed to the Apache Software Foundation (ASF) under one
--- or more contributor license agreements.  See the NOTICE file
--- distributed with this work for additional information
--- regarding copyright ownership.  The ASF licenses this file
--- to you under the Apache License, Version 2.0 (the
--- "License"); you may not use this file except in compliance
--- with the License.  You may obtain a copy of the License at
---
---     http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
---
-
-select price,lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query07.sql.disable
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query07.sql.disable b/query/src/test/resources/query/sql_verifyCount/query07.sql.disable
new file mode 100644
index 0000000..0afb493
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query07.sql.disable
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select price,lstg_format_name from test_kylin_fact limit 100

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query08.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query08.sql b/query/src/test/resources/query/sql_verifyCount/query08.sql
new file mode 100644
index 0000000..0f467f8
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query08.sql
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select lstg_format_name,sum(price) as sp from test_kylin_fact group by lstg_format_name limit 1 offset 2

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/query/src/test/resources/query/sql_verifyCount/query08.sql.expected
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_verifyCount/query08.sql.expected b/query/src/test/resources/query/sql_verifyCount/query08.sql.expected
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/query/src/test/resources/query/sql_verifyCount/query08.sql.expected
@@ -0,0 +1 @@
+1

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7797c2bf/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
index 5f55683..e433b78 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
@@ -99,7 +99,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
             return true;
 
         // 1. check limit
-        if (context.isLimitEnabled() && scanCount >= context.getLimit()) {
+        if (context.isLimitEnabled() && scanCount >= context.getLimit() + context.getOffset()) {
             return false;
         }
         // 2. check partial result


[23/28] incubator-kylin git commit: KYLIN-991 bug fix

Posted by ma...@apache.org.
KYLIN-991 bug fix


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

Branch: refs/heads/2.x-staging
Commit: b817dc25ef258f0e25ea2c9cbbb7835e7b5cd5fa
Parents: 0eddcf5
Author: honma <ho...@ebay.com>
Authored: Mon Sep 7 10:11:24 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Sep 7 10:11:24 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/hadoop/cube/StorageCleanupJob.java       | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b817dc25/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index 489b9ed..d83e455 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -18,6 +18,11 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -48,11 +53,6 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 /**
  * @author ysong1
  */
@@ -114,8 +114,9 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             String creationTime = desc.getValue(IRealizationConstants.HTableCreationTime);
             if (KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix().equalsIgnoreCase(host)) {
                 //only take care htables that belongs to self, and created more than 2 days
-                if (StringUtils.isNotEmpty(creationTime) || (System.currentTimeMillis() - Long.valueOf(creationTime) > TIME_THREADSHOLD))
+                if (StringUtils.isEmpty(creationTime) || (System.currentTimeMillis() - Long.valueOf(creationTime) > TIME_THREADSHOLD)) {
                     allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
+                }
             }
         }
 


[19/28] incubator-kylin git commit: KYLIN-972 merge pom settings from 1.x

Posted by ma...@apache.org.
KYLIN-972 merge pom settings from 1.x


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

Branch: refs/heads/2.x-staging
Commit: d2067c4649b02ff0901aeaa36f8ac288b86ff2d9
Parents: 3116d55
Author: Luke Han <lu...@apache.org>
Authored: Sun Sep 6 16:26:25 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 16:26:25 2015 +0800

----------------------------------------------------------------------
 LICENSE                                         | 253 +++++++++++++++----
 NOTICE                                          |   6 +
 README.md                                       |  76 +-----
 assembly/pom.xml                                |   2 +-
 atopcalcite/pom.xml                             |   2 +-
 core-common/pom.xml                             |   2 +-
 core-cube/pom.xml                               |   2 +-
 core-dictionary/pom.xml                         |   2 +-
 core-job/pom.xml                                |   2 +-
 core-metadata/pom.xml                           |   2 +-
 core-storage/pom.xml                            |   2 +-
 docs/README.md                                  |   6 -
 engine-mr/pom.xml                               |   2 +-
 engine-spark/pom.xml                            |   2 +-
 engine-streaming/pom.xml                        |   2 +-
 invertedindex/pom.xml                           |   2 +-
 jdbc/pom.xml                                    |   2 +-
 job/pom.xml                                     |   2 +-
 kylin_architecture.png                          | Bin 449125 -> 0 bytes
 pom.xml                                         |  65 ++++-
 query/pom.xml                                   |   2 +-
 server/pom.xml                                  |   2 +-
 source-hive/pom.xml                             |   2 +-
 source-kafka/pom.xml                            |   2 +-
 src/main/config/assemblies/source-assembly.xml  | 107 ++++++++
 .../checkstyle/checkstyle-java-header.txt       |  17 ++
 src/main/config/checkstyle/checkstyle.xml       |  44 ++++
 src/main/config/checkstyle/suppressions.xml     |  32 +++
 storage-hbase/pom.xml                           |   2 +-
 streaming/pom.xml                               |   2 +-
 30 files changed, 500 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 1b4eae4..4bfafaa 100644
--- a/LICENSE
+++ b/LICENSE
@@ -1,56 +1,205 @@
-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
+                                 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:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) 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
+
+      (d) 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
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 0000000..f18fba3
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,6 @@
+Apache Kylin (incubating)
+Copyright 2014-2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 9131162..73313f5 100644
--- a/README.md
+++ b/README.md
@@ -1,76 +1,26 @@
-Kylin OLAP
-===========
+Apache Kylin
+============
 
-
-What is Kylin
-------------
 > 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.
-
-This [slice](docs/Apache Kylin 2014 Dec.pdf) might help you better understand what kylin is and how it works.
-
-
-Tutorial 
-------------
-
-Please follow this installation tutorial to start with Kylin: [Installation Tutorial](docs/Installation/Installation.md)
-
-
-Advanced Topics
--------
-####Connectivity
-1.[How to use kylin remote jdbc driver](docs/Tutorial/How to use kylin remote jdbc driver.md)
-
-2.[Kylin ODBC Driver Tutorial](docs/Tutorial/Kylin ODBC Driver Tutorial.md) 
-
-3.[SQL Reference](https://github.com/apache/incubator-calcite/blob/master/doc/reference.md)
-
-####REST
-
-1.[Kylin Restful API List](docs/REST/Kylin Restful API List.md)
-
-2.[Build Cube with Restful API](docs/REST/Build Cube with Restful API.md)
-
-3.[How to consume Kylin REST API in javascript](docs/REST/How to consume Kylin REST API in javascript.md)
+Apache Kylin is an open source Distributed Analytics Engine, contributed by eBay Inc., provides SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets.
 
-####Operations
-1.[Kylin Metadata Store](docs/Operations/Kylin Metadata Store.md)
+For more details, see the website [http://kylin.incubator.apache.org](http://kylin.incubator.apache.org).
 
-2.[Export Kylin HBase data](docs/Operations/Export Kylin HBase data.md)
-
-3.[Advanced settings of Kylin environment](docs/Operations/Advanced settings of Kylin environment.md)
-
-####Test
-
-1.[Run Kylin test case with HBase Mini Cluster](docs/Test/Run Kylin test case with HBase Mini Cluster.md)
-
-
-####Technial Details
-
-1.[New meta data model structure](docs/TechInside/New meta data model structure.md)
-
-2.[Job Engine Design](docs/JobEngine/Design.md)
+Documentation
+=============
+Please refer to [http://kylin.incubator.apache.org/docs/](http://kylin.incubator.apache.org/docs/).
 
 Get Help
-------------
-
-The fastest way to get response from our developers is to send email to our mail list <de...@kylin.incubator.apache.org>, and remember to subscribe our mail list via <de...@kylin.incubator.apache.org>
-
-
-Resources
-------------
-
-* [FAQ](docs/MISC/FAQ on Kylin Installation and Usage.md)
-
-* Web Site: <http://kylin.incubator.apache.org/>
+============
+The fastest way to get response from our developers is to send email to our mail list <de...@kylin.incubator.apache.org>,   
+and remember to subscribe our mail list via <de...@kylin.incubator.apache.org>
 
-* Developer Mail: <de...@kylin.incubator.apache.org>
+Licnese
+============
+Please refer to [LICENSE](https://github.com/apache/incubator-kylin/blob/master/LICENSE) file.
 
-* How To Contribute: See [this](docs/MISC/How to Contribute.md)
 
-* Apache Proposal: [Apache Kylin](https://wiki.apache.org/incubator/KylinProposal)
 
-* [Kylin Document Conventions](docs/MISC/Kylin Document Conventions.md)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 3fd13b9..bc758f7 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/atopcalcite/pom.xml
----------------------------------------------------------------------
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
index fd44a8f..45cca31 100644
--- a/atopcalcite/pom.xml
+++ b/atopcalcite/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-common/pom.xml
----------------------------------------------------------------------
diff --git a/core-common/pom.xml b/core-common/pom.xml
index 76d6868..577db42 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-cube/pom.xml
----------------------------------------------------------------------
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 68f74a1..58c1a14 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-dictionary/pom.xml
----------------------------------------------------------------------
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
index c58576a..2ea09fb 100644
--- a/core-dictionary/pom.xml
+++ b/core-dictionary/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-job/pom.xml
----------------------------------------------------------------------
diff --git a/core-job/pom.xml b/core-job/pom.xml
index 3714ce4..4a15572 100644
--- a/core-job/pom.xml
+++ b/core-job/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-metadata/pom.xml
----------------------------------------------------------------------
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index 3702427..c8836c4 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/core-storage/pom.xml
----------------------------------------------------------------------
diff --git a/core-storage/pom.xml b/core-storage/pom.xml
index 73f15b0..4bb7695 100644
--- a/core-storage/pom.xml
+++ b/core-storage/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/docs/README.md
----------------------------------------------------------------------
diff --git a/docs/README.md b/docs/README.md
deleted file mode 100644
index 166f9fd..0000000
--- a/docs/README.md
+++ /dev/null
@@ -1,6 +0,0 @@
-# Apache Kylin Documentation
-
-__All documentations have been moved to Kylin website at:__  
-[http://kylin.incubator.apache.org/docs](http://kylin.incubator.apache.org/docs)
-
-Please visit there for furthe reference.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/engine-mr/pom.xml
----------------------------------------------------------------------
diff --git a/engine-mr/pom.xml b/engine-mr/pom.xml
index c967cb3..e00a693 100644
--- a/engine-mr/pom.xml
+++ b/engine-mr/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/engine-spark/pom.xml
----------------------------------------------------------------------
diff --git a/engine-spark/pom.xml b/engine-spark/pom.xml
index 0f3154f..3aa01e3 100644
--- a/engine-spark/pom.xml
+++ b/engine-spark/pom.xml
@@ -6,7 +6,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>kylin-engine-spark</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/engine-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/engine-streaming/pom.xml b/engine-streaming/pom.xml
index 81313fc..46b63b3 100644
--- a/engine-streaming/pom.xml
+++ b/engine-streaming/pom.xml
@@ -11,7 +11,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/invertedindex/pom.xml
----------------------------------------------------------------------
diff --git a/invertedindex/pom.xml b/invertedindex/pom.xml
index e8cae20..4d1796f 100644
--- a/invertedindex/pom.xml
+++ b/invertedindex/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>kylin-invertedindex</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 55bc844..6106f3c 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -27,7 +27,7 @@
 	<parent>
 		<groupId>org.apache.kylin</groupId>
 		<artifactId>kylin</artifactId>
-		<version>0.8.1-incubating-SNAPSHOT</version>
+		<version>2.0-incubating-SNAPSHOT</version>
 	</parent>
 
 	<properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/job/pom.xml
----------------------------------------------------------------------
diff --git a/job/pom.xml b/job/pom.xml
index 9342de9..2e1e64b 100644
--- a/job/pom.xml
+++ b/job/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <artifactId>kylin-job</artifactId>

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e5ee107..360c6b1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -30,7 +30,7 @@
     <groupId>org.apache.kylin</groupId>
     <artifactId>kylin</artifactId>
     <packaging>pom</packaging>
-    <version>0.8.1-incubating-SNAPSHOT</version>
+    <version>2.0-incubating-SNAPSHOT</version>
     <name>Kylin:HadoopOLAPEngine</name>
     <url>http://kylin.incubator.apache.org</url>
     <inceptionYear>2014</inceptionYear>
@@ -128,6 +128,13 @@
 
     <mailingLists>
         <mailingList>
+            <name>kylin-user</name>
+            <subscribe>user-subscribe@kylin.incubator.apache.org</subscribe>
+            <unsubscribe>user-unsubscribe@kylin.incubator.apache.org</unsubscribe>
+            <post>user@kylin.incubator.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/kylin-user/</archive>
+        </mailingList>
+        <mailingList>
             <name>kylin-dev</name>
             <subscribe>dev-subscribe@kylin.incubator.apache.org</subscribe>
             <unsubscribe>dev-unsubscribe@kylin.incubator.apache.org</unsubscribe>
@@ -135,6 +142,13 @@
             <archive>http://mail-archives.apache.org/mod_mbox/kylin-dev/</archive>
         </mailingList>
         <mailingList>
+            <name>kylin-issues</name>
+            <subscribe>issues-subscribe@kylin.incubator.apache.org</subscribe>
+            <unsubscribe>issues-unsubscribe@kylin.incubator.apache.org</unsubscribe>
+            <post>issues@kylin.incubator.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/kylin-issues/</archive>
+        </mailingList>
+        <mailingList>
             <name>kylin-commits</name>
             <subscribe>commits-subscribe@kylin.incubator.apache.org</subscribe>
             <unsubscribe>commits-unsubscribe@kylin.incubator.apache.org</unsubscribe>
@@ -147,7 +161,7 @@
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</developerConnection>
         <url>https://git-wip-us.apache.org/repos/asf/incubator-kylin.git</url>
-        <tag>apache-kylin-0.7.1-incubating</tag>
+        <tag>apache-kylin-2.0-incubating</tag>
     </scm>
 
     <dependencyManagement>
@@ -725,6 +739,36 @@
                 <plugins>
                     <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-gpg-plugin</artifactId>
+                        <configuration>
+                            <skip>false</skip>
+                        </configuration>
+                    </plugin>
+
+                    <!-- Override the parent assembly execution to customize the assembly
+              descriptor and final name. -->
+                    <plugin>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>source-release-assembly</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <tarLongFileMode>posix</tarLongFileMode>
+                                    <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
+                                    <appendAssemblyId>true</appendAssemblyId>
+                                    <descriptor>src/main/config/assemblies/source-assembly.xml</descriptor>
+                                    <finalName>apache-kylin-${project.version}-src</finalName>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-surefire-plugin</artifactId>
                         <version>2.16</version>
                         <configuration>
@@ -779,12 +823,18 @@
                                 <exclude>**/*.pdf</exclude>
                                 <exclude>**/*.log</exclude>
 
+                                <exclude>**/.checkstyle</exclude>
+                                <!--Job's Test Data-->
+                                <exclude>**/src/test/resources/data/**</exclude>
+
                                 <!-- generated files -->
                                 <exclude>**/target/**</exclude>
                                 <exclude>lib/**</exclude>
                                 <!-- Kylin's website content -->
                                 <exclude>**/.sass-cache/**</exclude>
-                                <exclude>docs/website/**</exclude>
+
+                                <exclude>website/**</exclude>
+
                                 <!-- tomcat package -->
                                 <exclude>tomcat/**</exclude>
                                 <!-- front end libary and generated files -->
@@ -802,8 +852,13 @@
                                 <exclude>webapp/app/css/AdminLTE.css</exclude>
 
                                 <!-- jdbc log -->
-                                <exclude>jdbc/kylin_jdbc.log.*</exclude>
-                                <exclude>jdbc/kylin_jdbc.*</exclude>
+                                <exclude>jdbc/kylin_jdbc.log*</exclude>
+                                <!-- server log -->
+                                <exclude>server/logs/**</exclude>
+
+                                <!-- HBase MiniCluster Testing Data, for testing only -->
+                                <exclude>examples/test_case_data/minicluster/hbase-export.tar.gz</exclude>
+                                <exclude>examples/test_case_data/**/*.xml</exclude>
 
                             </excludes>
                         </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 39d8faa..0f7267f 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 5ab8bc3..30743c3 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/source-hive/pom.xml
----------------------------------------------------------------------
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 29780e8..04eb7f0 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -27,7 +27,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/source-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/source-kafka/pom.xml b/source-kafka/pom.xml
index e75db65..3693771 100644
--- a/source-kafka/pom.xml
+++ b/source-kafka/pom.xml
@@ -11,7 +11,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
new file mode 100644
index 0000000..96a56a1
--- /dev/null
+++ b/src/main/config/assemblies/source-assembly.xml
@@ -0,0 +1,107 @@
+<?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.
+-->
+<assembly>
+    <id>src</id>
+    <formats>
+        <format>tar.gz</format>
+    </formats>
+    
+    <fileSets>
+        <!-- main project directory structure -->
+        <fileSet>
+            <directory>.</directory>
+            <outputDirectory>/</outputDirectory>
+            <useDefaultExcludes>true</useDefaultExcludes>
+            <excludes>
+                <!-- build output -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]
+                </exclude>
+
+                <!-- NOTE: Most of the following excludes should not be required
+                  if the standard release process is followed. This is because the release
+                  plugin checks out project sources into a location like target/checkout, then
+                  runs the build from there. The result is a source-release archive that comes
+                  from a pretty clean directory structure. HOWEVER, if the release plugin is
+                  configured to run extra goals or generate a project website, it's definitely
+                  possible that some of these files will be present. So, it's safer to exclude
+                  them. -->
+
+                <!-- IDEs -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.idea(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?out(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]
+                </exclude>
+
+
+                <!-- scm -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.gitignore(/.*)?]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs/website(/.*)?]
+                </exclude>
+
+                <!-- release-plugin temp files -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.next]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.tag]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
+                </exclude>
+
+            </excludes>
+        </fileSet>
+        <!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->
+        <fileSet>
+            <directory>${project.build.directory}/maven-shared-archive-resources/META-INF
+            </directory>
+            <outputDirectory>/</outputDirectory>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}</directory>
+            <includes>
+                <include>git.properties</include>
+            </includes>
+            <outputDirectory>/</outputDirectory>
+        </fileSet>
+    </fileSets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/src/main/config/checkstyle/checkstyle-java-header.txt
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/checkstyle-java-header.txt b/src/main/config/checkstyle/checkstyle-java-header.txt
new file mode 100644
index 0000000..c486fd3
--- /dev/null
+++ b/src/main/config/checkstyle/checkstyle-java-header.txt
@@ -0,0 +1,17 @@
+/*
+ * 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.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/src/main/config/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/checkstyle.xml b/src/main/config/checkstyle/checkstyle.xml
new file mode 100644
index 0000000..eab23dc
--- /dev/null
+++ b/src/main/config/checkstyle/checkstyle.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<!--
+  This version of checkstyle is based on the Apache Giraph checkstyle
+  configuration, which in turn is based on Hadoop and common-math
+  configurations.
+
+  The documentation for checkstyle is available at
+
+  http://checkstyle.sourceforge.net
+-->
+
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.1//EN" "http://www.puppycrawl.com/dtds/configuration_1_1.dtd">
+
+<module name="Checker">
+    <property name="localeLanguage" value="en"/>
+    <property name="charset" value="UTF-8"/>
+
+
+    <!-- Checks for headers -->
+    <!-- See http://checkstyle.sf.net/config_header.html -->
+    <!-- Verify that EVERY source file has the appropriate license -->
+    <module name="Header">
+        <property name="headerFile" value="${checkstyle.header.file}"/>
+        <property name="fileExtensions" value="java"/>
+    </module>
+
+</module>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/src/main/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
new file mode 100644
index 0000000..c6edc5f
--- /dev/null
+++ b/src/main/config/checkstyle/suppressions.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+    <!-- Suppress javadoc, line length, visibility checks for test code -->
+    <suppress checks="JavadocStyleCheck" files="Test*.java"/>
+    <suppress checks="JavadocTypeCheck" files="Test*.java"/>
+    <suppress checks="LineLength" files="Test*.java"/>
+    <suppress checks="Indentation" files="Test*.java"/>
+    <suppress checks="VisibilityModifier" files="Test*.java"/>
+
+</suppressions>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/storage-hbase/pom.xml b/storage-hbase/pom.xml
index 9bc7fe8..76992cb 100644
--- a/storage-hbase/pom.xml
+++ b/storage-hbase/pom.xml
@@ -28,7 +28,7 @@
     <parent>
         <groupId>org.apache.kylin</groupId>
         <artifactId>kylin</artifactId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2067c46/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 7bbb54a..2f430df 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -5,7 +5,7 @@
     <parent>
         <artifactId>kylin</artifactId>
         <groupId>org.apache.kylin</groupId>
-        <version>0.8.1-incubating-SNAPSHOT</version>
+        <version>2.0-incubating-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 


[04/28] incubator-kylin git commit: KYLIN-973 add a tool to analyse streaming logs

Posted by ma...@apache.org.
KYLIN-973 add a tool to analyse streaming logs


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

Branch: refs/heads/2.x-staging
Commit: b65d8c0b7aa2659d155c387c4a87049ce83d0c16
Parents: 08845a1
Author: honma <ho...@ebay.com>
Authored: Fri Aug 28 17:16:34 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 1 14:14:03 2015 +0800

----------------------------------------------------------------------
 .../kylin/job/tools/StreamingLogsAnalyser.java  | 96 ++++++++++++++++++++
 1 file changed, 96 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b65d8c0b/job/src/main/java/org/apache/kylin/job/tools/StreamingLogsAnalyser.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/StreamingLogsAnalyser.java b/job/src/main/java/org/apache/kylin/job/tools/StreamingLogsAnalyser.java
new file mode 100644
index 0000000..9d83d01
--- /dev/null
+++ b/job/src/main/java/org/apache/kylin/job/tools/StreamingLogsAnalyser.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.job.tools;
+
+import java.io.File;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.FileUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class StreamingLogsAnalyser {
+    public static void main(String[] args) {
+        int errorFileCount = 0;
+        List<Long> ellapsedTimes = Lists.newArrayList();
+
+        String patternStr = "(\\d{2}/\\d{2}/\\d{2} \\d{2}:\\d{2}:\\d{2})";
+        Pattern pattern = Pattern.compile(patternStr);
+
+        SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone("GMT")); // NOTE: this must be GMT to calculate epoch date correctly
+
+        Preconditions.checkArgument(args.length == 1, "Usage: StreamingLogsAnalyser streaming_logs_folder");
+        for (File file : FileUtils.listFiles(new File(args[0]), new String[] { "log" }, false)) {
+            System.out.println("Processing file " + file.toString());
+
+            long startTime = 0;
+            long endTime = 0;
+            try {
+                List<String> contents = Files.readAllLines(file.toPath(), Charset.defaultCharset());
+                for (int i = 0; i < contents.size(); ++i) {
+                    Matcher m = pattern.matcher(contents.get(i));
+                    if (m.find()) {
+                        startTime = format.parse("20" + m.group(1)).getTime();
+                        break;
+                    }
+                }
+
+                for (int i = contents.size() - 1; i >= 0; --i) {
+                    Matcher m = pattern.matcher(contents.get(i));
+                    if (m.find()) {
+                        endTime = format.parse("20" + m.group(1)).getTime();
+                        break;
+                    }
+                }
+
+                if (startTime == 0 || endTime == 0) {
+                    throw new RuntimeException("start time or end time is not found");
+                }
+
+                if (endTime - startTime < 60000) {
+                    System.out.println("Warning: this job took less than one minute!!!! " + file.toString());
+                }
+
+                ellapsedTimes.add(endTime - startTime);
+
+            } catch (Exception e) {
+                System.out.println("Exception when processing log file " + file.toString());
+                System.out.println(e);
+                errorFileCount++;
+            }
+        }
+
+        System.out.println("Totally error files count " + errorFileCount);
+        System.out.println("Totally normal files processed " + ellapsedTimes.size());
+
+        long sum = 0;
+        for (Long x : ellapsedTimes) {
+            sum += x;
+        }
+        System.out.println("Avg build time " + (sum / ellapsedTimes.size()));
+    }
+}


[15/28] incubator-kylin git commit: KYLIN-992

Posted by ma...@apache.org.
KYLIN-992


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

Branch: refs/heads/2.x-staging
Commit: b5c9722bb5cc5f2c2a66eac3080fcfb3691103a4
Parents: eb0652a
Author: qianhao.zhou <qi...@ebay.com>
Authored: Wed Sep 2 17:19:24 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Fri Sep 4 22:04:36 2015 +0800

----------------------------------------------------------------------
 .../kylin/common/persistence/ResourceStore.java | 50 ++++++++++----------
 1 file changed, 25 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b5c9722b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index ba61c37..b6ba550 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -72,35 +72,35 @@ abstract public class ResourceStore {
         }
         return knownImpl;
     }
-
-    public static ResourceStore getStore(KylinConfig kylinConfig) {
-        ResourceStore r = CACHE.get(kylinConfig);
+    
+    private static ResourceStore createResourceStore(KylinConfig 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 : getKnownImpl()) {
-                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());
+        logger.info("Using metadata url " + kylinConfig.getMetadataUrl() + " for resource store");
+        for (Class<? extends ResourceStore> cls : getKnownImpl()) {
+            try {
+                return cls.getConstructor(KylinConfig.class).newInstance(kylinConfig);
+            } catch (Throwable e) {
+                es.add(e);
             }
+        }
+        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);
+    public static ResourceStore getStore(KylinConfig kylinConfig) {
+        if (CACHE.containsKey(kylinConfig)) {
+            return CACHE.get(kylinConfig);
+        }
+        synchronized (ResourceStore.class) {
+            if (CACHE.containsKey(kylinConfig)) {
+                return CACHE.get(kylinConfig);
+            } else {
+                CACHE.putIfAbsent(kylinConfig, createResourceStore(kylinConfig));
+            }
         }
-        return r;
+        return CACHE.get(kylinConfig);
     }
 
     // ============================================================================


[02/28] incubator-kylin git commit: KYLIN-956 set gzip as default compression for test case and snappy for binary tar

Posted by ma...@apache.org.
KYLIN-956 set gzip as default compression for test case and snappy for binary tar


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

Branch: refs/heads/2.x-staging
Commit: c73f5dea4171a98a5daf3275e4feed921909a083
Parents: 42fa50d
Author: honma <ho...@ebay.com>
Authored: Mon Aug 31 11:06:08 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 1 14:14:03 2015 +0800

----------------------------------------------------------------------
 bin/setenv.sh                                   |  4 ++--
 .../engine/spark/BuildCubeWithSparkTest.java    | 17 ++++++++-------
 .../test_case_data/localmeta/kylin.properties   |  2 +-
 .../localmeta/kylin_hive_conf.xml               |  2 ++
 .../test_case_data/localmeta/kylin_job_conf.xml |  4 ++--
 .../test_case_data/sandbox/kylin.properties     |  2 +-
 .../test_case_data/sandbox/kylin_hive_conf.xml  |  4 +++-
 .../test_case_data/sandbox/kylin_job_conf.xml   |  3 ++-
 examples/test_case_data/sandbox/mapred-site.xml |  2 +-
 .../kylin/job/BuildCubeWithStreamTest.java      |  1 -
 .../apache/kylin/job/BuildIIWithEngineTest.java |  1 -
 .../apache/kylin/job/BuildIIWithStreamTest.java |  2 +-
 .../kylin/storage/hbase/steps/BulkLoadJob.java  |  1 +
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java | 22 +++++++-------------
 14 files changed, 33 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/bin/setenv.sh
----------------------------------------------------------------------
diff --git a/bin/setenv.sh b/bin/setenv.sh
index 5acb287..f4cce2d 100755
--- a/bin/setenv.sh
+++ b/bin/setenv.sh
@@ -18,7 +18,7 @@ then
     echo "KYLIN_JVM_SETTINGS is ${KYLIN_JVM_SETTINGS}"
     KYLIN_EXTRA_START_OPTS="${KYLIN_JVM_SETTINGS} ${KYLIN_EXTRA_START_OPTS}"
 else
-    echo "KYLIN_JVM_SETTINGS is not set, using default jvm settings"
+    echo "KYLIN_JVM_SETTINGS is not set, using default jvm settings: ${KYLIN_JVM_SETTINGS}"
 fi
 
 if [ ! -z "${KYLIN_DEBUG_SETTINGS}" ]
@@ -34,5 +34,5 @@ then
     echo "KYLIN_LD_LIBRARY_SETTINGS is ${KYLIN_LD_LIBRARY_SETTINGS}"
     KYLIN_EXTRA_START_OPTS="${KYLIN_LD_LIBRARY_SETTINGS} ${KYLIN_EXTRA_START_OPTS}"
 else
-    echo "KYLIN_LD_LIBRARY_SETTINGS is not set, Usually it's okay unless you want to enable LZO compression, etc. "
+    echo "KYLIN_LD_LIBRARY_SETTINGS is not set, Usually it's okay unless you want to specify your own native path"
 fi

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/engine-spark/src/test/java/org/apache/kylin/engine/spark/BuildCubeWithSparkTest.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/test/java/org/apache/kylin/engine/spark/BuildCubeWithSparkTest.java b/engine-spark/src/test/java/org/apache/kylin/engine/spark/BuildCubeWithSparkTest.java
index 178b10f..d24cc79 100644
--- a/engine-spark/src/test/java/org/apache/kylin/engine/spark/BuildCubeWithSparkTest.java
+++ b/engine-spark/src/test/java/org/apache/kylin/engine/spark/BuildCubeWithSparkTest.java
@@ -18,7 +18,13 @@
 
 package org.apache.kylin.engine.spark;
 
-import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.TimeZone;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.kylin.common.KylinConfig;
@@ -42,12 +48,7 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.List;
-import java.util.TimeZone;
-
-import static org.junit.Assert.assertEquals;
+import com.google.common.collect.Lists;
 
 public class BuildCubeWithSparkTest {
 
@@ -120,7 +121,7 @@ public class BuildCubeWithSparkTest {
         waitForJob(cubingJob.getId());
         assertEquals(ExecutableState.SUCCEED, jobService.getOutput(cubingJob.getId()).getState());
     }
-    
+
     private void clearSegment(String cubeName) throws Exception {
         CubeInstance cube = cubeManager.getCube(cubeName);
         // remove all existing segments

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/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
index 6240862..bdd6113 100644
--- a/examples/test_case_data/localmeta/kylin.properties
+++ b/examples/test_case_data/localmeta/kylin.properties
@@ -37,7 +37,7 @@ kylin.job.concurrent.max.limit=10
 kylin.job.yarn.app.rest.check.interval.seconds=10
 
 #default compression codec for htable,snappy,lzo,gzip,lz4
-kylin.hbase.default.compression.codec=snappy
+kylin.hbase.default.compression.codec=gzip
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/examples/test_case_data/localmeta/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin_hive_conf.xml b/examples/test_case_data/localmeta/kylin_hive_conf.xml
index 821b6ee..3cb5242 100644
--- a/examples/test_case_data/localmeta/kylin_hive_conf.xml
+++ b/examples/test_case_data/localmeta/kylin_hive_conf.xml
@@ -24,6 +24,7 @@
         <description>enable map-side join</description>
     </property>
 
+    <!--
     <property>
         <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
@@ -34,6 +35,7 @@
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
+    -->
     
     <property>
         <name>hive.merge.mapfiles</name>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/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
index 0f5bf30..10dfdb7 100644
--- a/examples/test_case_data/localmeta/kylin_job_conf.xml
+++ b/examples/test_case_data/localmeta/kylin_job_conf.xml
@@ -11,6 +11,7 @@
     </property>
 
 
+    <!--
     <property>
         <name>mapred.compress.map.output</name>
         <value>true</value>
@@ -42,8 +43,7 @@
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
-
-
+    -->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/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
index ef0c288..ce599ea 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -49,7 +49,7 @@ kylin.job.hbase.region.cut.large=50
 kylin.job.yarn.app.rest.check.interval.seconds=10
 
 #default compression codec for htable,snappy,lzo,gzip,lz4
-kylin.hbase.default.compression.codec=snappy
+kylin.hbase.default.compression.codec=gzip
 
 ## Config for Restful APP ##
 # database connection settings:

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/examples/test_case_data/sandbox/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_hive_conf.xml b/examples/test_case_data/sandbox/kylin_hive_conf.xml
index eab0399..faa00d9 100644
--- a/examples/test_case_data/sandbox/kylin_hive_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_hive_conf.xml
@@ -24,6 +24,7 @@
         <description>enable map-side join</description>
     </property>
 
+    <!--
     <property>
         <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
@@ -34,7 +35,8 @@
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
-   
+    -->
+
     <property>
         <name>hive.merge.mapfiles</name>
         <value>true</value>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/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
index 2d1fe10..69def6e 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf.xml
@@ -11,6 +11,7 @@
     </property>
 
 
+    <!--
     <property>
         <name>mapred.compress.map.output</name>
         <value>true</value>
@@ -42,7 +43,7 @@
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
-
+-->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/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
index f2b49c5..611bf8b 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -43,7 +43,7 @@
 
     <property>
         <name>mapreduce.application.classpath</name>
-        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,/usr/hdp/${hdp.version}/hive/lib/hive-metastore.jar,/usr/hdp/${hdp.version}/hive/lib/hive-exec.jar,/usr/hdp/${hdp.version}/hive-hcatalog/share/hcatalog/*,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/etc/hadoop/conf/secure</value>
+        <value>/tmp/kylin/*,$HADOOP_CONF_DIR,/usr/hdp/${hdp.version}/hbase/lib/hbase-common.jar,/usr/hdp/current/hive-client/conf/,/usr/hdp/${hdp.version}/hive/lib/hive-metastore.jar,/usr/hdp/${hdp.version}/hive/lib/hive-exec.jar,/usr/hdp/${hdp.version}/hive-hcatalog/share/hcatalog/*,$PWD/mr-framework/hadoop/share/hadoop/mapreduce/*:$PWD/mr-framework/hadoop/share/hadoop/mapreduce/lib/*:$PWD/mr-framework/hadoop/share/hadoop/common/*:$PWD/mr-framework/hadoop/share/hadoop/common/lib/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/*:$PWD/mr-framework/hadoop/share/hadoop/yarn/lib/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/*:$PWD/mr-framework/hadoop/share/hadoop/hdfs/lib/*:/usr/hdp/${hdp.version}/hadoop/lib/hadoop-lzo-0.6.0.${hdp.version}.jar:/usr/hdp/${hdp.version}/hadoop/lib/snappy-java-1.0.4.1.jar:/etc/hadoop/conf/secure</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
index d3f84a0..87b684e 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
@@ -75,7 +75,6 @@ public class BuildCubeWithStreamTest {
         ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
         System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");
         System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
-
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
index 1cf1a0f..ff88667 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithEngineTest.java
@@ -237,7 +237,6 @@ public class BuildIIWithEngineTest {
         return exitCode;
     }
 
-
     public static void main(String[] args) throws Exception {
         BuildIIWithEngineTest instance = new BuildIIWithEngineTest();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index 360ad8b..e3a12ea 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -120,7 +120,7 @@ public class BuildIIWithStreamTest {
         JobEngineConfig jobEngineConfig = new JobEngineConfig(kylinConfig);
         final String uuid = UUID.randomUUID().toString();
         final String dropTableHql = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc);
-        final String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, JobBuilderSupport.getJobWorkingDir(jobEngineConfig,uuid));
+        final String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, JobBuilderSupport.getJobWorkingDir(jobEngineConfig, uuid));
         String insertDataHqls;
         try {
             insertDataHqls = JoinedFlatTable.generateInsertDataStatement(intermediateTableDesc, jobEngineConfig);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
index 5059a37..ec4a8a9 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
@@ -48,6 +48,7 @@ public class BulkLoadJob extends AbstractHadoopJob {
         Options options = new Options();
 
         try {
+            
             options.addOption(OPTION_INPUT_PATH);
             options.addOption(OPTION_HTABLE_NAME);
             options.addOption(OPTION_CUBE_NAME);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c73f5dea/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
index 7d45fd6..f717d82 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
@@ -51,10 +51,12 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
+@Ignore
 public class TestFuzzyRowFilterV2EndToEnd {
     private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
     private final static byte fuzzyValue = (byte) 63;
@@ -75,8 +77,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
     public static void setUpBeforeClass() throws Exception {
         Configuration conf = TEST_UTIL.getConfiguration();
         conf.setInt("hbase.client.scanner.caching", 1000);
-        conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
-                ConstantSizeRegionSplitPolicy.class.getName());
+        conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, ConstantSizeRegionSplitPolicy.class.getName());
         // set no splits
         conf.setLong(HConstants.HREGION_MAX_FILESIZE, ((long) 1024) * 1024 * 1024 * 10);
 
@@ -111,8 +112,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
     public void testEndToEnd() throws Exception {
         String cf = "f";
 
-        HTable ht =
-                TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE);
+        HTable ht = TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE);
 
         // 10 byte row key - (2 bytes 4 bytes 4 bytes)
         // 4 byte qualifier
@@ -250,8 +250,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
     public void testFilterList() throws Exception {
         String cf = "f";
         String table = "TestFuzzyRowFiltersInFilterList";
-        HTable ht =
-                TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE);
+        HTable ht = TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE);
 
         // 10 byte row key - (2 bytes 4 bytes 4 bytes)
         // 4 byte qualifier
@@ -276,8 +275,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
                     p.setDurability(Durability.SKIP_WAL);
                     p.add(cf.getBytes(), cq, Bytes.toBytes(c));
                     ht.put(p);
-                    LOG.info("Inserting: rk: " + Bytes.toStringBinary(rk) + " cq: "
-                            + Bytes.toStringBinary(cq));
+                    LOG.info("Inserting: rk: " + Bytes.toStringBinary(rk) + " cq: " + Bytes.toStringBinary(cq));
                 }
             }
         }
@@ -320,8 +318,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
         runScanner(hTable, expectedSize, fuzzyRowFilter1, fuzzyRowFilter2);
     }
 
-    private void runScanner(HTable hTable, int expectedSize, Filter filter1, Filter filter2)
-            throws IOException {
+    private void runScanner(HTable hTable, int expectedSize, Filter filter1, Filter filter2) throws IOException {
         String cf = "f";
         Scan scan = new Scan();
         scan.addFamily(cf.getBytes());
@@ -334,8 +331,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
         long timeBeforeScan = System.currentTimeMillis();
         while ((result = scanner.next()) != null) {
             for (Cell kv : result.listCells()) {
-                LOG.info("Got rk: " + Bytes.toStringBinary(CellUtil.cloneRow(kv)) + " cq: "
-                        + Bytes.toStringBinary(CellUtil.cloneQualifier(kv)));
+                LOG.info("Got rk: " + Bytes.toStringBinary(CellUtil.cloneRow(kv)) + " cq: " + Bytes.toStringBinary(CellUtil.cloneQualifier(kv)));
                 results.add(kv);
             }
         }
@@ -348,5 +344,3 @@ public class TestFuzzyRowFilterV2EndToEnd {
         assertEquals(expectedSize, results.size());
     }
 }
-
-


[03/28] incubator-kylin git commit: KYLIN-977 adjust echcache settings

Posted by ma...@apache.org.
KYLIN-977 adjust echcache settings


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

Branch: refs/heads/2.x-staging
Commit: 42fa50d93049ccb6d0877b1961193d18226f48dc
Parents: b65d8c0
Author: honma <ho...@ebay.com>
Authored: Sat Aug 29 18:36:50 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 1 14:14:03 2015 +0800

----------------------------------------------------------------------
 server/src/main/resources/ehcache.xml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/42fa50d9/server/src/main/resources/ehcache.xml
----------------------------------------------------------------------
diff --git a/server/src/main/resources/ehcache.xml b/server/src/main/resources/ehcache.xml
index f8fc68d..29fce05 100644
--- a/server/src/main/resources/ehcache.xml
+++ b/server/src/main/resources/ehcache.xml
@@ -3,7 +3,7 @@
            eternal="false"
            timeToIdleSeconds="86400"
            memoryStoreEvictionPolicy="LRU"
-           maxBytesLocalHeap="500M"
+           maxBytesLocalHeap="50M"
             >
         <persistence strategy="none"/>
     </cache>
@@ -11,7 +11,7 @@
            eternal="false"
            timeToIdleSeconds="86400"
            memoryStoreEvictionPolicy="LRU"
-           maxBytesLocalHeap="100M"
+           maxBytesLocalHeap="50M"
             >
         <persistence strategy="none"/>
     </cache>
@@ -19,7 +19,7 @@
            eternal="false"
            timeToLiveSeconds="10800"
            memoryStoreEvictionPolicy="LRU"
-           maxBytesLocalHeap="100M"
+           maxBytesLocalHeap="50M"
             >
         <persistence strategy="none"/>
     </cache>


[09/28] incubator-kylin git commit: hot fix: add host name in streaming gap notification mail

Posted by ma...@apache.org.
hot fix: add host name in streaming gap notification mail


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

Branch: refs/heads/2.x-staging
Commit: 730fab04ff0b1247944e0a4805496086e0f8ef47
Parents: 8ae2c0f
Author: honma <ho...@ebay.com>
Authored: Wed Sep 2 10:31:43 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Wed Sep 2 10:31:43 2015 +0800

----------------------------------------------------------------------
 job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java   | 2 +-
 .../main/java/org/apache/kylin/job/monitor/StreamingMonitor.java | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/730fab04/job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java b/job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java
index 2b3314b..7b9831a 100644
--- a/job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/monitor/MonitorCLI.java
@@ -62,7 +62,7 @@ public class MonitorCLI {
         }
         if (cubeName != null) {
             logger.info(String.format("check cube cubeName:%s receivers:%s", cubeName, StringUtils.join(receivers, ";")));
-            streamingMonitor.checkCube(receivers, cubeName);
+            streamingMonitor.checkCube(receivers, cubeName,host);
         }
         System.exit(0);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/730fab04/job/src/main/java/org/apache/kylin/job/monitor/StreamingMonitor.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/monitor/StreamingMonitor.java b/job/src/main/java/org/apache/kylin/job/monitor/StreamingMonitor.java
index a56ec70..e23f065 100644
--- a/job/src/main/java/org/apache/kylin/job/monitor/StreamingMonitor.java
+++ b/job/src/main/java/org/apache/kylin/job/monitor/StreamingMonitor.java
@@ -113,7 +113,7 @@ public class StreamingMonitor {
         return overlaps;
     }
 
-    public void checkCube(List<String> receivers, String cubeName) {
+    public void checkCube(List<String> receivers, String cubeName, String host) {
         final CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).reloadCubeLocal(cubeName);
         if (cube == null) {
             logger.info("cube:" + cubeName + " does not exist");
@@ -136,7 +136,7 @@ public class StreamingMonitor {
         }
         if (content.length() > 0) {
             logger.info(content.toString());
-            sendMail(receivers, String.format("%s has gaps or overlaps", cubeName), content.toString());
+            sendMail(receivers, String.format("%s has gaps or overlaps on host %s", cubeName, host), content.toString());
         } else {
             logger.info("no gaps or overlaps");
         }


[11/28] incubator-kylin git commit: KYLIN-972 MR_V1 build well and pass query test

Posted by ma...@apache.org.
KYLIN-972 MR_V1 build well and pass query test


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

Branch: refs/heads/2.x-staging
Commit: a79db0562d8fcf299ea1ecd19ec701258fd7a25d
Parents: d501dc4
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Sep 2 15:31:34 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Sep 2 15:32:38 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    | 18 +++++---
 .../kylin/common/persistence/ResourceTool.java  |  2 +-
 .../org/apache/kylin/source/SourceFactory.java  |  2 +-
 .../engine/mr/steps/BaseCuboidMapperBase.java   |  4 +-
 .../engine/mr/steps/HiveToBaseCuboidMapper.java |  2 +-
 .../apache/kylin/source/hive/HiveMRInput.java   |  5 ++-
 .../apache/kylin/source/hive/HiveSource.java    | 44 +++++++++++++++++++
 .../kylin/source/hive/HiveTableSource.java      | 45 --------------------
 8 files changed, 64 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 4191f8e..06deb55 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -30,6 +30,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import com.google.common.collect.Sets;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.restclient.RestClient;
@@ -42,10 +43,14 @@ import java.util.Map;
 import java.util.Properties;
 
 /**
- * @author yangli9
  */
+@SuppressWarnings("serial")
 public class KylinConfig implements Serializable {
 
+    /*
+     * NOTE: These key constants should be private or even better be removed.
+     *       All external access should go through public methods.
+     */
     public static final String KYLIN_STORAGE_URL = "kylin.storage.url";
 
     public static final String KYLIN_METADATA_URL = "kylin.metadata.url";
@@ -287,8 +292,6 @@ public class KylinConfig implements Serializable {
         }
         return root + getMetadataUrlPrefix() + "/";
     }
-    
-    
 
     public String getKylinJobLogDir() {
         return getOptional(KYLIN_JOB_LOG_DIR, "/tmp/kylin/logs");
@@ -497,11 +500,15 @@ public class KylinConfig implements Serializable {
         percent = Math.min(percent, 100);
         return percent;
     }
-    
+
     public String getHbaseDefaultCompressionCodec() {
         return getOptional(HTABLE_DEFAULT_COMPRESSION_CODEC);
     }
 
+    public boolean isHiveKeepFlatTable() {
+        return Boolean.parseBoolean(this.getOptional("kylin.hive.keep.flat.table", "false"));
+    }
+
     private String getOptional(String prop) {
         final String property = System.getProperty(prop);
         return property != null ? property : properties.getProperty(prop);
@@ -512,7 +519,7 @@ public class KylinConfig implements Serializable {
         if (!StringUtils.isBlank(property)) {
             return property.split("\\s*,\\s*");
         } else {
-            return new String[]{};
+            return new String[] {};
         }
     }
 
@@ -625,7 +632,6 @@ public class KylinConfig implements Serializable {
             }
         }
 
-
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
index 0b766fb..7e89c21 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceTool.java
@@ -50,7 +50,7 @@ public class ResourceTool {
             reset(args.length == 1 ? KylinConfig.getInstanceFromEnv() : KylinConfig.createInstanceFromUri(args[1]));
             break;
         case "list":
-            list(KylinConfig.getInstanceFromEnv(), args[2]);
+            list(KylinConfig.getInstanceFromEnv(), args[1]);
             break;
         case "download":
             copy(KylinConfig.getInstanceFromEnv(), KylinConfig.createInstanceFromUri(args[1]));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java b/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
index 2fbf847..a771f5e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
@@ -31,7 +31,7 @@ public class SourceFactory {
     private static ImplementationSwitch sources;
     static {
         Map<Integer, String> impls = new HashMap<>();
-        impls.put(ID_HIVE, "org.apache.kylin.source.hive.HiveTableSource");
+        impls.put(ID_HIVE, "org.apache.kylin.source.hive.HiveSource");
         sources = new ImplementationSwitch(impls);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
index 0e22243..c2d0f92 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -189,9 +189,9 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
 
     protected void handleErrorRecord(BytesSplitter bytesSplitter, Exception ex) throws IOException {
 
-        System.err.println("Insane record: " + bytesSplitter);
-        ex.printStackTrace(System.err);
+        logger.error("Insane record: " + bytesSplitter, ex);
 
+        // TODO expose errorRecordCounter as hadoop counter
         errorRecordCounter++;
         if (errorRecordCounter > BatchConstants.ERROR_RECORD_THRESHOLD) {
             if (ex instanceof IOException)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
index a1b95c7..e2a49df 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/HiveToBaseCuboidMapper.java
@@ -60,7 +60,7 @@ public class HiveToBaseCuboidMapper<KEYIN> extends BaseCuboidMapperBase<KEYIN, O
     private byte[][] convertUTF8Bytes(String[] row) throws UnsupportedEncodingException {
         byte[][] result = new byte[row.length][];
         for (int i = 0; i < row.length; i++) {
-            result[i] = row[i].getBytes("UTF-8");
+            result[i] = row[i] == null ? HIVE_NULL : row[i].getBytes("UTF-8");
         }
         return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index ec9b432..b4edd2e 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -152,15 +152,16 @@ public class HiveMRInput implements IMRInput {
 
         @Override
         protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+            KylinConfig config = context.getConfig();
             StringBuffer output = new StringBuffer();
 
             final String hiveTable = this.getIntermediateTableIdentity();
-            if (StringUtils.isNotEmpty(hiveTable)) {
+            if (config.isHiveKeepFlatTable() == false && StringUtils.isNotEmpty(hiveTable)) {
                 final String dropSQL = "DROP TABLE IF EXISTS  " + hiveTable + ";";
                 final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
                 ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
                 try {
-                    context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
+                    config.getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
                     output.append("Hive table " + hiveTable + " is dropped. \n");
                 } catch (IOException e) {
                     logger.error("job:" + getId() + " execute finished with exception", e);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSource.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSource.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSource.java
new file mode 100644
index 0000000..b7dbff0
--- /dev/null
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSource.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source.hive;
+
+import org.apache.kylin.engine.mr.IMRInput;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.source.ISource;
+import org.apache.kylin.source.ReadableTable;
+
+//used by reflection
+public class HiveSource implements ISource {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        if (engineInterface == IMRInput.class) {
+            return (I) new HiveMRInput();
+        } else {
+            throw new RuntimeException("Cannot adapt to " + engineInterface);
+        }
+    }
+
+    @Override
+    public ReadableTable createReadableTable(TableDesc tableDesc) {
+        return new HiveTable(tableDesc);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a79db056/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
deleted file mode 100644
index 2dbe071..0000000
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.source.hive;
-
-import org.apache.kylin.engine.mr.IMRInput;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.ISource;
-import org.apache.kylin.source.ReadableTable;
-
-@SuppressWarnings("unused")
-//used by reflection
-public class HiveTableSource implements ISource {
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
-        if (engineInterface == IMRInput.class) {
-            return (I) new HiveMRInput();
-        } else {
-            throw new RuntimeException("Cannot adapt to " + engineInterface);
-        }
-    }
-
-    @Override
-    public ReadableTable createReadableTable(TableDesc tableDesc) {
-        return new HiveTable(tableDesc);
-    }
-
-}


[21/28] incubator-kylin git commit: KYLIN-1005

Posted by ma...@apache.org.
KYLIN-1005


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

Branch: refs/heads/2.x-staging
Commit: ec9bbb96ea0bef73f19f12f2419144037a5e47c5
Parents: c799073
Author: qianhao.zhou <qi...@ebay.com>
Authored: Sun Sep 6 19:56:37 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Sun Sep 6 19:57:04 2015 +0800

----------------------------------------------------------------------
 .../storage/hbase/util/ZookeeperJobLock.java    | 23 ++++++++++++++++----
 1 file changed, 19 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ec9bbb96/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperJobLock.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperJobLock.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperJobLock.java
index a96f639..ab15b72 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperJobLock.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperJobLock.java
@@ -1,7 +1,10 @@
 package org.apache.kylin.storage.hbase.util;
 
+import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
@@ -17,6 +20,8 @@ import org.apache.kylin.storage.hbase.steps.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 /**
  */
 public class ZookeeperJobLock implements JobLock {
@@ -31,13 +36,15 @@ public class ZookeeperJobLock implements JobLock {
     @Override
     public boolean lock() {
         this.scheduleID = schedulerId();
-        String ZKConnectString = getZKConnectString();
-        if (StringUtils.isEmpty(ZKConnectString)) {
+        String zkConnectString = getZKConnectString();
+        logger.info("zk connection string:" + zkConnectString);
+        logger.info("schedulerId:" + scheduleID);
+        if (StringUtils.isEmpty(zkConnectString)) {
             throw new IllegalArgumentException("ZOOKEEPER_QUORUM is empty!");
         }
 
         RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
-        this.zkClient = CuratorFrameworkFactory.newClient(ZKConnectString, retryPolicy);
+        this.zkClient = CuratorFrameworkFactory.newClient(zkConnectString, retryPolicy);
         this.zkClient.start();
         this.sharedLock = new InterProcessMutex(zkClient, this.scheduleID);
         boolean hasLock = false;
@@ -61,7 +68,15 @@ public class ZookeeperJobLock implements JobLock {
 
     private String getZKConnectString() {
         Configuration conf = HBaseConnection.newHBaseConfiguration(KylinConfig.getInstanceFromEnv().getStorageUrl());
-        return conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+        final String serverList = conf.get(HConstants.ZOOKEEPER_QUORUM);
+        final String port = conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
+        return org.apache.commons.lang3.StringUtils.join(Iterables.transform(Arrays.asList(serverList.split(",")), new Function<String, String>() {
+            @Nullable
+            @Override
+            public String apply(String input) {
+                return input + ":" + port;
+            }
+        }), ",");
     }
 
     private void releaseLock() {


[22/28] incubator-kylin git commit: KYLIN-957 prepare for upcoming changes

Posted by ma...@apache.org.
KYLIN-957 prepare for upcoming changes


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

Branch: refs/heads/2.x-staging
Commit: 0eddcf5a8d1125df77ac7b583782b900592908ac
Parents: ec9bbb9
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 21:29:17 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 21:31:59 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0eddcf5a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
index ec4a8a9..29e9457 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
@@ -71,7 +71,7 @@ public class BulkLoadJob extends AbstractHadoopJob {
             FsPermission permission = new FsPermission((short) 0777);
             for (HBaseColumnFamilyDesc cf : cubeDesc.getHBaseMapping().getColumnFamily()) {
                 String cfName = cf.getName();
-                Path columnFamilyPath = new Path(input + cfName);
+                Path columnFamilyPath = new Path(input, cfName);
 
                 // File may have already been auto-loaded (in the case of MapR DB)
                 if (fs.exists(columnFamilyPath)) {


[27/28] incubator-kylin git commit: KYLIN-910, add auto_merge_time_ranges to CubeDesc

Posted by ma...@apache.org.
KYLIN-910,add auto_merge_time_ranges to CubeDesc


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

Branch: refs/heads/2.x-staging
Commit: c083e874079c51a44a580946a7161083d0937d34
Parents: 3ced3a4
Author: jiazhong <ji...@ebay.com>
Authored: Mon Sep 7 18:22:44 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Mon Sep 7 18:23:03 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     | 25 ++-----------------
 .../java/org/apache/kylin/cube/CubeManager.java |  2 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   | 20 +++++++++------
 .../org/apache/kylin/cube/CubeManagerTest.java  |  4 +--
 .../kylin/rest/controller/CubeController.java   |  1 -
 .../rest/controller/CubeDescController.java     |  1 -
 .../apache/kylin/rest/request/CubeRequest.java  |  8 ------
 .../apache/kylin/rest/service/CubeService.java  |  6 ++---
 webapp/app/js/controllers/cubeAdvanceSetting.js |  7 ++++++
 webapp/app/js/model/cubeDescModel.js            |  4 +--
 .../cubeDesigner/advanced_settings.html         | 26 ++++++++++++++++++--
 11 files changed, 53 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index 81aa4d6..e95a5c6 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -66,8 +66,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         cubeInstance.setStatus(RealizationStatusEnum.DISABLED);
         cubeInstance.updateRandomUuid();
         cubeInstance.setProjectName(projectName);
-        cubeInstance.setRetentionRange(cubeDesc.getRetentionRange());
-        
+
         // MR_V2 is the default engine since 0.8
         cubeInstance.setEngineType(IEngineAware.ID_MR_V2);
         cubeInstance.setStorageType(IStorageAware.ID_HBASE);
@@ -97,10 +96,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
 
     @JsonProperty("create_time_utc")
     private long createTimeUTC;
-    @JsonProperty("auto_merge_time_ranges")
-    private long[] autoMergeTimeRanges;
-    @JsonProperty("retention_range")
-    private long retentionRange = 0;
     @JsonProperty("engine_type")
     private int engineType = IEngineAware.ID_MR_V1;
     @JsonProperty("storage_type")
@@ -434,14 +429,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         return Lists.newArrayList(getDescriptor().listDimensionColumnsIncludingDerived());
     }
 
-    public long[] getAutoMergeTimeRanges() {
-        return autoMergeTimeRanges;
-    }
-
-    public void setAutoMergeTimeRanges(long[] autoMergeTimeRanges) {
-        this.autoMergeTimeRanges = autoMergeTimeRanges;
-    }
-
     public boolean needAutoMerge() {
         if (!this.getDescriptor().getModel().getPartitionDesc().isPartitioned())
             return false;
@@ -449,15 +436,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         if (this.getDescriptor().hasHolisticCountDistinctMeasures())
             return false;
 
-        return autoMergeTimeRanges != null && autoMergeTimeRanges.length > 0;
-    }
-
-    public long getRetentionRange() {
-        return retentionRange;
-    }
-
-    public void setRetentionRange(long retentionRange) {
-        this.retentionRange = retentionRange;
+        return this.getDescriptor().getAutoMergeTimeRanges() != null && this.getDescriptor().getAutoMergeTimeRanges().length > 0;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index c6f066c..c6e6b88 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -609,7 +609,7 @@ public class CubeManager implements IRealizationProvider {
             return null;
         }
 
-        long[] timeRanges = cube.getAutoMergeTimeRanges();
+        long[] timeRanges = cube.getDescriptor().getAutoMergeTimeRanges();
         Arrays.sort(timeRanges);
 
         CubeSegment newSeg = null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index ae49eb0..c3ffd24 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -54,7 +54,6 @@ import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.collect.Lists;
@@ -115,11 +114,10 @@ public class CubeDesc extends RootPersistentEntity {
     private List<String> notifyList;
     @JsonProperty("status_need_notify")
     private List<String> statusNeedNotify = Collections.emptyList();
-
-    /*
-     * this field will not be serialized ,can be deserialized to front javascript
-    */
-    private long retentionRange;
+    @JsonProperty("auto_merge_time_ranges")
+    private long[] autoMergeTimeRanges;
+    @JsonProperty("retention_range")
+    private long retentionRange = 0;
 
     private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>();
     private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>();
@@ -718,16 +716,22 @@ public class CubeDesc extends RootPersistentEntity {
         return false;
     }
 
-    @JsonProperty
     public long getRetentionRange() {
         return retentionRange;
     }
 
-    @JsonIgnore
     public void setRetentionRange(long retentionRange) {
         this.retentionRange = retentionRange;
     }
 
+    public long[] getAutoMergeTimeRanges() {
+        return autoMergeTimeRanges;
+    }
+
+    public void setAutoMergeTimeRanges(long[] autoMergeTimeRanges) {
+        this.autoMergeTimeRanges = autoMergeTimeRanges;
+    }
+
     /**
      * Add error info and thrown exception out
      *

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
index a8476e4..708bf32 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeManagerTest.java
@@ -95,7 +95,7 @@ public class CubeManagerTest extends LocalFileMetadataTestCase {
         CubeManager mgr = CubeManager.getInstance(getTestConfig());
         CubeInstance cube = mgr.getCube("test_kylin_cube_with_slr_empty");
 
-        cube.setAutoMergeTimeRanges(new long[] { 2000, 6000 });
+        cube.getDescriptor().setAutoMergeTimeRanges(new long[]{2000, 6000});
         mgr.updateCube(new CubeUpdate(cube));
 
         assertTrue(cube.needAutoMerge());
@@ -138,7 +138,7 @@ public class CubeManagerTest extends LocalFileMetadataTestCase {
         CubeManager mgr = CubeManager.getInstance(getTestConfig());
         CubeInstance cube = mgr.getCube("test_kylin_cube_with_slr_empty");
 
-        cube.setAutoMergeTimeRanges(new long[] { 2000, 6000 });
+        cube.getDescriptor().setAutoMergeTimeRanges(new long[] { 2000, 6000 });
         mgr.updateCube(new CubeUpdate(cube));
 
         assertTrue(cube.needAutoMerge());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index b8ec976..d61793d 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -340,7 +340,6 @@ public class CubeController extends BasicController {
 
         try {
             CubeInstance cube = cubeService.getCubeManager().getCube(cubeRequest.getCubeName());
-            cube.setRetentionRange(desc.getRetentionRange());
             String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME : cubeRequest.getProject();
             desc = cubeService.updateCubeAndDesc(cube, desc, projectName);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/server/src/main/java/org/apache/kylin/rest/controller/CubeDescController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/CubeDescController.java b/server/src/main/java/org/apache/kylin/rest/controller/CubeDescController.java
index 395365e..0a39634 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/CubeDescController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/CubeDescController.java
@@ -58,7 +58,6 @@ public class CubeDescController {
         }
         CubeDesc cSchema = cubeInstance.getDescriptor();
         if (cSchema != null) {
-            cSchema.setRetentionRange(cubeInstance.getRetentionRange());
             return new CubeDesc[] { cSchema };
         } else {
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/server/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/request/CubeRequest.java b/server/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
index 74712a7..1371a4f 100644
--- a/server/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
+++ b/server/src/main/java/org/apache/kylin/rest/request/CubeRequest.java
@@ -26,7 +26,6 @@ public class CubeRequest {
     private boolean successful;
     private String message;
     private String project;
-    private long retentionRange;
 
     public String getUuid() {
         return uuid;
@@ -105,11 +104,4 @@ public class CubeRequest {
         this.project = project;
     }
 
-    public long getRetentionRange() {
-        return retentionRange;
-    }
-
-    public void setRetentionRange(long retentionRange) {
-        this.retentionRange = retentionRange;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index ce51623..aac32e8 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -260,7 +260,6 @@ public class CubeService extends BasicService {
             }
 
             CubeDesc updatedCubeDesc = getCubeDescManager().updateCubeDesc(desc);
-            cube = getCubeManager().updateCube(new CubeUpdate(cube));
             int cuboidCount = CuboidCLI.simulateCuboidGeneration(updatedCubeDesc);
             logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
 
@@ -591,7 +590,8 @@ public class CubeService extends BasicService {
 
     private void keepCubeRetention(String cubeName) {
         CubeInstance cube = getCubeManager().getCube(cubeName);
-        if (cube.getRetentionRange() > 0) {
+        CubeDesc desc = cube.getDescriptor();
+        if (desc.getRetentionRange() > 0) {
             synchronized (CubeService.class) {
                 cube = getCubeManager().getCube(cubeName);
                 List<CubeSegment> readySegs = cube.getSegment(SegmentStatusEnum.READY);
@@ -599,7 +599,7 @@ public class CubeService extends BasicService {
                 int position = readySegs.size() - 1;
                 while (position >= 0) {
                     currentRange += (readySegs.get(position).getDateRangeEnd() - readySegs.get(position).getDateRangeStart());
-                    if (currentRange >= cube.getRetentionRange()) {
+                    if (currentRange >= desc.getRetentionRange()) {
                         break;
                     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/webapp/app/js/controllers/cubeAdvanceSetting.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeAdvanceSetting.js b/webapp/app/js/controllers/cubeAdvanceSetting.js
index 93c13e3..5ced1bd 100644
--- a/webapp/app/js/controllers/cubeAdvanceSetting.js
+++ b/webapp/app/js/controllers/cubeAdvanceSetting.js
@@ -40,4 +40,11 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal,cubeConfi
         }
 
     }
+
+  $scope.refreshAutoMergeTimeRanges = function(list,index,item){
+    if (item) {
+      list[index] = item;
+    }
+  }
+
 });

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/webapp/app/js/model/cubeDescModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeDescModel.js b/webapp/app/js/model/cubeDescModel.js
index 43efa93..86c8444 100644
--- a/webapp/app/js/model/cubeDescModel.js
+++ b/webapp/app/js/model/cubeDescModel.js
@@ -45,11 +45,11 @@ KylinApp.service('CubeDescModel',function(){
                     "aggregation_groups": []
                 },
                 "notify_list": [],
-                "capacity": "",
                 "hbase_mapping": {
                     "column_family": []
                 },
-               "retentionRange":"0"
+               "retention_range":"0",
+               "auto_merge_time_ranges":[604800000,2419200000]
             };
 
             return cubeMeta;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c083e874/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index b56983c..037a003 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -29,12 +29,34 @@
               <!--Cube Size-->
               <div class="form-group">
                 <div class="row">
+                  <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Auto Merge Time Ranges(days)</b></label>
+                  <div class="col-xs-12 col-sm-6">
+                    <!--retention range is store in CubeInstance, will convert to cubeMetaFrame for front end-->
+                    <div class="row"  ng-if="state.mode=='edit'">
+                      <div class="col-xs-3" ng-repeat="timeRange in cubeMetaFrame.auto_merge_time_ranges">
+                        <input type="text" retention-format  ng-change="refreshAutoMergeTimeRanges(cubeMetaFrame.auto_merge_time_ranges,$index,timeRange)" class="form-control" placeholder=".col-xs-3" ng-model="timeRange">
+                      </div>
+                    </div>
+                    <span ng-if="state.mode=='view'">{{cubeMetaFrame.auto_merge_time_ranges[0] | millisecondsToDay}}</span>
+                    <span  ng-if="state.mode=='view'">,</span>
+                    <span ng-if="state.mode=='view'">{{cubeMetaFrame.auto_merge_time_ranges[1] | millisecondsToDay}}</span>
+                  </div>
+                </div>
+              </div>
+            </td>
+          </tr>
+
+          <tr>
+            <td>
+              <!--Cube Size-->
+              <div class="form-group">
+                <div class="row">
                   <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Retention Range(days)</b></label>
                   <div class="col-xs-12 col-sm-6">
                     <!--retention range is store in CubeInstance, will convert to cubeMetaFrame for front end-->
-                    <input type="text" retention-format class="form-control ng-scope ng-pristine ng-valid" placeholder="how many days cube retention" ng-model="cubeMetaFrame.retentionRange" ng-if="state.mode=='edit'">
+                    <input type="text" retention-format class="form-control ng-scope ng-pristine ng-valid" placeholder="how many days cube retention" ng-model="cubeMetaFrame.retention_range" ng-if="state.mode=='edit'">
                     <small class="help-block text-red" ng-show="state.mode=='edit'">(by default it's '0',which will keep all historic cube segments ,or will keep latest [Retention Range] days cube segments )</small>
-                    <span ng-if="state.mode=='view'">{{cube.retention_range | millisecondsToDay}}</span>
+                    <span ng-if="state.mode=='view'">{{cubeMetaFrame.retention_range | millisecondsToDay}}</span>
                   </div>
                 </div>
               </div>


[16/28] incubator-kylin git commit: KYLIN-1001 fix duplicated “/“ in hdfs path

Posted by ma...@apache.org.
KYLIN-1001 fix duplicated “/“ in hdfs path


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

Branch: refs/heads/2.x-staging
Commit: cc0ede30a9dc27cf615b8b311066de49201e81a6
Parents: b5c9722
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 10:54:30 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 10:54:55 2015 +0800

----------------------------------------------------------------------
 core-job/src/main/java/org/apache/kylin/job/JobInstance.java       | 2 +-
 .../main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java    | 2 +-
 .../java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java   | 2 +-
 .../main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cc0ede30/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/JobInstance.java b/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
index 80a5d78..001dfe5 100644
--- a/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
+++ b/core-job/src/main/java/org/apache/kylin/job/JobInstance.java
@@ -65,7 +65,7 @@ public class JobInstance extends RootPersistentEntity implements Comparable<JobI
         if (jobUuid == null || jobUuid.equals("")) {
             throw new IllegalArgumentException("jobUuid can't be null or empty");
         }
-        return hdfsWorkdingDir + "/" + JOB_WORKING_DIR_PREFIX + jobUuid;
+        return hdfsWorkdingDir + JOB_WORKING_DIR_PREFIX + jobUuid;
     }
 
     @JsonProperty("name")

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cc0ede30/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 5f977bc..5fba37c 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -157,7 +157,7 @@ public class JobBuilderSupport {
     // ----------------------------------------------------------------------------
 
     public static String getJobWorkingDir(JobEngineConfig conf, String jobId) {
-        return conf.getHdfsWorkingDirectory() + "/" + "kylin-" + jobId;
+        return conf.getHdfsWorkingDirectory() + "kylin-" + jobId;
     }
 
     public static StringBuilder appendExecCmdParameters(StringBuilder buf, String paraName, String paraValue) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cc0ede30/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index f140784..53b489e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -177,7 +177,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             String path = status.getPath().getName();
             // System.out.println(path);
             if (path.startsWith(JobInstance.JOB_WORKING_DIR_PREFIX)) {
-                String kylinJobPath = engineConfig.getHdfsWorkingDirectory() + "/" + path;
+                String kylinJobPath = engineConfig.getHdfsWorkingDirectory() + path;
                 allHdfsPathsNeedToBeDeleted.add(kylinJobPath);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cc0ede30/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
index b4f69d6..4bd06c5 100644
--- a/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/invertedindex/IIJobBuilder.java
@@ -221,7 +221,7 @@ public final class IIJobBuilder {
     }
 
     private String getJobWorkingDir(String uuid) {
-        return engineConfig.getHdfsWorkingDirectory() + "/" + "kylin-" + uuid;
+        return engineConfig.getHdfsWorkingDirectory() + "kylin-" + uuid;
     }
 
     private String getIntermediateTableIdentity(IIJoinedFlatTableDesc intermediateTableDesc) {


[18/28] incubator-kylin git commit: KYLIN-1004 Dictionary with '' value cause cube merge to fail

Posted by ma...@apache.org.
KYLIN-1004 Dictionary with '' value cause cube merge to fail


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

Branch: refs/heads/2.x-staging
Commit: 3116d5563620ed30918a99295ab96ca27edca259
Parents: cdc33f5
Author: Li, Yang <ya...@ebay.com>
Authored: Sun Sep 6 15:33:40 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Sun Sep 6 15:34:16 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/dict/TrieDictionary.java | 9 +++------
 1 file changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3116d556/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index 995ea7a..39a8e7e 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -148,9 +148,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
     @Override
     final protected int getIdFromValueImpl(T value, int roundingFlag) {
         if (enableValueCache && roundingFlag == 0) {
-            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache
-                                                  // gracefully when short of
-                                                  // memory
+            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
             if (cache != null) {
                 Integer id = null;
                 id = (Integer) cache.get(value);
@@ -194,7 +192,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
      *            -- >0: return closest bigger if not found, return nValues
      */
     private int lookupSeqNoFromValue(int n, byte[] inp, int o, int inpEnd, int roundingFlag) {
-        if (inp.length == 0) // special 'empty' value
+        if (o == inpEnd) // 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
@@ -231,8 +229,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
             while (true) {
                 p = c + firstByteOffset;
                 comp = BytesUtil.compareByteUnsigned(trieBytes[p], inpByte);
-                if (comp == 0) { // continue in the matching child, reset n and
-                                 // loop again
+                if (comp == 0) { // continue in the matching child, reset n and loop again
                     n = c;
                     o++;
                     break;


[05/28] incubator-kylin git commit: KYLIN-984 Behavior change in streaming data consuming

Posted by ma...@apache.org.
KYLIN-984 Behavior change in streaming data consuming


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

Branch: refs/heads/2.x-staging
Commit: ab2abeec50cc6f924d21da3f8a01dcf5a5c7c650
Parents: c73f5de
Author: honma <ho...@ebay.com>
Authored: Tue Sep 1 14:01:07 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 1 14:14:04 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/job/streaming/BootstrapConfig.java | 12 +++++++++++-
 .../apache/kylin/job/streaming/StreamingBootstrap.java  |  8 ++++++++
 .../org/apache/kylin/job/streaming/StreamingCLI.java    |  3 +++
 .../java/org/apache/kylin/streaming/StreamFetcher.java  |  8 ++------
 4 files changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ab2abeec/job/src/main/java/org/apache/kylin/job/streaming/BootstrapConfig.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/BootstrapConfig.java b/job/src/main/java/org/apache/kylin/job/streaming/BootstrapConfig.java
index ceff3c1..029d4d2 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/BootstrapConfig.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/BootstrapConfig.java
@@ -10,10 +10,20 @@ public class BootstrapConfig {
     //one off default value set to true
     private boolean oneOff = true;
     private long start = 0L;
-    private long end = 0L;
+    private long end = 0L; 
+    private long margin = 0L;
+    
 
     private boolean fillGap;
 
+    public long getMargin() {
+        return margin;
+    }
+
+    public void setMargin(long margin) {
+        this.margin = margin;
+    }
+
     public boolean isOneOff() {
         return oneOff;
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ab2abeec/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java b/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
index 0811451..b7a8335 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/StreamingBootstrap.java
@@ -135,6 +135,14 @@ public class StreamingBootstrap {
         final String streaming = bootstrapConfig.getStreaming();
         Preconditions.checkNotNull(streaming, "streaming name cannot be empty");
         final StreamingConfig streamingConfig = streamingManager.getStreamingConfig(streaming);
+
+        if (bootstrapConfig.getMargin() != 0L) {
+            streamingConfig.setMargin(bootstrapConfig.getMargin());
+            logger.info("Margin is overwrite to " + streamingConfig.getMargin());
+        } else {
+            logger.info("Margin is default value: " + streamingConfig.getMargin());
+        }
+
         Preconditions.checkArgument(streamingConfig != null, "cannot find kafka config:" + streaming);
 
         if (!StringUtils.isEmpty(streamingConfig.getIiName())) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ab2abeec/job/src/main/java/org/apache/kylin/job/streaming/StreamingCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/streaming/StreamingCLI.java b/job/src/main/java/org/apache/kylin/job/streaming/StreamingCLI.java
index 78b1e93..8346ec0 100644
--- a/job/src/main/java/org/apache/kylin/job/streaming/StreamingCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/streaming/StreamingCLI.java
@@ -79,6 +79,9 @@ public class StreamingCLI {
                 case "-fillGap":
                     bootstrapConfig.setFillGap(Boolean.parseBoolean(args[++i]));
                     break;
+                case "-margin":
+                    bootstrapConfig.setMargin(Long.parseLong(args[++i]));
+                    break;
                 default:
                     logger.warn("ignore this arg:" + argName);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ab2abeec/streaming/src/main/java/org/apache/kylin/streaming/StreamFetcher.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/kylin/streaming/StreamFetcher.java b/streaming/src/main/java/org/apache/kylin/streaming/StreamFetcher.java
index f78389e..85d09be 100644
--- a/streaming/src/main/java/org/apache/kylin/streaming/StreamFetcher.java
+++ b/streaming/src/main/java/org/apache/kylin/streaming/StreamFetcher.java
@@ -59,14 +59,10 @@ public class StreamFetcher implements Callable<MicroStreamBatch> {
                     microStreamBatch = new MicroStreamBatch(partitionId);
                     clearCounter();
                 }
-                StreamMessage streamMessage = peek(streamMessageQueue, 30000);
+                StreamMessage streamMessage = peek(streamMessageQueue, 60000);
                 if (streamMessage == null) {
                     logger.info("The stream queue is drained, current available stream count: " + microStreamBatch.size());
-                    if (!microStreamBatch.isEmpty()) {
-                        return microStreamBatch;
-                    } else {
-                        continue;
-                    }
+                    return microStreamBatch;
                 }
                 if (streamMessage.getOffset() < 0) {
                     logger.warn("streaming encountered EOF, stop building");


[26/28] incubator-kylin git commit: KYLIN-965 Allow user to configure the region split size for cube

Posted by ma...@apache.org.
KYLIN-965 Allow user to configure the region split size for cube


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

Branch: refs/heads/2.x-staging
Commit: 3ced3a481374682c3d9581dc64dba688308ced8d
Parents: 9f4b93b
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 16:10:39 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 16:11:04 2015 +0800

----------------------------------------------------------------------
 conf/kylin.properties                           |  8 ++++----
 .../org/apache/kylin/common/KylinConfig.java    | 20 ++++++++++----------
 .../test_case_data/sandbox/kylin.properties     |  8 ++++----
 .../storage/hbase/steps/CreateHTableJob.java    |  4 ++--
 4 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3ced3a48/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index aca5ede..8dfb05b 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -51,10 +51,10 @@ kylin.hbase.default.compression.codec=snappy
 kylin.job.cubing.inMem.sampling.percent=25
 
 # The cut size for hbase region, in GB.
-# E.g, for cube whose capacity be marked as "SMALL", split region per 5GB by default
-kylin.job.hbase.region.cut.small=5
-kylin.job.hbase.region.cut.medium=10
-kylin.job.hbase.region.cut.large=50
+# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
+kylin.hbase.region.cut.small=10
+kylin.hbase.region.cut.medium=20
+kylin.hbase.region.cut.large=100
 
 ## Config for Restful APP ##
 # database connection settings:

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3ced3a48/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 06deb55..db213f7 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -129,9 +129,9 @@ public class KylinConfig implements Serializable {
     public static final String KYLIN_HOME = "KYLIN_HOME";
     public static final String KYLIN_CONF = "KYLIN_CONF";
 
-    public static final String HBASE_REGION_CUT_SMALL = "kylin.job.hbase.region.cut.small";
-    public static final String HBASE_REGION_CUT_MEDIUM = "kylin.job.hbase.region.cut.medium";
-    public static final String HBASE_REGION_CUT_LARGE = "kylin.job.hbase.region.cut.large";
+    public static final String HBASE_REGION_CUT_SMALL = "kylin.hbase.region.cut.small";
+    public static final String HBASE_REGION_CUT_MEDIUM = "kylin.hbase.region.cut.medium";
+    public static final String HBASE_REGION_CUT_LARGE = "kylin.hbase.region.cut.large";
 
     public static final String SPARK_HOME = "kylin.spark.home";
     public static final String SPARK_MASTER = "kylin.spark.master";
@@ -454,12 +454,12 @@ public class KylinConfig implements Serializable {
         return Integer.parseInt(getOptional("kylin.table.snapshot.max_mb", "300"));
     }
 
-    public int getHBaseRegionCutMin() {
-        return Integer.parseInt(getOptional("kylin.job.hbase.region.cut.min", "2"));
+    public int getHBaseRegionCountMin() {
+        return Integer.parseInt(getOptional("kylin.hbase.region.count.min", "1"));
     }
 
-    public int getHBaseRegionCutMax() {
-        return Integer.parseInt(getOptional("kylin.job.hbase.region.cut.max", "1000"));
+    public int getHBaseRegionCountMax() {
+        return Integer.parseInt(getOptional("kylin.hbase.region.count.max", "500"));
     }
 
     public int getScanThreshold() {
@@ -726,13 +726,13 @@ public class KylinConfig implements Serializable {
         String cut;
         switch (capacity) {
         case "SMALL":
-            cut = getProperty(HBASE_REGION_CUT_SMALL, "5");
+            cut = getProperty(HBASE_REGION_CUT_SMALL, "10");
             break;
         case "MEDIUM":
-            cut = getProperty(HBASE_REGION_CUT_MEDIUM, "10");
+            cut = getProperty(HBASE_REGION_CUT_MEDIUM, "20");
             break;
         case "LARGE":
-            cut = getProperty(HBASE_REGION_CUT_LARGE, "50");
+            cut = getProperty(HBASE_REGION_CUT_LARGE, "100");
             break;
         default:
             throw new IllegalArgumentException("Capacity not recognized: " + capacity);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3ced3a48/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
index ce599ea..beb9c44 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -40,10 +40,10 @@ kylin.job.cubing.inMem=true
 kylin.job.cubing.inMem.sampling.percent=25
 
 # The cut size for hbase region, in GB.
-# E.g, for cube whose capacity be marked as "SMALL", split region per 5GB by default
-kylin.job.hbase.region.cut.small=5
-kylin.job.hbase.region.cut.medium=10
-kylin.job.hbase.region.cut.large=50
+# E.g, for cube whose capacity be marked as "SMALL", split region per 10GB by default
+kylin.hbase.region.cut.small=10
+kylin.hbase.region.cut.medium=20
+kylin.hbase.region.cut.large=100
 
 # Time interval to check hadoop job status
 kylin.job.yarn.app.rest.check.interval.seconds=10

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3ced3a48/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
index 9a80ae1..35a35c1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java
@@ -251,8 +251,8 @@ public class CreateHTableJob extends AbstractHadoopJob {
         }
 
         int nRegion = Math.round((float) totalSizeInM / (cut * 1024L));
-        nRegion = Math.max(kylinConfig.getHBaseRegionCutMin(), nRegion);
-        nRegion = Math.min(kylinConfig.getHBaseRegionCutMax(), nRegion);
+        nRegion = Math.max(kylinConfig.getHBaseRegionCountMin(), nRegion);
+        nRegion = Math.min(kylinConfig.getHBaseRegionCountMax(), nRegion);
 
         int mbPerRegion = (int) (totalSizeInM / (nRegion));
         mbPerRegion = Math.max(1, mbPerRegion);


[07/28] incubator-kylin git commit: KYLIN-972 Make MR_V2 the default engine for new cubes. Old cubes (0.7) continue to build with MR_V1 engine.

Posted by ma...@apache.org.
KYLIN-972 Make MR_V2 the default engine for new cubes. Old cubes (0.7)
continue to build with MR_V1 engine.


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

Branch: refs/heads/2.x-staging
Commit: bf83339c1a69770365dded79cf5c3062ec9839de
Parents: ab2abee
Author: Li, Yang <ya...@ebay.com>
Authored: Sat Aug 29 07:57:09 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Tue Sep 1 16:22:14 2015 +0800

----------------------------------------------------------------------
 .../persistence/RootPersistentEntity.java       |   1 +
 .../org/apache/kylin/common/util/BytesUtil.java |   2 +-
 .../org/apache/kylin/common/util/ClassUtil.java |  10 +-
 .../kylin/common/util/CompressionUtils.java     |  17 ++
 .../kylin/common/util/DaemonThreadFactory.java  |  17 ++
 .../apache/kylin/common/util/DateFormat.java    |  17 ++
 .../apache/kylin/common/util/FIFOIterable.java  |  17 ++
 .../apache/kylin/common/util/FIFOIterator.java  |  17 ++
 .../apache/kylin/common/util/IdentityUtils.java |  17 ++
 .../kylin/common/util/ImmutableBitSet.java      |  17 ++
 .../kylin/common/util/ImplementationSwitch.java |  70 +++++++
 .../common/util/ImplementationSwitchTest.java   |  58 ++++++
 .../org/apache/kylin/cube/CubeInstance.java     |  47 ++++-
 .../java/org/apache/kylin/cube/CubeManager.java |   4 +-
 .../java/org/apache/kylin/cube/CubeSegment.java |   9 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |  41 +---
 .../apache/kylin/dict/DictionaryManager.java    |   4 +-
 .../apache/kylin/dict/lookup/SnapshotCLI.java   |   4 +-
 .../apache/kylin/engine/BuildEngineFactory.java |  52 -----
 .../org/apache/kylin/engine/EngineFactory.java  |  66 ++++++
 .../kylin/metadata/model/DataModelDesc.java     |  21 +-
 .../apache/kylin/metadata/model/IBuildable.java |   7 +-
 .../kylin/metadata/model/IEngineAware.java      |  28 +++
 .../kylin/metadata/model/ISourceAware.java      |  27 +++
 .../kylin/metadata/model/IStorageAware.java     |  27 +++
 .../apache/kylin/metadata/model/TableDesc.java  |  11 +-
 .../metadata/realization/IRealization.java      |   3 +-
 .../java/org/apache/kylin/source/ISource.java   |  28 +++
 .../org/apache/kylin/source/ITableSource.java   |  28 ---
 .../org/apache/kylin/source/SourceFactory.java  |  50 +++++
 .../apache/kylin/source/TableSourceFactory.java |  40 ----
 .../java/org/apache/kylin/storage/IStorage.java |   2 +-
 .../apache/kylin/storage/StorageFactory.java    |  32 ++-
 .../kylin/storage/StorageQueryFactory.java      |  97 ---------
 .../cache/AbstractCacheFledgedQuery.java        |  84 ++++++++
 .../AbstractCacheFledgedStorageEngine.java      |  84 --------
 .../storage/cache/CacheFledgedDynamicQuery.java | 149 ++++++++++++++
 .../cache/CacheFledgedDynamicStorageEngine.java | 149 --------------
 .../storage/cache/CacheFledgedStaticQuery.java  |  88 ++++++++
 .../cache/CacheFledgedStaticStorageEngine.java  |  88 --------
 .../exception/ScanOutOfLimitException.java      |   2 +-
 .../kylin/storage/hybrid/HybridInstance.java    |  24 ++-
 .../kylin/storage/hybrid/HybridManager.java     |  17 ++
 .../kylin/storage/hybrid/HybridStorage.java     |  36 ++++
 .../storage/hybrid/HybridStorageEngine.java     |  44 ----
 .../storage/hybrid/HybridStorageQuery.java      |  61 ++++++
 .../kylin/storage/cache/DynamicCacheTest.java   |   2 +-
 .../kylin/storage/cache/StaticCacheTest.java    |   2 +-
 .../kylin/engine/mr/BatchCubingJobBuilder.java  |  10 +-
 .../kylin/engine/mr/BatchCubingJobBuilder2.java |   5 +
 .../kylin/engine/mr/BatchMergeJobBuilder.java   |   5 +
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |   5 +
 .../kylin/engine/mr/JobBuilderSupport.java      |   3 -
 .../java/org/apache/kylin/engine/mr/MRUtil.java |   6 +-
 .../apache/kylin/engine/mr/steps/CuboidJob.java |   3 +-
 .../engine/spark/SparkCubingJobBuilder.java     |   3 +
 .../apache/kylin/invertedindex/IIInstance.java  |   7 +
 .../job/hadoop/cube/NewBaseCuboidMapper.java    |   6 +-
 .../kylin/job/BuildCubeWithEngineTest.java      |   4 +-
 .../java/org/apache/kylin/job/DeployUtil.java   |   5 +-
 .../source/hive/ITSnapshotManagerTest.java      |   4 +-
 .../kylin/query/enumerator/OLAPEnumerator.java  |   4 +-
 .../AdjustForWeaklyMatchedRealization.java      |   6 +-
 .../kylin/rest/controller/QueryController.java  |   4 +-
 .../apache/kylin/rest/service/CubeService.java  |   4 +-
 .../apache/kylin/rest/service/JobService.java   |   8 +-
 .../kylin/source/hive/HiveTableSource.java      |   4 +-
 .../kylin/storage/hbase/HBaseStorage.java       |  70 ++++++-
 .../storage/hbase/steps/DeprecatedGCStep.java   | 201 +++++++++++++++++++
 .../storage/hbase/common/ITStorageTest.java     |   8 +-
 70 files changed, 1395 insertions(+), 698 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java b/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
index 0cbf9c2..bc72c1e 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/RootPersistentEntity.java
@@ -40,6 +40,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
  * 
  * @author yangli9
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 abstract public class RootPersistentEntity implements AclEntity, Serializable {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index 0503ad6..0880da1 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -14,7 +14,7 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
-*/
+ */
 
 package org.apache.kylin.common.util;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
index f5474f5..93790e6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ClassUtil.java
@@ -47,11 +47,13 @@ public class ClassUtil {
     private static final Map<String, String> classRenameMap;
     static {
         classRenameMap = new HashMap<>();
+        classRenameMap.put("org.apache.kylin.job.common.HadoopShellExecutable", "org.apache.kylin.engine.mr.common.HadoopShellExecutable");
+        classRenameMap.put("org.apache.kylin.job.common.MapReduceExecutable", "org.apache.kylin.engine.mr.common.MapReduceExecutable");
         classRenameMap.put("org.apache.kylin.job.cube.CubingJob", "org.apache.kylin.engine.mr.CubingJob");
-        classRenameMap.put("org.apache.kylin.job.cube.GarbageCollectionStep", "org.apache.kylin.engine.mr.GarbageCollectionStep");
-        classRenameMap.put("org.apache.kylin.job.cube.MergeDictionaryStep", "org.apache.kylin.engine.mr.MergeDictionaryStep");
-        classRenameMap.put("org.apache.kylin.job.cube.UpdateCubeInfoAfterBuildStep", "org.apache.kylin.engine.mr.UpdateCubeInfoAfterBuildStep");
-        classRenameMap.put("org.apache.kylin.job.cube.UpdateCubeInfoAfterMergeStep", "org.apache.kylin.engine.mr.UpdateCubeInfoAfterMergeStep");
+        classRenameMap.put("org.apache.kylin.job.cube.GarbageCollectionStep", "org.apache.kylin.storage.hbase.steps.DeprecatedGCStep");
+        classRenameMap.put("org.apache.kylin.job.cube.MergeDictionaryStep", "org.apache.kylin.engine.mr.steps.MergeDictionaryStep");
+        classRenameMap.put("org.apache.kylin.job.cube.UpdateCubeInfoAfterBuildStep", "org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterBuildStep");
+        classRenameMap.put("org.apache.kylin.job.cube.UpdateCubeInfoAfterMergeStep", "org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterMergeStep");
     }
 
     @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/CompressionUtils.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/CompressionUtils.java b/core-common/src/main/java/org/apache/kylin/common/util/CompressionUtils.java
index 3ed279a..13abab5 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/CompressionUtils.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/CompressionUtils.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.io.ByteArrayOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/DaemonThreadFactory.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/DaemonThreadFactory.java b/core-common/src/main/java/org/apache/kylin/common/util/DaemonThreadFactory.java
index bc4502c..56f4a36 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/DaemonThreadFactory.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/DaemonThreadFactory.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.util.concurrent.Executors;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java b/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
index f74debd..f46edae 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/DateFormat.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.text.ParseException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterable.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterable.java b/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterable.java
index 4c4bc6b..7204e33 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterable.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterable.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterator.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterator.java b/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterator.java
index f734143..ccea37c 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterator.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/FIFOIterator.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java b/core-common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
index d873959..35ade60 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/IdentityUtils.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/ImmutableBitSet.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ImmutableBitSet.java b/core-common/src/main/java/org/apache/kylin/common/util/ImmutableBitSet.java
index 2ee7d4f..f5a22d2 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ImmutableBitSet.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ImmutableBitSet.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.common.util;
 
 import java.util.BitSet;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java b/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
new file mode 100644
index 0000000..3101c81
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ImplementationSwitch.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kylin.common.util;
+
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provide switch between different implementations of a same interface.
+ * Each implementation is identified by an integer ID.
+ */
+public class ImplementationSwitch {
+
+    private static final Logger logger = LoggerFactory.getLogger(ImplementationSwitch.class);
+
+    final private Object[] instances;
+
+    public ImplementationSwitch(Map<Integer, String> impls) {
+        instances = initInstances(impls);
+    }
+
+    private Object[] initInstances(Map<Integer, String> impls) {
+        int maxId = 0;
+        for (Integer id : impls.keySet()) {
+            maxId = Math.max(maxId, id);
+        }
+        if (maxId > 100)
+            throw new IllegalArgumentException("you have more than 100 implentations?");
+
+        Object[] result = new Object[maxId + 1];
+
+        for (Integer id : impls.keySet()) {
+            String clzName = impls.get(id);
+            try {
+                result[id] = ClassUtil.newInstance(clzName);
+            } catch (Exception ex) {
+                logger.warn("Implementation missing " + clzName + " - " + ex);
+            }
+        }
+
+        return result;
+    }
+
+    public <I> I get(int id, Class<I> interfaceClz) {
+        @SuppressWarnings("unchecked")
+        I result = (I) instances[id];
+
+        if (result == null)
+            throw new IllegalArgumentException("Implementations missing, ID " + id + ", interafce " + interfaceClz.getName());
+        
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-common/src/test/java/org/apache/kylin/common/util/ImplementationSwitchTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/ImplementationSwitchTest.java b/core-common/src/test/java/org/apache/kylin/common/util/ImplementationSwitchTest.java
new file mode 100644
index 0000000..4c69eeb
--- /dev/null
+++ b/core-common/src/test/java/org/apache/kylin/common/util/ImplementationSwitchTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kylin.common.util;
+
+import static org.junit.Assert.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+public class ImplementationSwitchTest {
+    
+    ImplementationSwitch sw;
+
+    public ImplementationSwitchTest() {
+        Map<Integer, String> impls = new HashMap<>();
+        impls.put(0, "non.exist.class");
+        impls.put(1, Impl1.class.getName());
+        impls.put(2, Impl2.class.getName());
+        sw = new ImplementationSwitch(impls);
+    }
+    
+    public static interface I {
+    }
+    
+    public static class Impl1 implements I {
+    }
+    
+    public static class Impl2 implements I {
+    }
+    
+    @Test
+    public void test() {
+        assertTrue(sw.get(1, I.class) instanceof Impl1);
+        assertTrue(sw.get(2, I.class) instanceof Impl2);
+    }
+    
+    @Test(expected = IllegalArgumentException.class)  
+    public void testException() {
+        sw.get(0, I.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index eb3b3e2..fcd338c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -26,9 +26,13 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.IEngineAware;
+import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.LookupDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
@@ -43,8 +47,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Objects;
 import com.google.common.collect.Lists;
 
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class CubeInstance extends RootPersistentEntity implements IRealization {
+public class CubeInstance extends RootPersistentEntity implements IRealization, IBuildable {
 
     public static CubeInstance create(String cubeName, String projectName, CubeDesc cubeDesc) {
         CubeInstance cubeInstance = new CubeInstance();
@@ -58,6 +63,10 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
         cubeInstance.updateRandomUuid();
         cubeInstance.setProjectName(projectName);
         cubeInstance.setRetentionRange(cubeDesc.getRetentionRange());
+        
+        // MR_V2 is the default engine since 0.8
+        cubeInstance.setEngineType(IEngineAware.ID_MR_V2);
+        cubeInstance.setStorageType(IStorageAware.ID_HBASE);
 
         return cubeInstance;
     }
@@ -84,12 +93,14 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
 
     @JsonProperty("create_time_utc")
     private long createTimeUTC;
-
     @JsonProperty("auto_merge_time_ranges")
     private long[] autoMergeTimeRanges;
-
     @JsonProperty("retention_range")
     private long retentionRange = 0;
+    @JsonProperty("engine_type")
+    private int engineType = IEngineAware.ID_MR_V1;
+    @JsonProperty("storage_type")
+    private int storageType = IStorageAware.ID_HBASE;
 
     private String projectName;
 
@@ -205,7 +216,11 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
 
     @Override
     public String getFactTable() {
-        return this.getDescriptor().getFactTable();
+        return getDescriptor().getFactTable();
+    }
+
+    public TableDesc getFactTableDesc() {
+        return getDescriptor().getFactTableDesc();
     }
 
     @Override
@@ -441,4 +456,28 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
     public void setRetentionRange(long retentionRange) {
         this.retentionRange = retentionRange;
     }
+
+    @Override
+    public int getSourceType() {
+        return getFactTableDesc().getSourceType();
+    }
+
+    @Override
+    public int getStorageType() {
+        return storageType;
+    }
+    
+    private void setStorageType(int storageType) {
+        this.storageType = storageType;
+    }
+
+    @Override
+    public int getEngineType() {
+        return engineType;
+    }
+
+    private void setEngineType(int engineType) {
+        this.engineType = engineType;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index 038c7cb..5cfecf1 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -59,7 +59,7 @@ import org.apache.kylin.metadata.realization.IRealizationProvider;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.source.ReadableTable;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -207,7 +207,7 @@ public class CubeManager implements IRealizationProvider {
         SnapshotManager snapshotMgr = getSnapshotManager();
 
         TableDesc tableDesc = metaMgr.getTableDesc(lookupTable);
-        ReadableTable hiveTable = TableSourceFactory.createReadableTable(tableDesc);
+        ReadableTable hiveTable = SourceFactory.createReadableTable(tableDesc);
         SnapshotTable snapshot = snapshotMgr.buildSnapshot(hiveTable, tableDesc);
 
         cubeSeg.putSnapshotResPath(lookupTable, snapshot.getResourcePath());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index a80bbd2..7d89470 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -326,7 +326,7 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
 
     @Override
     public String toString() {
-        return Objects.toStringHelper(this).add("uuid", uuid).add("create_time_utc:", createTimeUTC).add("name", name).add("last_build_job_id", lastBuildJobID).add("status", status).toString();
+        return Objects.toStringHelper(this).add("cube", cubeInstance.getName()).add("name", name).add("status", status).add("uuid", uuid).add("create_time_utc:", createTimeUTC).add("last_build_job_id", lastBuildJobID).toString();
     }
 
     public void setDictionaries(ConcurrentHashMap<String, String> dictionaries) {
@@ -347,16 +347,17 @@ public class CubeSegment implements Comparable<CubeSegment>, IDictionaryAware, I
 
     @Override
     public int getSourceType() {
-        return 0;
+        return cubeInstance.getSourceType();
     }
 
     @Override
     public int getEngineType() {
-        return 0;
+        return cubeInstance.getEngineType();
     }
 
     @Override
     public int getStorageType() {
-        return 0;
+        return cubeInstance.getStorageType();
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index f8d71b2..ae49eb0 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -26,7 +26,6 @@ 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;
@@ -63,6 +62,7 @@ import com.google.common.collect.Maps;
 
 /**
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class CubeDesc extends RootPersistentEntity {
 
@@ -217,29 +217,6 @@ public class CubeDesc extends RootPersistentEntity {
         return functions;
     }
 
-    /**
-     * @return
-     * @deprecated use getModel().getAllTables() instead
-     */
-    public List<TableDesc> listTables() {
-        MetadataManager metaMgr = MetadataManager.getInstance(config);
-        HashSet<String> tableNames = new HashSet<String>();
-        List<TableDesc> result = new ArrayList<TableDesc>();
-
-        tableNames.add(this.getFactTable().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 isDerived(TblColRef col) {
         return derivedToHostMap.containsKey(col);
     }
@@ -330,7 +307,11 @@ public class CubeDesc extends RootPersistentEntity {
     }
 
     public String getFactTable() {
-        return model.getFactTable().toUpperCase();
+        return model.getFactTable();
+    }
+    
+    public TableDesc getFactTableDesc() {
+        return model.getFactTableDesc();
     }
 
     public String[] getNullStrings() {
@@ -456,8 +437,8 @@ public class CubeDesc extends RootPersistentEntity {
         }
 
         sortDimAndMeasure();
-        initDimensionColumns(tables);
-        initMeasureColumns(tables);
+        initDimensionColumns();
+        initMeasureColumns();
 
         rowkey.init(this);
         if (hbaseMapping != null) {
@@ -473,7 +454,7 @@ public class CubeDesc extends RootPersistentEntity {
         }
     }
 
-    private void initDimensionColumns(Map<String, TableDesc> tables) {
+    private void initDimensionColumns() {
         for (DimensionDesc dim : dimensions) {
             JoinDesc join = dim.getJoin();
 
@@ -622,12 +603,12 @@ public class CubeDesc extends RootPersistentEntity {
         return ref;
     }
 
-    private void initMeasureColumns(Map<String, TableDesc> tables) {
+    private void initMeasureColumns() {
         if (measures == null || measures.isEmpty()) {
             return;
         }
 
-        TableDesc factTable = tables.get(getFactTable());
+        TableDesc factTable = getFactTableDesc();
         for (MeasureDesc m : measures) {
             m.setName(m.getName().toUpperCase());
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index f6d76dc..c4b6ef0 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -35,7 +35,7 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable;
 import org.apache.kylin.source.ReadableTable.TableSignature;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -220,7 +220,7 @@ public class DictionaryManager {
             inpTable = factTableValueProvider.getDistinctValuesFor(srcCol);
         } else {
             TableDesc tableDesc = MetadataManager.getInstance(config).getTableDesc(srcTable);
-            inpTable = TableSourceFactory.createReadableTable(tableDesc);
+            inpTable = SourceFactory.createReadableTable(tableDesc);
         }
 
         TableSignature inputSig = inpTable.getSignature();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotCLI.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotCLI.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotCLI.java
index d3e2c7d..149badc 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotCLI.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotCLI.java
@@ -5,7 +5,7 @@ import java.io.IOException;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 
 public class SnapshotCLI {
 
@@ -23,7 +23,7 @@ public class SnapshotCLI {
         if (tableDesc == null)
             throw new IllegalArgumentException("Not table found by " + table);
 
-        SnapshotTable snapshot = snapshotMgr.rebuildSnapshot(TableSourceFactory.createReadableTable(tableDesc), tableDesc, overwriteUUID);
+        SnapshotTable snapshot = snapshotMgr.rebuildSnapshot(SourceFactory.createReadableTable(tableDesc), tableDesc, overwriteUUID);
         System.out.println("resource path updated: " + snapshot.getResourcePath());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java b/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
deleted file mode 100644
index 7c21e69..0000000
--- a/core-job/src/main/java/org/apache/kylin/engine/BuildEngineFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.engine;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-
-public class BuildEngineFactory {
-
-    private static IBatchCubingEngine defaultBatchEngine;
-
-    public static IBatchCubingEngine defaultBatchEngine() {
-        if (defaultBatchEngine == null) {
-            KylinConfig conf = KylinConfig.getInstanceFromEnv();
-            if (conf.isCubingInMem()) {
-                defaultBatchEngine = (IBatchCubingEngine) ClassUtil.newInstance("org.apache.kylin.engine.mr.MRBatchCubingEngine2");
-            } else {
-                defaultBatchEngine = (IBatchCubingEngine) ClassUtil.newInstance("org.apache.kylin.engine.mr.MRBatchCubingEngine");
-            }
-        }
-        return defaultBatchEngine;
-    }
-
-    /** Build a new cube segment, typically its time range appends to the end of current cube. */
-    public static DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
-        return defaultBatchEngine().createBatchCubingJob(newSegment, submitter);
-    }
-
-    /** Merge multiple small segments into a big one. */
-    public static DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
-        return defaultBatchEngine().createBatchMergeJob(mergeSegment, submitter);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java b/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
new file mode 100644
index 0000000..8b8fb87
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/engine/EngineFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ImplementationSwitch;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.metadata.model.IEngineAware;
+import static org.apache.kylin.metadata.model.IEngineAware.*;
+
+public class EngineFactory {
+    
+    private static ImplementationSwitch batchEngines;
+    private static ImplementationSwitch streamingEngines;
+    static {
+        Map<Integer, String> impls = new HashMap<>();
+        impls.put(ID_MR_V1, "org.apache.kylin.engine.mr.MRBatchCubingEngine");
+        impls.put(ID_MR_V2, "org.apache.kylin.engine.mr.MRBatchCubingEngine2");
+        batchEngines = new ImplementationSwitch(impls);
+        
+        impls.clear();
+        streamingEngines = new ImplementationSwitch(impls); // TODO
+    }
+    
+    public static IBatchCubingEngine batchEngine(IEngineAware aware) {
+        return batchEngines.get(aware.getEngineType(), IBatchCubingEngine.class);
+    }
+    
+    public static IStreamingCubingEngine streamingEngine(IEngineAware aware) {
+        return streamingEngines.get(aware.getEngineType(), IStreamingCubingEngine.class);
+    }
+    
+    /** Build a new cube segment, typically its time range appends to the end of current cube. */
+    public static DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
+        return batchEngine(newSegment).createBatchCubingJob(newSegment, submitter);
+    }
+
+    /** Merge multiple small segments into a big one. */
+    public static DefaultChainedExecutable createBatchMergeJob(CubeSegment mergeSegment, String submitter) {
+        return batchEngine(mergeSegment).createBatchMergeJob(mergeSegment, submitter);
+    }
+    
+    public static Runnable createStreamingCubingBuilder(CubeSegment seg) {
+        return streamingEngine(seg).createStreamingCubingBuilder(seg);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
index 31d7d6c..1c6ef62 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
@@ -36,6 +36,7 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.Sets;
 
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class DataModelDesc extends RootPersistentEntity {
 
@@ -69,6 +70,8 @@ public class DataModelDesc extends RootPersistentEntity {
     @JsonProperty("capacity")
     private RealizationCapacity capacity = RealizationCapacity.MEDIUM;
 
+    private TableDesc factTableDesc;
+
     /**
      * Error messages during resolving json metadata
      */
@@ -101,6 +104,10 @@ public class DataModelDesc extends RootPersistentEntity {
     public String getFactTable() {
         return factTable;
     }
+    
+    public TableDesc getFactTableDesc() {
+        return factTableDesc;
+    }
 
     public void setFactTable(String factTable) {
         this.factTable = factTable.toUpperCase();
@@ -168,6 +175,11 @@ public class DataModelDesc extends RootPersistentEntity {
     }
 
     public void init(Map<String, TableDesc> tables) {
+        this.factTableDesc = tables.get(this.factTable.toUpperCase());
+        if (factTableDesc == null) {
+            throw new IllegalStateException("Fact table does not exist:" + this.factTable);
+        }
+
         initJoinColumns(tables);
         DimensionDesc.capicalizeStrings(dimensions);
         initPartitionDesc(tables);
@@ -195,6 +207,7 @@ public class DataModelDesc extends RootPersistentEntity {
 
             StringUtil.toUpperCaseArray(join.getForeignKey(), join.getForeignKey());
             StringUtil.toUpperCaseArray(join.getPrimaryKey(), join.getPrimaryKey());
+            
             // primary key
             String[] pks = join.getPrimaryKey();
             TblColRef[] pkCols = new TblColRef[pks.length];
@@ -208,15 +221,12 @@ public class DataModelDesc extends RootPersistentEntity {
                 pkCols[i] = colRef;
             }
             join.setPrimaryKeyColumns(pkCols);
+            
             // foreign key
-            TableDesc factTable = tables.get(this.factTable.toUpperCase());
-            if (factTable == null) {
-                throw new IllegalStateException("Fact table does not exist:" + this.getFactTable());
-            }
             String[] fks = join.getForeignKey();
             TblColRef[] fkCols = new TblColRef[fks.length];
             for (int i = 0; i < fks.length; i++) {
-                ColumnDesc col = factTable.findColumnByName(fks[i]);
+                ColumnDesc col = factTableDesc.findColumnByName(fks[i]);
                 if (col == null) {
                     throw new IllegalStateException("Can't find column " + fks[i] + " in table " + this.getFactTable());
                 }
@@ -225,6 +235,7 @@ public class DataModelDesc extends RootPersistentEntity {
                 fkCols[i] = colRef;
             }
             join.setForeignKeyColumns(fkCols);
+            
             // Validate join in dimension
             if (pkCols.length != fkCols.length) {
                 throw new IllegalStateException("Primary keys(" + lookup.getTable() + ")" + Arrays.toString(pks) + " are not consistent with Foreign keys(" + this.getFactTable() + ") " + Arrays.toString(fks));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/IBuildable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IBuildable.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IBuildable.java
index 3090de0..39129f8 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IBuildable.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IBuildable.java
@@ -18,11 +18,6 @@
 
 package org.apache.kylin.metadata.model;
 
-public interface IBuildable {
+public interface IBuildable extends ISourceAware, IEngineAware, IStorageAware {
 
-    int getSourceType();
-
-    int getEngineType();
-
-    int getStorageType();
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
new file mode 100644
index 0000000..60bd825
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IEngineAware.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.model;
+
+public interface IEngineAware {
+
+    public static final int ID_MR_V1 = 0;
+    public static final int ID_MR_V2 = 2;
+    public static final int ID_SPARK = 5;
+
+    int getEngineType();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java
new file mode 100644
index 0000000..3d89f40
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISourceAware.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.model;
+
+public interface ISourceAware {
+
+    public static final int ID_HIVE = 0;
+    public static final int ID_SPARKSQL = 5;
+
+    int getSourceType();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
new file mode 100644
index 0000000..ea1aae9
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IStorageAware.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metadata.model;
+
+public interface IStorageAware {
+
+    public static final int ID_HBASE = 0;
+    public static final int ID_HYBRID = 1;
+
+    int getStorageType();
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
index 785e9d4..d5e4dbb 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
@@ -32,12 +32,16 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 /**
  * Table Metadata from Source. All name should be uppercase.
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class TableDesc extends RootPersistentEntity {
+public class TableDesc extends RootPersistentEntity implements ISourceAware {
+    
     @JsonProperty("name")
     private String name;
     @JsonProperty("columns")
     private ColumnDesc[] columns;
+    @JsonProperty("source_type")
+    private int sourceType = ISourceAware.ID_HIVE;
 
     private DatabaseDesc database = new DatabaseDesc();
 
@@ -171,4 +175,9 @@ public class TableDesc extends RootPersistentEntity {
         mockup.setName(tableName);
         return mockup;
     }
+
+    @Override
+    public int getSourceType() {
+        return sourceType;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
index 6f90e14..8c9258a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealization.java
@@ -21,10 +21,11 @@ package org.apache.kylin.metadata.realization;
 import java.util.List;
 
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 
-public interface IRealization {
+public interface IRealization extends IStorageAware {
 
     public boolean isCapable(SQLDigest digest);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/source/ISource.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/ISource.java b/core-metadata/src/main/java/org/apache/kylin/source/ISource.java
new file mode 100644
index 0000000..3cd8a02
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/source/ISource.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source;
+
+import org.apache.kylin.metadata.model.TableDesc;
+
+public interface ISource {
+
+    public <I> I adaptToBuildEngine(Class<I> engineInterface);
+
+    public ReadableTable createReadableTable(TableDesc tableDesc);
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java b/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
deleted file mode 100644
index 83ae8b3..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/source/ITableSource.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.source;
-
-import org.apache.kylin.metadata.model.TableDesc;
-
-public interface ITableSource {
-
-    public <I> I adaptToBuildEngine(Class<I> engineInterface);
-
-    public ReadableTable createReadableTable(TableDesc tableDesc);
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java b/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
new file mode 100644
index 0000000..2fbf847
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/source/SourceFactory.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.source;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ImplementationSwitch;
+import org.apache.kylin.metadata.model.ISourceAware;
+import static org.apache.kylin.metadata.model.ISourceAware.*;
+import org.apache.kylin.metadata.model.TableDesc;
+
+public class SourceFactory {
+
+    private static ImplementationSwitch sources;
+    static {
+        Map<Integer, String> impls = new HashMap<>();
+        impls.put(ID_HIVE, "org.apache.kylin.source.hive.HiveTableSource");
+        sources = new ImplementationSwitch(impls);
+    }
+
+    public static ISource tableSource(ISourceAware aware) {
+        return sources.get(aware.getSourceType(), ISource.class);
+    }
+
+    public static ReadableTable createReadableTable(TableDesc table) {
+        return tableSource(table).createReadableTable(table);
+    }
+
+    public static <T> T createEngineAdapter(ISourceAware table, Class<T> engineInterface) {
+        return tableSource(table).adaptToBuildEngine(engineInterface);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java b/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
deleted file mode 100644
index 67191e3..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/source/TableSourceFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.source;
-
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.metadata.model.IBuildable;
-import org.apache.kylin.metadata.model.TableDesc;
-
-public class TableSourceFactory {
-
-    private static ITableSource dft = (ITableSource) ClassUtil.newInstance("org.apache.kylin.source.hive.HiveTableSource");
-
-    public static ReadableTable createReadableTable(TableDesc table) {
-        return dft.createReadableTable(table);
-    }
-
-    public static <T> T createEngineAdapter(IBuildable buildable, Class<T> engineInterface) {
-        return dft.adaptToBuildEngine(engineInterface);
-    }
-
-    public static <T> T createEngineAdapter(TableDesc tableDesc, Class<T> engineInterface) {
-        return dft.adaptToBuildEngine(engineInterface);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java b/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
index 6506a4f..e229e14 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/IStorage.java
@@ -22,7 +22,7 @@ import org.apache.kylin.metadata.realization.IRealization;
 
 public interface IStorage {
 
-    public IStorageQuery createStorageQuery(IRealization realization);
+    public IStorageQuery createQuery(IRealization realization);
 
     public <I> I adaptToBuildEngine(Class<I> engineInterface);
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java
index b8e3e91..b26dfdb 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/StorageFactory.java
@@ -18,17 +18,37 @@
 
 package org.apache.kylin.storage;
 
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.metadata.model.IBuildable;
+import static org.apache.kylin.metadata.model.IStorageAware.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.kylin.common.util.ImplementationSwitch;
+import org.apache.kylin.metadata.model.IStorageAware;
+import org.apache.kylin.metadata.realization.IRealization;
 
 /**
  */
 public class StorageFactory {
 
-    private static final IStorage dft = (IStorage) ClassUtil.newInstance("org.apache.kylin.storage.hbase.HBaseStorage");
-
-    public static <T> T createEngineAdapter(IBuildable buildable, Class<T> engineInterface) {
-        return dft.adaptToBuildEngine(engineInterface);
+    private static ImplementationSwitch storages;
+    static {
+        Map<Integer, String> impls = new HashMap<>();
+        impls.put(ID_HBASE, "org.apache.kylin.storage.hbase.HBaseStorage");
+        impls.put(ID_HYBRID, "org.apache.kylin.storage.hybrid.HybridStorage");
+        storages = new ImplementationSwitch(impls);
+    }
+    
+    public static IStorage storage(IStorageAware aware) {
+        return storages.get(aware.getStorageType(), IStorage.class);
+    }
+    
+    public static IStorageQuery createQuery(IRealization realization) {
+        return storage(realization).createQuery(realization);
+    }
+    
+    public static <T> T createEngineAdapter(IStorageAware aware, Class<T> engineInterface) {
+        return storage(aware).adaptToBuildEngine(engineInterface);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/StorageQueryFactory.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/StorageQueryFactory.java b/core-storage/src/main/java/org/apache/kylin/storage/StorageQueryFactory.java
deleted file mode 100644
index eb6e6b1..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/StorageQueryFactory.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.storage;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.PartitionDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.realization.IRealization;
-import org.apache.kylin.metadata.realization.RealizationType;
-import org.apache.kylin.storage.cache.CacheFledgedDynamicStorageEngine;
-import org.apache.kylin.storage.cache.CacheFledgedStaticStorageEngine;
-import org.apache.kylin.storage.hybrid.HybridInstance;
-import org.apache.kylin.storage.hybrid.HybridStorageEngine;
-
-import com.google.common.base.Preconditions;
-
-/**
- * @author xjiang
- */
-public class StorageQueryFactory {
-
-    private final static boolean allowStorageLayerCache = true;
-    private final static String defaultCubeStorageQuery = "org.apache.kylin.storage.hbase.cube.v1.CubeStorageQuery";
-    private final static String defaultIIStorageQuery = "org.apache.kylin.storage.hbase.ii.InvertedIndexStorageQuery";
-
-    public static IStorageQuery createQuery(IRealization realization) {
-
-        if (realization.getType() == RealizationType.INVERTED_INDEX) {
-            ICachableStorageQuery ret;
-            try {
-                ret = (ICachableStorageQuery) Class.forName(defaultIIStorageQuery).getConstructor(IIInstance.class).newInstance((IIInstance) realization);
-            } catch (Exception e) {
-                throw new RuntimeException("Failed to initialize storage query for " + defaultIIStorageQuery, e);
-            }
-
-            if (allowStorageLayerCache) {
-                return wrapWithCache(ret, realization);
-            } else {
-                return ret;
-            }
-        } else if (realization.getType() == RealizationType.CUBE) {
-            ICachableStorageQuery ret;
-            try {
-                ret = (ICachableStorageQuery) Class.forName(defaultCubeStorageQuery).getConstructor(CubeInstance.class).newInstance((CubeInstance) realization);
-            } catch (Exception e) {
-                throw new RuntimeException("Failed to initialize storage query for " + defaultCubeStorageQuery, e);
-            }
-
-            if (allowStorageLayerCache) {
-                return wrapWithCache(ret, realization);
-            } else {
-                return ret;
-            }
-        } else {
-            return new HybridStorageEngine((HybridInstance) realization);
-        }
-    }
-
-    private static IStorageQuery wrapWithCache(ICachableStorageQuery underlyingStorageEngine, IRealization realization) {
-        if (underlyingStorageEngine.isDynamic()) {
-            return new CacheFledgedDynamicStorageEngine(underlyingStorageEngine, getPartitionCol(realization));
-        } else {
-            return new CacheFledgedStaticStorageEngine(underlyingStorageEngine);
-        }
-    }
-
-    private static TblColRef getPartitionCol(IRealization realization) {
-        String modelName = realization.getModelName();
-        DataModelDesc dataModelDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getDataModelDesc(modelName);
-        PartitionDesc partitionDesc = dataModelDesc.getPartitionDesc();
-        Preconditions.checkArgument(partitionDesc != null, "PartitionDesc for " + realization + " is null!");
-        TblColRef partitionColRef = partitionDesc.getPartitionDateColumnRef();
-        Preconditions.checkArgument(partitionColRef != null, "getPartitionDateColumnRef for " + realization + " is null");
-        return partitionColRef;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedQuery.java
new file mode 100644
index 0000000..5ffdf91
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedQuery.java
@@ -0,0 +1,84 @@
+package org.apache.kylin.storage.cache;
+
+import net.sf.ehcache.Cache;
+import net.sf.ehcache.CacheManager;
+import net.sf.ehcache.config.CacheConfiguration;
+import net.sf.ehcache.config.Configuration;
+import net.sf.ehcache.config.MemoryUnit;
+import net.sf.ehcache.config.PersistenceConfiguration;
+import net.sf.ehcache.store.MemoryStoreEvictionPolicy;
+
+import org.apache.kylin.metadata.realization.StreamSQLDigest;
+import org.apache.kylin.metadata.tuple.TeeTupleItrListener;
+import org.apache.kylin.storage.ICachableStorageQuery;
+import org.apache.kylin.storage.IStorageQuery;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public abstract class AbstractCacheFledgedQuery implements IStorageQuery, TeeTupleItrListener {
+    private static final Logger logger = LoggerFactory.getLogger(AbstractCacheFledgedQuery.class);
+    private static final String storageCacheTemplate = "StorageCache";
+
+    protected static CacheManager CACHE_MANAGER;
+
+    protected boolean queryCacheExists;
+    protected ICachableStorageQuery underlyingStorage;
+    protected StreamSQLDigest streamSQLDigest;
+
+    public AbstractCacheFledgedQuery(ICachableStorageQuery underlyingStorage) {
+        this.underlyingStorage = underlyingStorage;
+        this.makeCacheIfNecessary(underlyingStorage.getStorageUUID());
+    }
+
+    public static void setCacheManager(CacheManager cacheManager) {
+        CACHE_MANAGER = cacheManager;
+    }
+
+    private static void initCacheManger() {
+        Configuration conf = new Configuration();
+        conf.setMaxBytesLocalHeap("128M");
+        CACHE_MANAGER = CacheManager.create(conf);
+
+        //a fake template for test cases
+        Cache storageCache = new Cache(new CacheConfiguration(storageCacheTemplate, 0).//
+                memoryStoreEvictionPolicy(MemoryStoreEvictionPolicy.LRU).//
+                eternal(false).//
+                timeToIdleSeconds(86400).//
+                diskExpiryThreadIntervalSeconds(0).//
+                maxBytesLocalHeap(10, MemoryUnit.MEGABYTES).//
+                persistence(new PersistenceConfiguration().strategy(PersistenceConfiguration.Strategy.NONE)));
+
+        CACHE_MANAGER.addCache(storageCache);
+    }
+
+    private void makeCacheIfNecessary(String storageUUID) {
+        if (CACHE_MANAGER == null) {
+            logger.warn("CACHE_MANAGER is not provided");
+            initCacheManger();
+        }
+
+        if (CACHE_MANAGER.getCache(storageUUID) == null) {
+            logger.info("Cache for {} initting...", storageUUID);
+
+            //Create a Cache specifying its configuration.
+            CacheConfiguration templateConf = CACHE_MANAGER.getCache(storageCacheTemplate).getCacheConfiguration();
+            PersistenceConfiguration pconf = templateConf.getPersistenceConfiguration();
+            if (pconf != null) {
+                logger.info("PersistenceConfiguration strategy: " + pconf.getStrategy());
+            } else {
+                logger.warn("PersistenceConfiguration is null");
+            }
+
+            Cache storageCache = new Cache(new CacheConfiguration(storageUUID, (int) templateConf.getMaxEntriesLocalHeap()).//
+                    memoryStoreEvictionPolicy(templateConf.getMemoryStoreEvictionPolicy()).//
+                    eternal(templateConf.isEternal()).//
+                    timeToIdleSeconds(templateConf.getTimeToIdleSeconds()).//
+                    maxBytesLocalHeap(templateConf.getMaxBytesLocalHeap(), MemoryUnit.BYTES).persistence(pconf));
+            //TODO: deal with failed queries, and only cache too long query
+
+            CACHE_MANAGER.addCache(storageCache);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedStorageEngine.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedStorageEngine.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedStorageEngine.java
deleted file mode 100644
index 61e008f..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/cache/AbstractCacheFledgedStorageEngine.java
+++ /dev/null
@@ -1,84 +0,0 @@
-package org.apache.kylin.storage.cache;
-
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.CacheManager;
-import net.sf.ehcache.config.CacheConfiguration;
-import net.sf.ehcache.config.Configuration;
-import net.sf.ehcache.config.MemoryUnit;
-import net.sf.ehcache.config.PersistenceConfiguration;
-import net.sf.ehcache.store.MemoryStoreEvictionPolicy;
-
-import org.apache.kylin.metadata.realization.StreamSQLDigest;
-import org.apache.kylin.metadata.tuple.TeeTupleItrListener;
-import org.apache.kylin.storage.ICachableStorageQuery;
-import org.apache.kylin.storage.IStorageQuery;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-public abstract class AbstractCacheFledgedStorageEngine implements IStorageQuery, TeeTupleItrListener {
-    private static final Logger logger = LoggerFactory.getLogger(AbstractCacheFledgedStorageEngine.class);
-    private static final String storageCacheTemplate = "StorageCache";
-
-    protected static CacheManager CACHE_MANAGER;
-
-    protected boolean queryCacheExists;
-    protected ICachableStorageQuery underlyingStorage;
-    protected StreamSQLDigest streamSQLDigest;
-
-    public AbstractCacheFledgedStorageEngine(ICachableStorageQuery underlyingStorage) {
-        this.underlyingStorage = underlyingStorage;
-        this.makeCacheIfNecessary(underlyingStorage.getStorageUUID());
-    }
-
-    public static void setCacheManager(CacheManager cacheManager) {
-        CACHE_MANAGER = cacheManager;
-    }
-
-    private static void initCacheManger() {
-        Configuration conf = new Configuration();
-        conf.setMaxBytesLocalHeap("128M");
-        CACHE_MANAGER = CacheManager.create(conf);
-
-        //a fake template for test cases
-        Cache storageCache = new Cache(new CacheConfiguration(storageCacheTemplate, 0).//
-                memoryStoreEvictionPolicy(MemoryStoreEvictionPolicy.LRU).//
-                eternal(false).//
-                timeToIdleSeconds(86400).//
-                diskExpiryThreadIntervalSeconds(0).//
-                maxBytesLocalHeap(10, MemoryUnit.MEGABYTES).//
-                persistence(new PersistenceConfiguration().strategy(PersistenceConfiguration.Strategy.NONE)));
-
-        CACHE_MANAGER.addCache(storageCache);
-    }
-
-    private void makeCacheIfNecessary(String storageUUID) {
-        if (CACHE_MANAGER == null) {
-            logger.warn("CACHE_MANAGER is not provided");
-            initCacheManger();
-        }
-
-        if (CACHE_MANAGER.getCache(storageUUID) == null) {
-            logger.info("Cache for {} initting...", storageUUID);
-
-            //Create a Cache specifying its configuration.
-            CacheConfiguration templateConf = CACHE_MANAGER.getCache(storageCacheTemplate).getCacheConfiguration();
-            PersistenceConfiguration pconf = templateConf.getPersistenceConfiguration();
-            if (pconf != null) {
-                logger.info("PersistenceConfiguration strategy: " + pconf.getStrategy());
-            } else {
-                logger.warn("PersistenceConfiguration is null");
-            }
-
-            Cache storageCache = new Cache(new CacheConfiguration(storageUUID, (int) templateConf.getMaxEntriesLocalHeap()).//
-                    memoryStoreEvictionPolicy(templateConf.getMemoryStoreEvictionPolicy()).//
-                    eternal(templateConf.isEternal()).//
-                    timeToIdleSeconds(templateConf.getTimeToIdleSeconds()).//
-                    maxBytesLocalHeap(templateConf.getMaxBytesLocalHeap(), MemoryUnit.BYTES).persistence(pconf));
-            //TODO: deal with failed queries, and only cache too long query
-
-            CACHE_MANAGER.addCache(storageCache);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicQuery.java
new file mode 100644
index 0000000..febe1a9
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicQuery.java
@@ -0,0 +1,149 @@
+package org.apache.kylin.storage.cache;
+
+import java.util.List;
+
+import net.sf.ehcache.Cache;
+import net.sf.ehcache.Element;
+
+import org.apache.kylin.common.util.RangeUtil;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.realization.SQLDigestUtil;
+import org.apache.kylin.metadata.realization.StreamSQLDigest;
+import org.apache.kylin.metadata.tuple.CompoundTupleIterator;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
+import org.apache.kylin.metadata.tuple.TeeTupleIterator;
+import org.apache.kylin.storage.ICachableStorageQuery;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.tuple.TupleInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+
+/**
+ */
+public class CacheFledgedDynamicQuery extends AbstractCacheFledgedQuery {
+    private static final Logger logger = LoggerFactory.getLogger(CacheFledgedDynamicQuery.class);
+
+    private final TblColRef partitionColRef;
+
+    private Range<Long> ts;
+
+    public CacheFledgedDynamicQuery(ICachableStorageQuery underlyingStorage, TblColRef partitionColRef) {
+        super(underlyingStorage);
+        this.partitionColRef = partitionColRef;
+
+        Preconditions.checkArgument(this.partitionColRef != null, "For dynamic columns like " + //
+                this.underlyingStorage.getStorageUUID() + ", partition column must be provided");
+    }
+
+    @Override
+    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
+        //check if ts condition in sqlDigest valid
+        ts = TsConditionExtractor.extractTsCondition(partitionColRef, sqlDigest.filter);
+        if (ts == null || ts.isEmpty()) {
+            logger.info("ts range in the query conflicts,return empty directly");
+            return ITupleIterator.EMPTY_TUPLE_ITERATOR;
+        }
+
+        //enable dynamic cache iff group by columns contains partition col
+        //because cache extraction requires partition col value as selection key
+        boolean needUpdateCache = sqlDigest.groupbyColumns.contains(partitionColRef);
+
+        streamSQLDigest = new StreamSQLDigest(sqlDigest, partitionColRef);
+        StreamSQLResult cachedResult = null;
+        Cache cache = CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID());
+        Element element = cache.get(streamSQLDigest.hashCode());
+        if (element != null) {
+            this.queryCacheExists = true;
+            cachedResult = (StreamSQLResult) element.getObjectValue();
+        }
+
+        ITupleIterator ret = null;
+        if (cachedResult != null) {
+            Range<Long> reusePeriod = cachedResult.getReusableResults(ts);
+
+            logger.info("existing cache    : " + cachedResult);
+            logger.info("ts Range in query: " + RangeUtil.formatTsRange(ts));
+            logger.info("potential reusable range   : " + RangeUtil.formatTsRange(reusePeriod));
+
+            if (reusePeriod != null) {
+                List<Range<Long>> remainings = RangeUtil.remove(ts, reusePeriod);
+                if (remainings.size() == 1) {//if using cache causes two underlyingStorage searches, we'd rather not use the cache
+
+                    SimpleTupleIterator reusedTuples = new SimpleTupleIterator(cachedResult.reuse(reusePeriod));
+                    List<ITupleIterator> iTupleIteratorList = Lists.newArrayList();
+                    iTupleIteratorList.add(reusedTuples);
+
+                    for (Range<Long> remaining : remainings) {//actually there will be only one loop
+                        logger.info("Appending ts " + RangeUtil.formatTsRange(remaining) + " as additional filter");
+
+                        ITupleIterator freshTuples = SQLDigestUtil.appendTsFilterToExecute(sqlDigest, partitionColRef, remaining, new Function<Void, ITupleIterator>() {
+                            @Override
+                            public ITupleIterator apply(Void input) {
+                                return underlyingStorage.search(context, sqlDigest, returnTupleInfo);
+                            }
+                        });
+                        iTupleIteratorList.add(freshTuples);
+                    }
+
+                    ret = new CompoundTupleIterator(iTupleIteratorList);
+                } else if (remainings.size() == 0) {
+                    logger.info("The ts range in new query was fully cached");
+                    needUpdateCache = false;
+                    ret = new SimpleTupleIterator(cachedResult.reuse(reusePeriod));
+                } else {
+                    //if using cache causes more than one underlyingStorage searches
+                    //the incurred overhead might be more expensive than the cache benefit
+                    logger.info("Give up using cache to avoid complexity");
+                }
+            }
+        } else {
+            logger.info("no cache entry for this query");
+        }
+
+        if (ret == null) {
+            ret = underlyingStorage.search(context, sqlDigest, returnTupleInfo);
+            logger.info("No Cache being used");
+        } else {
+            logger.info("Cache being used");
+        }
+
+        if (needUpdateCache) {
+            //use another nested ITupleIterator to deal with cache
+            final TeeTupleIterator tee = new TeeTupleIterator(ret);
+            tee.addCloseListener(this);
+            return tee;
+        } else {
+            return ret;
+        }
+    }
+
+    @Override
+    public void notify(List<ITuple> duplicated, long createTime) {
+
+        Range<Long> cacheExclude = this.underlyingStorage.getVolatilePeriod();
+        if (cacheExclude != null) {
+            List<Range<Long>> cachablePeriods = RangeUtil.remove(ts, cacheExclude);
+            if (cachablePeriods.size() == 1) {
+                if (!ts.equals(cachablePeriods.get(0))) {
+                    logger.info("With respect to growing storage, the cacheable tsRange shrinks from " + RangeUtil.formatTsRange(ts) + " to " + RangeUtil.formatTsRange(cachablePeriods.get(0)));
+                }
+                ts = cachablePeriods.get(0);
+            } else {
+                //give up updating the cache, in avoid to make cache complicated
+                logger.info("Skip updating cache to avoid complexity");
+            }
+        }
+
+        StreamSQLResult newCacheEntry = new StreamSQLResult(duplicated, ts, partitionColRef);
+        CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID()).put(new Element(streamSQLDigest.hashCode(), newCacheEntry));
+        logger.info("cache after the query: " + newCacheEntry);
+    }
+}



[08/28] incubator-kylin git commit: KYLIN-986 add streaming scripts

Posted by ma...@apache.org.
KYLIN-986 add streaming scripts


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

Branch: refs/heads/2.x-staging
Commit: 8ae2c0fd2d31abb21a2605b0cb853742dd386856
Parents: bf83339
Author: honma <ho...@ebay.com>
Authored: Tue Sep 1 17:51:16 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 1 17:51:31 2015 +0800

----------------------------------------------------------------------
 bin/streaming_build.sh   | 17 +++++++++++++++++
 bin/streaming_check.sh   | 12 ++++++++++++
 bin/streaming_fillgap.sh | 10 ++++++++++
 bin/streaming_rolllog.sh | 12 ++++++++++++
 4 files changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8ae2c0fd/bin/streaming_build.sh
----------------------------------------------------------------------
diff --git a/bin/streaming_build.sh b/bin/streaming_build.sh
new file mode 100644
index 0000000..2149ea4
--- /dev/null
+++ b/bin/streaming_build.sh
@@ -0,0 +1,17 @@
+#!/bin/bash
+
+source /etc/profile
+source ~/.bash_profile
+
+STREAMING=$1
+INTERVAL=$2
+DELAY=$3
+MARGIN=$4
+CURRENT_TIME_IN_SECOND=`date +%s`
+CURRENT_TIME=$((CURRENT_TIME_IN_SECOND * 1000))
+START=$(($CURRENT_TIME - CURRENT_TIME%INTERVAL - DELAY))
+END=$(($CURRENT_TIME - CURRENT_TIME%INTERVAL - DELAY + INTERVAL))
+
+ID="$START"_"$END"
+echo "building for ${ID}" >> ${KYLIN_HOME}/logs/build_trace.log
+sh ${KYLIN_HOME}/bin/kylin.sh streaming start ${STREAMING} ${ID} -oneoff true -start ${START} -end ${END} -streaming ${STREAMING} -margin ${MARGIN}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8ae2c0fd/bin/streaming_check.sh
----------------------------------------------------------------------
diff --git a/bin/streaming_check.sh b/bin/streaming_check.sh
new file mode 100644
index 0000000..5f70e25
--- /dev/null
+++ b/bin/streaming_check.sh
@@ -0,0 +1,12 @@
+#!/bin/bash
+
+source /etc/profile
+source ~/.bash_profile
+
+receivers=$1
+host=$2
+tablename=$3
+authorization=$4
+projectname=$5
+cubename=$6
+sh ${KYLIN_HOME}/bin/kylin.sh monitor -receivers ${receivers} -host ${host} -tableName ${tablename} -authorization ${authorization} -cubeName ${cubename} -projectName ${projectname}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8ae2c0fd/bin/streaming_fillgap.sh
----------------------------------------------------------------------
diff --git a/bin/streaming_fillgap.sh b/bin/streaming_fillgap.sh
new file mode 100644
index 0000000..cecaaa9
--- /dev/null
+++ b/bin/streaming_fillgap.sh
@@ -0,0 +1,10 @@
+#!/bin/bash
+
+source /etc/profile
+source ~/.bash_profile
+
+streaming=$1
+margin=$2
+
+cd ${KYLIN_HOME}
+sh ${KYLIN_HOME}/bin/kylin.sh streaming start ${streaming} fillgap -streaming ${streaming} -fillGap true -margin ${margin}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8ae2c0fd/bin/streaming_rolllog.sh
----------------------------------------------------------------------
diff --git a/bin/streaming_rolllog.sh b/bin/streaming_rolllog.sh
new file mode 100644
index 0000000..9380f49
--- /dev/null
+++ b/bin/streaming_rolllog.sh
@@ -0,0 +1,12 @@
+#!/bin/bash
+
+source /etc/profile
+source ~/.bash_profile
+
+KYLIN_LOG_HOME=${KYLIN_HOME}/logs
+cd ${KYLIN_LOG_HOME}
+timestamp=`date +%Y_%m_%d_%H_%M_%S`
+tarfile=logs_archived_at_${timestamp}.tar
+files=`find . ! -name '*.tar' -type f -mtime +1` # keep two days' log
+echo ${files} | xargs tar -cvf ${tarfile}
+echo ${files} | xargs rm
\ No newline at end of file


[13/28] incubator-kylin git commit: KYLIN-985 Fix support of AVG()

Posted by ma...@apache.org.
KYLIN-985 Fix support of AVG()


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

Branch: refs/heads/2.x-staging
Commit: 8c56d17774cbb0a161cf346c38d9d9d3fb94312f
Parents: a7c272c
Author: Yang Li <li...@apache.org>
Authored: Fri Sep 4 20:43:06 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Sep 4 20:43:06 2015 +0800

----------------------------------------------------------------------
 .../kylin/query/optrule/OLAPAggregateRule.java  | 19 ++++++++++++++++
 .../resources/query/sql_tableau/query28.sql     | 23 ++++++++++++++++++++
 2 files changed, 42 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8c56d177/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
index 6fa2731..2cb73c8 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
@@ -23,7 +23,10 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.kylin.query.relnode.OLAPAggregateRel;
 import org.apache.kylin.query.relnode.OLAPRel;
 
@@ -40,6 +43,13 @@ public class OLAPAggregateRule extends ConverterRule {
     @Override
     public RelNode convert(RelNode rel) {
         LogicalAggregate agg = (LogicalAggregate) rel;
+
+        // AVG() will be transformed into SUM()/COUNT() by AggregateReduceFunctionsRule.
+        // Here only let the transformed plan pass.
+        if (containsAvg(agg)) {
+            return null;
+        }
+
         RelTraitSet traitSet = agg.getTraitSet().replace(OLAPRel.CONVENTION);
         try {
             return new OLAPAggregateRel(agg.getCluster(), traitSet, convert(agg.getInput(), traitSet), agg.getGroupSet(), agg.getAggCallList());
@@ -48,4 +58,13 @@ public class OLAPAggregateRule extends ConverterRule {
         }
     }
 
+    private boolean containsAvg(LogicalAggregate agg) {
+        for (AggregateCall call : agg.getAggCallList()) {
+            SqlAggFunction func = call.getAggregation();
+            if (func instanceof SqlAvgAggFunction)
+                return true;
+        }
+        return false;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8c56d177/query/src/test/resources/query/sql_tableau/query28.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_tableau/query28.sql b/query/src/test/resources/query/sql_tableau/query28.sql
new file mode 100644
index 0000000..48cc342
--- /dev/null
+++ b/query/src/test/resources/query/sql_tableau/query28.sql
@@ -0,0 +1,23 @@
+--
+-- 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.
+--
+
+-- This query don't result exact same average number as H2 DB
+
+select lstg_format_name, avg(price) as GMV
+ from test_kylin_fact 
+ group by lstg_format_name 


[28/28] incubator-kylin git commit: code clean: change all variable log to logger

Posted by ma...@apache.org.
code clean: change all variable log to logger


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

Branch: refs/heads/2.x-staging
Commit: 175392c6e3d23245a388378a21344f0cfe078a96
Parents: c083e87
Author: honma <ho...@ebay.com>
Authored: Mon Sep 7 15:57:11 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Sep 7 19:05:49 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/cmd/ShellCmd.java |  6 ++--
 .../apache/kylin/job/cmd/ShellCmdOutput.java    |  4 +--
 .../engine/mr/common/AbstractHadoopJob.java     |  4 ---
 .../common/DefaultSslProtocolSocketFactory.java |  4 +--
 .../mr/common/DefaultX509TrustManager.java      | 12 +++----
 .../kylin/engine/mr/common/HadoopCmdOutput.java |  8 ++---
 .../engine/mr/common/HadoopStatusGetter.java    |  6 ++--
 .../engine/mr/steps/FactDistinctColumnsJob.java |  4 +--
 .../engine/mr/steps/MetadataCleanupJob.java     | 10 +++---
 .../mr/steps/RangeKeyDistributionJob.java       |  2 +-
 .../util/DefaultSslProtocolSocketFactory.java   |  4 +--
 .../jdbc/util/DefaultX509TrustManager.java      | 12 +++----
 .../job/hadoop/cube/OrphanHBaseCleanJob.java    | 20 ++++++------
 .../job/hadoop/cube/StorageCleanupJob.java      | 33 +++++++++-----------
 .../hadoop/invertedindex/IICreateHFileJob.java  |  2 +-
 .../invertedindex/IIDistinctColumnsJob.java     |  6 ++--
 .../hadoop/invertedindex/InvertedIndexJob.java  |  2 +-
 .../apache/kylin/job/tools/CleanHtableCLI.java  |  4 +--
 .../kylin/job/tools/HtableAlterMetadataCLI.java |  2 +-
 .../kylin/storage/hbase/steps/BulkLoadJob.java  |  6 ++--
 .../kylin/storage/hbase/util/HBaseClean.java    | 13 ++++----
 21 files changed, 77 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
index 19f68c7..b93c058 100644
--- a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmd.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
  */
 public class ShellCmd implements IJobCommand {
 
-    private static Logger log = LoggerFactory.getLogger(ShellCmd.class);
+    private static Logger logger = LoggerFactory.getLogger(ShellCmd.class);
 
     private final String executeCommand;
     private final ICommandOutput output;
@@ -74,9 +74,9 @@ public class ShellCmd implements IJobCommand {
         if (!isAsync) {
             try {
                 exitCode = future.get();
-                log.info("finish executing");
+                logger.info("finish executing");
             } catch (CancellationException e) {
-                log.debug("Command is cancelled");
+                logger.debug("Command is cancelled");
                 exitCode = -2;
             } catch (Exception e) {
                 throw new JobException("Error when execute job " + executeCommand, e);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
index aaf1ba8..44609c2 100644
--- a/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
+++ b/core-job/src/main/java/org/apache/kylin/job/cmd/ShellCmdOutput.java
@@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory;
  */
 public class ShellCmdOutput extends BaseCommandOutput implements ICommandOutput {
 
-    protected static final Logger log = LoggerFactory.getLogger(ShellCmdOutput.class);
+    protected static final Logger logger = LoggerFactory.getLogger(ShellCmdOutput.class);
 
     protected StringBuilder output;
     protected int exitCode;
@@ -62,7 +62,7 @@ public class ShellCmdOutput extends BaseCommandOutput implements ICommandOutput
     @Override
     public void appendOutput(String message) {
         output.append(message).append(System.getProperty("line.separator"));
-        log.debug(message);
+        logger.debug(message);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index 5679a6a..7412e71 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -78,20 +78,16 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     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 static final Option OPTION_STATISTICS_ENABLED = OptionBuilder.withArgName("statisticsenabled").hasArg().isRequired(false).withDescription("Statistics enabled").create("statisticsenabled");
     protected static final Option OPTION_STATISTICS_OUTPUT = OptionBuilder.withArgName("statisticsoutput").hasArg().isRequired(false).withDescription("Statistics output").create("statisticsoutput");
     protected static final Option OPTION_STATISTICS_SAMPLING_PERCENT = OptionBuilder.withArgName("statisticssamplingpercent").hasArg().isRequired(false).withDescription("Statistics sampling percentage").create("statisticssamplingpercent");
 
     protected String name;
-    protected String description;
     protected boolean isAsync = false;
     protected OptionsHelper optionsHelper = new OptionsHelper();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
index c8e74f6..d66e4eb 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultSslProtocolSocketFactory.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  */
 public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
     /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
+    private static Logger logger = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
     private SSLContext sslcontext = null;
 
     /**
@@ -135,7 +135,7 @@ public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFact
 
             return context;
         } catch (Exception e) {
-            LOG.error(e.getMessage(), e);
+            logger.error(e.getMessage(), e);
             throw new HttpClientError(e.toString());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
index d7901e5..4a8cfb6 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/DefaultX509TrustManager.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
 public class DefaultX509TrustManager implements X509TrustManager {
 
     /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
+    private static Logger logger = LoggerFactory.getLogger(DefaultX509TrustManager.class);
     private X509TrustManager standardTrustManager = null;
 
     /**
@@ -71,12 +71,12 @@ public class DefaultX509TrustManager implements X509TrustManager {
     }
 
     public boolean isServerTrusted(X509Certificate[] certificates) {
-        if ((certificates != null) && LOG.isDebugEnabled()) {
-            LOG.debug("Server certificate chain:");
+        if ((certificates != null) && logger.isDebugEnabled()) {
+            logger.debug("Server certificate chain:");
 
             for (int i = 0; i < certificates.length; i++) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
+                if (logger.isDebugEnabled()) {
+                    logger.debug("X509Certificate[" + i + "]=" + certificates[i]);
                 }
             }
         }
@@ -87,7 +87,7 @@ public class DefaultX509TrustManager implements X509TrustManager {
             try {
                 certificate.checkValidity();
             } catch (CertificateException e) {
-                LOG.error(e.toString());
+                logger.error(e.toString());
 
                 return false;
             }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
index f0363a2..9d016cc 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory;
  */
 public class HadoopCmdOutput {
 
-    protected static final Logger log = LoggerFactory.getLogger(HadoopCmdOutput.class);
+    protected static final Logger logger = LoggerFactory.getLogger(HadoopCmdOutput.class);
 
     private final StringBuilder output;
     private final Job job;
@@ -86,18 +86,18 @@ public class HadoopCmdOutput {
             Counters counters = job.getCounters();
             if (counters == null) {
                 String errorMsg = "no counters for job " + getMrJobId();
-                log.warn(errorMsg);
+                logger.warn(errorMsg);
                 output.append(errorMsg);
                 return;
             }
             this.output.append(counters.toString()).append("\n");
-            log.debug(counters.toString());
+            logger.debug(counters.toString());
 
             mapInputRecords = String.valueOf(counters.findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue());
             hdfsBytesWritten = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_WRITTEN").getValue());
             hdfsBytesRead = String.valueOf(counters.findCounter("FileSystemCounters", "HDFS_BYTES_READ").getValue());
         } catch (Exception e) {
-            log.error(e.getLocalizedMessage(), e);
+            logger.error(e.getLocalizedMessage(), e);
             output.append(e.getLocalizedMessage());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
index 53b8850..3a79a3d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
@@ -41,7 +41,7 @@ public class HadoopStatusGetter {
     private final String mrJobId;
     private final String yarnUrl;
 
-    protected static final Logger log = LoggerFactory.getLogger(HadoopStatusChecker.class);
+    protected static final Logger logger = LoggerFactory.getLogger(HadoopStatusChecker.class);
 
     public HadoopStatusGetter(String yarnUrl, String mrJobId) {
         this.yarnUrl = yarnUrl;
@@ -88,10 +88,10 @@ public class HadoopStatusGetter {
 
                 if (redirect == null) {
                     response = get.getResponseBodyAsString();
-                    log.debug("Job " + mrJobId + " get status check result.\n");
+                    logger.debug("Job " + mrJobId + " get status check result.\n");
                 } else {
                     url = redirect;
-                    log.debug("Job " + mrJobId + " check redirect url " + url + ".\n");
+                    logger.debug("Job " + mrJobId + " check redirect url " + url + ".\n");
                 }
             } finally {
                 get.releaseConnection();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
index 07d600e..6bcef28 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
@@ -44,7 +44,7 @@ import org.slf4j.LoggerFactory;
 /**
  */
 public class FactDistinctColumnsJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(FactDistinctColumnsJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(FactDistinctColumnsJob.class);
 
     @Override
     public int run(String[] args) throws Exception {
@@ -79,7 +79,7 @@ public class FactDistinctColumnsJob extends AbstractHadoopJob {
             job.getConfiguration().set(BatchConstants.CFG_STATISTICS_ENABLED, statistics_enabled);
             job.getConfiguration().set(BatchConstants.CFG_STATISTICS_OUTPUT, statistics_output);
             job.getConfiguration().set(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT, statistics_sampling_percent);
-            log.info("Starting: " + job.getJobName());
+            logger.info("Starting: " + job.getJobName());
 
             setJobClasspath(job);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
index f9b1dba..3fbeb35 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MetadataCleanupJob.java
@@ -44,7 +44,7 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
     @SuppressWarnings("static-access")
     private static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(false).withDescription("Delete the unused metadata").create("delete");
 
-    protected static final Logger log = LoggerFactory.getLogger(MetadataCleanupJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(MetadataCleanupJob.class);
 
     boolean delete = false;
 
@@ -61,13 +61,13 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
     public int run(String[] args) throws Exception {
         Options options = new Options();
 
-        log.info("----- jobs args: " + Arrays.toString(args));
+        logger.info("jobs args: " + Arrays.toString(args));
         try {
             options.addOption(OPTION_DELETE);
             parseOptions(options, args);
 
-            log.info("options: '" + getOptionsAsString() + "'");
-            log.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
+            logger.info("options: '" + getOptionsAsString() + "'");
+            logger.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
             delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
 
             config = KylinConfig.getInstanceFromEnv();
@@ -76,7 +76,7 @@ public class MetadataCleanupJob extends AbstractHadoopJob {
 
             return 0;
         } catch (Exception e) {
-            e.printStackTrace(System.err);
+            printUsage(options);
             throw e;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
index 463ed58..757c70d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/RangeKeyDistributionJob.java
@@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory;
  */
 
 public class RangeKeyDistributionJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(RangeKeyDistributionJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(RangeKeyDistributionJob.class);
 
     /*
      * (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultSslProtocolSocketFactory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultSslProtocolSocketFactory.java b/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultSslProtocolSocketFactory.java
index 3f40d42..2e41148 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultSslProtocolSocketFactory.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultSslProtocolSocketFactory.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  */
 public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFactory {
     /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
+    private static Logger logger = LoggerFactory.getLogger(DefaultSslProtocolSocketFactory.class);
     private SSLContext sslcontext = null;
 
     /**
@@ -133,7 +133,7 @@ public class DefaultSslProtocolSocketFactory implements SecureProtocolSocketFact
 
             return context;
         } catch (Exception e) {
-            LOG.error(e.getMessage(), e);
+            logger.error(e.getMessage(), e);
             throw new HttpClientError(e.toString());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultX509TrustManager.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultX509TrustManager.java b/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultX509TrustManager.java
index b4347f3..35be21b 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultX509TrustManager.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/util/DefaultX509TrustManager.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
 public class DefaultX509TrustManager implements X509TrustManager {
 
     /** Log object for this class. */
-    private static Logger LOG = LoggerFactory.getLogger(DefaultX509TrustManager.class);
+    private static Logger logger = LoggerFactory.getLogger(DefaultX509TrustManager.class);
     private X509TrustManager standardTrustManager = null;
 
     /**
@@ -71,12 +71,12 @@ public class DefaultX509TrustManager implements X509TrustManager {
     }
 
     public boolean isServerTrusted(X509Certificate[] certificates) {
-        if ((certificates != null) && LOG.isDebugEnabled()) {
-            LOG.debug("Server certificate chain:");
+        if ((certificates != null) && logger.isDebugEnabled()) {
+            logger.debug("Server certificate chain:");
 
             for (int i = 0; i < certificates.length; i++) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("X509Certificate[" + i + "]=" + certificates[i]);
+                if (logger.isDebugEnabled()) {
+                    logger.debug("X509Certificate[" + i + "]=" + certificates[i]);
                 }
             }
         }
@@ -87,7 +87,7 @@ public class DefaultX509TrustManager implements X509TrustManager {
             try {
                 certificate.checkValidity();
             } catch (CertificateException e) {
-                LOG.error(e.toString());
+                logger.error(e.toString());
 
                 return false;
             }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
index c4d56ad..5c94542 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/OrphanHBaseCleanJob.java
@@ -47,7 +47,7 @@ public class OrphanHBaseCleanJob extends AbstractHadoopJob {
     @SuppressWarnings("static-access")
     private static final Option OPTION_WHITELIST = OptionBuilder.withArgName("whitelist").hasArg().isRequired(true).withDescription("metadata store whitelist, separated with comma").create("whitelist");
 
-    protected static final Logger log = LoggerFactory.getLogger(OrphanHBaseCleanJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(OrphanHBaseCleanJob.class);
 
     boolean delete = false;
     Set<String> metastoreWhitelistSet = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
@@ -56,19 +56,19 @@ public class OrphanHBaseCleanJob extends AbstractHadoopJob {
     public int run(String[] args) throws Exception {
         Options options = new Options();
 
-        log.info("----- jobs args: " + Arrays.toString(args));
+        logger.info("jobs args: " + Arrays.toString(args));
         try {
             options.addOption(OPTION_DELETE);
             options.addOption(OPTION_WHITELIST);
             parseOptions(options, args);
 
-            log.info("options: '" + getOptionsAsString() + "'");
-            log.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
+            logger.info("options: '" + getOptionsAsString() + "'");
+            logger.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
             delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
             String[] metastoreWhitelist = getOptionValue(OPTION_WHITELIST).split(",");
 
             for (String ms : metastoreWhitelist) {
-                log.info("metadata store in white list: " + ms);
+                logger.info("metadata store in white list: " + ms);
                 metastoreWhitelistSet.add(ms);
             }
 
@@ -93,27 +93,27 @@ public class OrphanHBaseCleanJob extends AbstractHadoopJob {
         for (HTableDescriptor desc : tableDescriptors) {
             String host = desc.getValue(IRealizationConstants.HTableTag);
             if (!metastoreWhitelistSet.contains(host)) {
-                log.info("HTable {} is recognized as orphan because its tag is {}", desc.getTableName(), host);
+                logger.info("HTable {} is recognized as orphan because its tag is {}", desc.getTableName(), host);
                 //collect orphans
                 allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
             } else {
-                log.info("HTable {} belongs to {}", desc.getTableName(), host);
+                logger.info("HTable {} belongs to {}", desc.getTableName(), host);
             }
         }
 
         if (delete == true) {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
-                log.info("Deleting HBase table " + htableName);
+                logger.info("Deleting HBase table " + htableName);
                 if (hbaseAdmin.tableExists(htableName)) {
                     if (hbaseAdmin.isTableEnabled(htableName)) {
                         hbaseAdmin.disableTable(htableName);
                     }
 
                     hbaseAdmin.deleteTable(htableName);
-                    log.info("Deleted HBase table " + htableName);
+                    logger.info("Deleted HBase table " + htableName);
                 } else {
-                    log.info("HBase table" + htableName + " does not exist");
+                    logger.info("HBase table" + htableName + " does not exist");
                 }
             }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index d83e455..c7c54ea 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -53,15 +53,12 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @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);
+    protected static final Logger logger = LoggerFactory.getLogger(StorageCleanupJob.class);
 
     public static final long TIME_THREADSHOLD = 2 * 24 * 3600 * 1000l; // 2 days
 
@@ -78,13 +75,13 @@ public class StorageCleanupJob extends AbstractHadoopJob {
     public int run(String[] args) throws Exception {
         Options options = new Options();
 
-        log.info("----- jobs args: " + Arrays.toString(args));
+        logger.info("jobs args: " + Arrays.toString(args));
         try {
             options.addOption(OPTION_DELETE);
             parseOptions(options, args);
 
-            log.info("options: '" + getOptionsAsString() + "'");
-            log.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
+            logger.info("options: '" + getOptionsAsString() + "'");
+            logger.info("delete option value: '" + getOptionValue(OPTION_DELETE) + "'");
             delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
 
             Configuration conf = HBaseConfiguration.create(getConf());
@@ -95,7 +92,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
             return 0;
         } catch (Exception e) {
-            e.printStackTrace(System.err);
+            printUsage(options);
             throw e;
         }
     }
@@ -126,7 +123,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
                 String tablename = seg.getStorageLocationIdentifier();
                 if (allTablesNeedToBeDropped.contains(tablename)) {
                     allTablesNeedToBeDropped.remove(tablename);
-                    log.info("Exclude table " + tablename + " from drop list, as the table belongs to cube " + cube.getName() + " with status " + cube.getStatus());
+                    logger.info("Exclude table " + tablename + " from drop list, as the table belongs to cube " + cube.getName() + " with status " + cube.getStatus());
                 }
             }
         }
@@ -138,7 +135,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
                 if (allTablesNeedToBeDropped.contains(tablename)) {
                     allTablesNeedToBeDropped.remove(tablename);
-                    log.info("Exclude table " + tablename + " from drop list, as the table belongs to ii " + ii.getName() + " with status " + ii.getStatus());
+                    logger.info("Exclude table " + tablename + " from drop list, as the table belongs to ii " + ii.getName() + " with status " + ii.getStatus());
                 }
             }
         }
@@ -146,16 +143,16 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         if (delete == true) {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
-                log.info("Deleting HBase table " + htableName);
+                logger.info("Deleting HBase table " + htableName);
                 if (hbaseAdmin.tableExists(htableName)) {
                     if (hbaseAdmin.isTableEnabled(htableName)) {
                         hbaseAdmin.disableTable(htableName);
                     }
 
                     hbaseAdmin.deleteTable(htableName);
-                    log.info("Deleted HBase table " + htableName);
+                    logger.info("Deleted HBase table " + htableName);
                 } else {
-                    log.info("HBase table" + htableName + " does not exist");
+                    logger.info("HBase table" + htableName + " does not exist");
                 }
             }
         } else {
@@ -195,7 +192,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             if (!state.isFinalState()) {
                 String path = JobInstance.getJobWorkingDir(jobId, engineConfig.getHdfsWorkingDirectory());
                 allHdfsPathsNeedToBeDeleted.remove(path);
-                log.info("Remove " + path + " from deletion list, as the path belongs to job " + jobId + " with status " + state);
+                logger.info("Remove " + path + " from deletion list, as the path belongs to job " + jobId + " with status " + state);
             }
         }
 
@@ -206,7 +203,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
                 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());
+                    logger.info("Remove " + path + " from deletion list, as the path belongs to segment " + seg + " of cube " + cube.getName());
                 }
             }
         }
@@ -214,13 +211,13 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         if (delete == true) {
             // remove files
             for (String hdfsPath : allHdfsPathsNeedToBeDeleted) {
-                log.info("Deleting hdfs path " + hdfsPath);
+                logger.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);
+                    logger.info("Deleted hdfs path " + hdfsPath);
                 } else {
-                    log.info("Hdfs path " + hdfsPath + "does not exist");
+                    logger.info("Hdfs path " + hdfsPath + "does not exist");
                 }
             }
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
index 90f85fa..b2282f1 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  */
 public class IICreateHFileJob extends AbstractHadoopJob {
 
-    protected static final Logger log = LoggerFactory.getLogger(IICreateHFileJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(IICreateHFileJob.class);
 
     public int run(String[] args) throws Exception {
         Options options = new Options();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
index d6d6d4e..042678e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIDistinctColumnsJob.java
@@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory;
  * @author yangli9
  */
 public class IIDistinctColumnsJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(IIDistinctColumnsJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(IIDistinctColumnsJob.class);
 
     @Override
     public int run(String[] args) throws Exception {
@@ -65,7 +65,7 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
 
             // ----------------------------------------------------------------------------
 
-            log.info("Starting: " + job.getJobName() + " on table " + tableName);
+            logger.info("Starting: " + job.getJobName() + " on table " + tableName);
 
             IIManager iiMgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
             IIInstance ii = iiMgr.getII(iiName);
@@ -102,7 +102,7 @@ public class IIDistinctColumnsJob extends AbstractHadoopJob {
         String tableName = job.getConfiguration().get(BatchConstants.TABLE_NAME);
         String[] dbTableNames = HadoopUtil.parseHiveTableName(tableName);
 
-        log.info("setting hcat input format, db name {} , table name {}", dbTableNames[0], dbTableNames[1]);
+        logger.info("setting hcat input format, db name {} , table name {}", dbTableNames[0], dbTableNames[1]);
 
         HCatInputFormat.setInput(job, dbTableNames[0], dbTableNames[1]);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
index cd408fd..c9ad448 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/InvertedIndexJob.java
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
  * @author yangli9
  */
 public class InvertedIndexJob extends AbstractHadoopJob {
-    protected static final Logger log = LoggerFactory.getLogger(InvertedIndexJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(InvertedIndexJob.class);
 
     @Override
     public int run(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
index 1f3afcf..02c36b7 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
@@ -34,9 +34,7 @@ import org.slf4j.LoggerFactory;
  */
 public class CleanHtableCLI extends AbstractHadoopJob {
 
-    protected static final Logger log = LoggerFactory.getLogger(CleanHtableCLI.class);
-
-    String tableName;
+    protected static final Logger logger = LoggerFactory.getLogger(CleanHtableCLI.class);
 
     @Override
     public int run(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
index 2eaa737..7e66930 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
@@ -41,7 +41,7 @@ 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);
+    protected static final Logger logger = LoggerFactory.getLogger(HtableAlterMetadataCLI.class);
 
     String tableName;
     String metadataKey;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
index 29e9457..2be61f4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/BulkLoadJob.java
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  */
 public class BulkLoadJob extends AbstractHadoopJob {
 
-    protected static final Logger log = LoggerFactory.getLogger(BulkLoadJob.class);
+    protected static final Logger logger = LoggerFactory.getLogger(BulkLoadJob.class);
 
     @Override
     public int run(String[] args) throws Exception {
@@ -83,9 +83,9 @@ public class BulkLoadJob extends AbstractHadoopJob {
             newArgs[0] = input;
             newArgs[1] = tableName;
 
-            log.debug("Start to run LoadIncrementalHFiles");
+            logger.debug("Start to run LoadIncrementalHFiles");
             int ret = ToolRunner.run(new LoadIncrementalHFiles(conf), newArgs);
-            log.debug("End to run LoadIncrementalHFiles");
+            logger.debug("End to run LoadIncrementalHFiles");
             return ret;
         } catch (Exception e) {
             printUsage(options);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/175392c6/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
index 1d28283..621909a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.storage.hbase.util;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -48,7 +47,7 @@ public class HBaseClean extends AbstractHadoopJob {
     @SuppressWarnings("static-access")
     private static final Option OPTION_TAG = OptionBuilder.withArgName("tag").hasArg().isRequired(true).withDescription("the tag of HTable").create("tag");
 
-    protected static final Logger log = LoggerFactory.getLogger(HBaseClean.class);
+    protected static final Logger logger = LoggerFactory.getLogger(HBaseClean.class);
     boolean delete = false;
     String tag = null;
 
@@ -56,13 +55,13 @@ public class HBaseClean extends AbstractHadoopJob {
     public int run(String[] args) throws Exception {
         Options options = new Options();
 
-        log.info("----- jobs args: " + Arrays.toString(args));
+        logger.info("jobs args: " + Arrays.toString(args));
         try {
             options.addOption(OPTION_DELETE);
             options.addOption(OPTION_TAG);
             parseOptions(options, args);
 
-            log.info("options: '" + getOptionsAsString() + "'");
+            logger.info("options: '" + getOptionsAsString() + "'");
             
             tag = getOptionValue(OPTION_TAG);
             delete = Boolean.parseBoolean(getOptionValue(OPTION_DELETE));
@@ -94,16 +93,16 @@ public class HBaseClean extends AbstractHadoopJob {
             if (delete) {
                 // drop tables
                 for (String htableName : allTablesNeedToBeDropped) {
-                    log.info("Deleting HBase table " + htableName);
+                    logger.info("Deleting HBase table " + htableName);
                     if (hbaseAdmin.tableExists(htableName)) {
                         if (hbaseAdmin.isTableEnabled(htableName)) {
                             hbaseAdmin.disableTable(htableName);
                         }
 
                         hbaseAdmin.deleteTable(htableName);
-                        log.info("Deleted HBase table " + htableName);
+                        logger.info("Deleted HBase table " + htableName);
                     } else {
-                        log.info("HBase table" + htableName + " does not exist");
+                        logger.info("HBase table" + htableName + " does not exist");
                     }
                 }
             } else {



[20/28] incubator-kylin git commit: KYLIN-991 Add timestamp for htables, and check the time before cleanup

Posted by ma...@apache.org.
KYLIN-991 Add timestamp for htables, and check the time before cleanup

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

Branch: refs/heads/2.x-staging
Commit: c7990735c6f901d64adbd8927afd57fd63a69e51
Parents: d2067c4
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 17:16:51 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 17:16:51 2015 +0800

----------------------------------------------------------------------
 .../realization/IRealizationConstants.java       |  3 +++
 .../kylin/job/hadoop/cube/StorageCleanupJob.java | 19 ++++++++++++-------
 .../hadoop/invertedindex/IICreateHTableJob.java  |  1 +
 .../storage/hbase/steps/CubeHTableUtil.java      |  1 +
 4 files changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c7990735/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationConstants.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationConstants.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationConstants.java
index 0f2c012..7cbcdf8 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationConstants.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/IRealizationConstants.java
@@ -32,4 +32,7 @@ public class IRealizationConstants {
      */
     public final static String HTableTag = "KYLIN_HOST";
 
+
+    public final static String HTableCreationTime = "CREATION_TIME";
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c7990735/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index 53b489e..489b9ed 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -18,14 +18,10 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,6 +48,11 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 /**
  * @author ysong1
  */
@@ -62,6 +63,8 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
     protected static final Logger log = LoggerFactory.getLogger(StorageCleanupJob.class);
 
+    public static final long TIME_THREADSHOLD = 2 * 24 * 3600 * 1000l; // 2 days
+
     boolean delete = false;
 
     protected static ExecutableManager executableManager = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
@@ -108,9 +111,11 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
         for (HTableDescriptor desc : tableDescriptors) {
             String host = desc.getValue(IRealizationConstants.HTableTag);
+            String creationTime = desc.getValue(IRealizationConstants.HTableCreationTime);
             if (KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix().equalsIgnoreCase(host)) {
-                //only take care htables that belongs to self
-                allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
+                //only take care htables that belongs to self, and created more than 2 days
+                if (StringUtils.isNotEmpty(creationTime) || (System.currentTimeMillis() - Long.valueOf(creationTime) > TIME_THREADSHOLD))
+                    allTablesNeedToBeDropped.add(desc.getTableName().getNameAsString());
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c7990735/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
index 258f519..22ed3a2 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
@@ -99,6 +99,7 @@ public class IICreateHTableJob extends AbstractHadoopJob {
             cf.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
             tableDesc.addFamily(cf);
             tableDesc.setValue(IRealizationConstants.HTableTag, config.getMetadataUrlPrefix());
+            tableDesc.setValue(IRealizationConstants.HTableCreationTime, String.valueOf(System.currentTimeMillis()));
             tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName());
 
             Configuration conf = HBaseConfiguration.create(getConf());

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c7990735/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index 7b0dad9..221be8a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -35,6 +35,7 @@ public class CubeHTableUtil {
         // https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html
         tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
         tableDesc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
+        tableDesc.setValue(IRealizationConstants.HTableCreationTime, String.valueOf(System.currentTimeMillis()));
 
         Configuration conf = HBaseConfiguration.create();
         HBaseAdmin admin = new HBaseAdmin(conf);


[12/28] incubator-kylin git commit: KYLIN-972 MR_V2 build well and pass query test

Posted by ma...@apache.org.
KYLIN-972 MR_V2 build well and pass query test


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

Branch: refs/heads/2.x-staging
Commit: a7c272c6f91c31b127e2faf8018d8fb62412e5b3
Parents: a79db05
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Sep 2 16:11:49 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Sep 2 16:11:49 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     | 15 +++++---
 .../java/org/apache/kylin/cube/CubeManager.java | 39 ++++++++++----------
 .../apache/kylin/rest/service/CubeService.java  |  6 +--
 3 files changed, 31 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a7c272c6/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index fcd338c..81aa4d6 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -50,6 +50,10 @@ import com.google.common.collect.Lists;
 @SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class CubeInstance extends RootPersistentEntity implements IRealization, IBuildable {
+    private static final int COST_WEIGHT_DIMENSION = 1;
+    private static final int COST_WEIGHT_MEASURE = 1;
+    private static final int COST_WEIGHT_LOOKUP_TABLE = 1;
+    private static final int COST_WEIGHT_INNER_JOIN = 2;
 
     public static CubeInstance create(String cubeName, String projectName, CubeDesc cubeDesc) {
         CubeInstance cubeInstance = new CubeInstance();
@@ -70,7 +74,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
 
         return cubeInstance;
     }
-
+    
     @JsonIgnore
     private KylinConfig config;
     @JsonProperty("name")
@@ -104,10 +108,9 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
 
     private String projectName;
 
-    private static final int COST_WEIGHT_DIMENSION = 1;
-    private static final int COST_WEIGHT_MEASURE = 1;
-    private static final int COST_WEIGHT_LOOKUP_TABLE = 1;
-    private static final int COST_WEIGHT_INNER_JOIN = 2;
+    // default constructor for jackson
+    public CubeInstance() {
+    }
 
     public List<CubeSegment> getBuildingSegments() {
         List<CubeSegment> buildingSegments = new ArrayList<CubeSegment>();
@@ -134,7 +137,7 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
         }
         return mergingSegments;
     }
-
+    
     public CubeDesc getDescriptor() {
         return CubeDescManager.getInstance(config).getCubeDesc(descName);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a7c272c6/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index 5cfecf1..c6f066c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -254,8 +254,8 @@ public class CubeManager implements IRealizationProvider {
         return cube;
     }
 
-    public CubeInstance updateCube(CubeUpdate cubeBuilder) throws IOException {
-        return updateCube(cubeBuilder, 0);
+    public CubeInstance updateCube(CubeUpdate update) throws IOException {
+        return updateCube(update, 0);
     }
 
     private boolean validateReadySegments(CubeInstance cube) {
@@ -283,22 +283,22 @@ public class CubeManager implements IRealizationProvider {
         return true;
     }
 
-    private CubeInstance updateCube(CubeUpdate cubeBuilder, int retry) throws IOException {
-        if (cubeBuilder == null || cubeBuilder.getCubeInstance() == null)
+    private CubeInstance updateCube(CubeUpdate update, int retry) throws IOException {
+        if (update == null || update.getCubeInstance() == null)
             throw new IllegalStateException();
 
-        CubeInstance cube = cubeBuilder.getCubeInstance();
+        CubeInstance cube = update.getCubeInstance();
         logger.info("Updating cube instance '" + cube.getName() + "'");
 
-        if (cubeBuilder.getToAddSegs() != null)
-            cube.getSegments().addAll(Arrays.asList(cubeBuilder.getToAddSegs()));
+        if (update.getToAddSegs() != null)
+            cube.getSegments().addAll(Arrays.asList(update.getToAddSegs()));
 
         List<String> toRemoveResources = Lists.newArrayList();
-        if (cubeBuilder.getToRemoveSegs() != null) {
+        if (update.getToRemoveSegs() != null) {
             Iterator<CubeSegment> iterator = cube.getSegments().iterator();
             while (iterator.hasNext()) {
                 CubeSegment currentSeg = iterator.next();
-                for (CubeSegment toRemoveSeg : cubeBuilder.getToRemoveSegs()) {
+                for (CubeSegment toRemoveSeg : update.getToRemoveSegs()) {
                     if (currentSeg.getUuid().equals(toRemoveSeg.getUuid())) {
                         iterator.remove();
                         toRemoveResources.add(toRemoveSeg.getStatisticsResourcePath());
@@ -308,14 +308,13 @@ public class CubeManager implements IRealizationProvider {
 
         }
 
-        if (cubeBuilder.getToUpdateSegs() != null) {
-            for (CubeSegment segment : cubeBuilder.getToUpdateSegs()) {
+        if (update.getToUpdateSegs() != null) {
+            for (CubeSegment segment : update.getToUpdateSegs()) {
                 for (int i = 0; i < cube.getSegments().size(); i++) {
                     if (cube.getSegments().get(i).getUuid().equals(segment.getUuid())) {
                         cube.getSegments().set(i, segment);
                     }
                 }
-
             }
         }
 
@@ -325,16 +324,16 @@ public class CubeManager implements IRealizationProvider {
             throw new IllegalStateException("Has invalid Ready segments in cube " + cube.getName());
         }
 
-        if (cubeBuilder.getStatus() != null) {
-            cube.setStatus(cubeBuilder.getStatus());
+        if (update.getStatus() != null) {
+            cube.setStatus(update.getStatus());
         }
 
-        if (cubeBuilder.getOwner() != null) {
-            cube.setOwner(cubeBuilder.getOwner());
+        if (update.getOwner() != null) {
+            cube.setOwner(update.getOwner());
         }
 
-        if (cubeBuilder.getCost() > 0) {
-            cube.setCost(cubeBuilder.getCost());
+        if (update.getCost() > 0) {
+            cube.setCost(update.getCost());
         }
 
         try {
@@ -347,9 +346,9 @@ public class CubeManager implements IRealizationProvider {
             }
 
             cube = reloadCubeLocal(cube.getName());
-            cubeBuilder.setCubeInstance(cube);
+            update.setCubeInstance(cube);
             retry++;
-            cube = updateCube(cubeBuilder, retry);
+            cube = updateCube(update, retry);
         }
 
         if (toRemoveResources.size() > 0) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a7c272c6/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index a5cc08f..ce51623 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -550,9 +550,9 @@ public class CubeService extends BasicService {
                 getExecutableManager().discardJob(cubingJob.getId());
             }
         }
-        CubeUpdate cubeBuilder = new CubeUpdate(cube);
-        cubeBuilder.setToRemoveSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
-        return CubeManager.getInstance(getConfig()).updateCube(cubeBuilder);
+        CubeUpdate update = new CubeUpdate(cube);
+        update.setToRemoveSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
+        return CubeManager.getInstance(getConfig()).updateCube(update);
     }
 
     @PreAuthorize(Constant.ACCESS_HAS_ROLE_MODELER + " or " + Constant.ACCESS_HAS_ROLE_ADMIN)


[06/28] incubator-kylin git commit: KYLIN-972 Make MR_V2 the default engine for new cubes. Old cubes (0.7) continue to build with MR_V1 engine.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
deleted file mode 100644
index 15d8edb..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedDynamicStorageEngine.java
+++ /dev/null
@@ -1,149 +0,0 @@
-package org.apache.kylin.storage.cache;
-
-import java.util.List;
-
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.Element;
-
-import org.apache.kylin.common.util.RangeUtil;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.realization.SQLDigestUtil;
-import org.apache.kylin.metadata.realization.StreamSQLDigest;
-import org.apache.kylin.metadata.tuple.CompoundTupleIterator;
-import org.apache.kylin.metadata.tuple.ITuple;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
-import org.apache.kylin.metadata.tuple.TeeTupleIterator;
-import org.apache.kylin.storage.ICachableStorageQuery;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.tuple.TupleInfo;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Range;
-
-/**
- */
-public class CacheFledgedDynamicStorageEngine extends AbstractCacheFledgedStorageEngine {
-    private static final Logger logger = LoggerFactory.getLogger(CacheFledgedDynamicStorageEngine.class);
-
-    private final TblColRef partitionColRef;
-
-    private Range<Long> ts;
-
-    public CacheFledgedDynamicStorageEngine(ICachableStorageQuery underlyingStorage, TblColRef partitionColRef) {
-        super(underlyingStorage);
-        this.partitionColRef = partitionColRef;
-
-        Preconditions.checkArgument(this.partitionColRef != null, "For dynamic columns like " + //
-                this.underlyingStorage.getStorageUUID() + ", partition column must be provided");
-    }
-
-    @Override
-    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
-        //check if ts condition in sqlDigest valid
-        ts = TsConditionExtractor.extractTsCondition(partitionColRef, sqlDigest.filter);
-        if (ts == null || ts.isEmpty()) {
-            logger.info("ts range in the query conflicts,return empty directly");
-            return ITupleIterator.EMPTY_TUPLE_ITERATOR;
-        }
-
-        //enable dynamic cache iff group by columns contains partition col
-        //because cache extraction requires partition col value as selection key
-        boolean needUpdateCache = sqlDigest.groupbyColumns.contains(partitionColRef);
-
-        streamSQLDigest = new StreamSQLDigest(sqlDigest, partitionColRef);
-        StreamSQLResult cachedResult = null;
-        Cache cache = CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID());
-        Element element = cache.get(streamSQLDigest.hashCode());
-        if (element != null) {
-            this.queryCacheExists = true;
-            cachedResult = (StreamSQLResult) element.getObjectValue();
-        }
-
-        ITupleIterator ret = null;
-        if (cachedResult != null) {
-            Range<Long> reusePeriod = cachedResult.getReusableResults(ts);
-
-            logger.info("existing cache    : " + cachedResult);
-            logger.info("ts Range in query: " + RangeUtil.formatTsRange(ts));
-            logger.info("potential reusable range   : " + RangeUtil.formatTsRange(reusePeriod));
-
-            if (reusePeriod != null) {
-                List<Range<Long>> remainings = RangeUtil.remove(ts, reusePeriod);
-                if (remainings.size() == 1) {//if using cache causes two underlyingStorage searches, we'd rather not use the cache
-
-                    SimpleTupleIterator reusedTuples = new SimpleTupleIterator(cachedResult.reuse(reusePeriod));
-                    List<ITupleIterator> iTupleIteratorList = Lists.newArrayList();
-                    iTupleIteratorList.add(reusedTuples);
-
-                    for (Range<Long> remaining : remainings) {//actually there will be only one loop
-                        logger.info("Appending ts " + RangeUtil.formatTsRange(remaining) + " as additional filter");
-
-                        ITupleIterator freshTuples = SQLDigestUtil.appendTsFilterToExecute(sqlDigest, partitionColRef, remaining, new Function<Void, ITupleIterator>() {
-                            @Override
-                            public ITupleIterator apply(Void input) {
-                                return underlyingStorage.search(context, sqlDigest, returnTupleInfo);
-                            }
-                        });
-                        iTupleIteratorList.add(freshTuples);
-                    }
-
-                    ret = new CompoundTupleIterator(iTupleIteratorList);
-                } else if (remainings.size() == 0) {
-                    logger.info("The ts range in new query was fully cached");
-                    needUpdateCache = false;
-                    ret = new SimpleTupleIterator(cachedResult.reuse(reusePeriod));
-                } else {
-                    //if using cache causes more than one underlyingStorage searches
-                    //the incurred overhead might be more expensive than the cache benefit
-                    logger.info("Give up using cache to avoid complexity");
-                }
-            }
-        } else {
-            logger.info("no cache entry for this query");
-        }
-
-        if (ret == null) {
-            ret = underlyingStorage.search(context, sqlDigest, returnTupleInfo);
-            logger.info("No Cache being used");
-        } else {
-            logger.info("Cache being used");
-        }
-
-        if (needUpdateCache) {
-            //use another nested ITupleIterator to deal with cache
-            final TeeTupleIterator tee = new TeeTupleIterator(ret);
-            tee.addCloseListener(this);
-            return tee;
-        } else {
-            return ret;
-        }
-    }
-
-    @Override
-    public void notify(List<ITuple> duplicated, long createTime) {
-
-        Range<Long> cacheExclude = this.underlyingStorage.getVolatilePeriod();
-        if (cacheExclude != null) {
-            List<Range<Long>> cachablePeriods = RangeUtil.remove(ts, cacheExclude);
-            if (cachablePeriods.size() == 1) {
-                if (!ts.equals(cachablePeriods.get(0))) {
-                    logger.info("With respect to growing storage, the cacheable tsRange shrinks from " + RangeUtil.formatTsRange(ts) + " to " + RangeUtil.formatTsRange(cachablePeriods.get(0)));
-                }
-                ts = cachablePeriods.get(0);
-            } else {
-                //give up updating the cache, in avoid to make cache complicated
-                logger.info("Skip updating cache to avoid complexity");
-            }
-        }
-
-        StreamSQLResult newCacheEntry = new StreamSQLResult(duplicated, ts, partitionColRef);
-        CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID()).put(new Element(streamSQLDigest.hashCode(), newCacheEntry));
-        logger.info("cache after the query: " + newCacheEntry);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticQuery.java
new file mode 100644
index 0000000..2272aac
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticQuery.java
@@ -0,0 +1,88 @@
+package org.apache.kylin.storage.cache;
+
+import java.util.List;
+
+import net.sf.ehcache.Cache;
+import net.sf.ehcache.Element;
+
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.realization.StreamSQLDigest;
+import org.apache.kylin.metadata.tuple.ITuple;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
+import org.apache.kylin.metadata.tuple.TeeTupleIterator;
+import org.apache.kylin.storage.ICachableStorageQuery;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.tuple.TupleInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Ranges;
+
+public class CacheFledgedStaticQuery extends AbstractCacheFledgedQuery {
+    private static final Logger logger = LoggerFactory.getLogger(CacheFledgedStaticQuery.class);
+
+    public CacheFledgedStaticQuery(ICachableStorageQuery underlyingStorage) {
+        super(underlyingStorage);
+    }
+
+    @Override
+    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
+
+        streamSQLDigest = new StreamSQLDigest(sqlDigest, null);
+        StreamSQLResult cachedResult = null;
+        Cache cache = CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID());
+        Element element = cache.get(streamSQLDigest.hashCode());
+        if (element != null) {
+            this.queryCacheExists = true;
+            cachedResult = (StreamSQLResult) element.getObjectValue();
+        }
+
+        ITupleIterator ret = null;
+        if (cachedResult != null) {
+            ret = new SimpleTupleIterator(cachedResult.reuse(Ranges.<Long> all()));
+        } else {
+            logger.info("no cache entry for this query");
+        }
+
+        if (ret == null) {
+            logger.info("decision: not using cache");
+            ret = underlyingStorage.search(context, sqlDigest, returnTupleInfo);
+        } else {
+            logger.info("decision: use cache");
+        }
+
+        if (!queryCacheExists) {
+            //use another nested ITupleIterator to deal with cache
+            final TeeTupleIterator tee = new TeeTupleIterator(ret);
+            tee.addCloseListener(this);
+            return tee;
+        } else {
+            return ret;
+        }
+    }
+
+    @Override
+    public void notify(List<ITuple> duplicated, long createTime) {
+        boolean cacheIt = true;
+        //        long storageQueryTime = System.currentTimeMillis() - createTime;
+        //        long durationThreshold = KylinConfig.getInstanceFromEnv().getQueryDurationCacheThreshold();
+        //        long scancountThreshold = KylinConfig.getInstanceFromEnv().getQueryScanCountCacheThreshold();
+        //
+        //        if (storageQueryTime < durationThreshold) {
+        //            logger.info("Skip storage caching for storage cache because storage query time {} less than {}", storageQueryTime, durationThreshold);
+        //            cacheIt = false;
+        //        }
+        //
+        //        if (duplicated.size() < scancountThreshold) {
+        //            logger.info("Skip storage caching for storage cache because scan count {} less than {}", duplicated.size(), scancountThreshold);
+        //            cacheIt = false;
+        //        }
+
+        if (cacheIt) {
+            StreamSQLResult newCacheEntry = new StreamSQLResult(duplicated, Ranges.<Long> all(), null);
+            CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID()).put(new Element(streamSQLDigest.hashCode(), newCacheEntry));
+            logger.info("cache after the query: " + newCacheEntry);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java b/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
deleted file mode 100644
index 9e2287f..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/cache/CacheFledgedStaticStorageEngine.java
+++ /dev/null
@@ -1,88 +0,0 @@
-package org.apache.kylin.storage.cache;
-
-import java.util.List;
-
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.Element;
-
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.realization.StreamSQLDigest;
-import org.apache.kylin.metadata.tuple.ITuple;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.metadata.tuple.SimpleTupleIterator;
-import org.apache.kylin.metadata.tuple.TeeTupleIterator;
-import org.apache.kylin.storage.ICachableStorageQuery;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.tuple.TupleInfo;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Ranges;
-
-public class CacheFledgedStaticStorageEngine extends AbstractCacheFledgedStorageEngine {
-    private static final Logger logger = LoggerFactory.getLogger(CacheFledgedStaticStorageEngine.class);
-
-    public CacheFledgedStaticStorageEngine(ICachableStorageQuery underlyingStorage) {
-        super(underlyingStorage);
-    }
-
-    @Override
-    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
-
-        streamSQLDigest = new StreamSQLDigest(sqlDigest, null);
-        StreamSQLResult cachedResult = null;
-        Cache cache = CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID());
-        Element element = cache.get(streamSQLDigest.hashCode());
-        if (element != null) {
-            this.queryCacheExists = true;
-            cachedResult = (StreamSQLResult) element.getObjectValue();
-        }
-
-        ITupleIterator ret = null;
-        if (cachedResult != null) {
-            ret = new SimpleTupleIterator(cachedResult.reuse(Ranges.<Long> all()));
-        } else {
-            logger.info("no cache entry for this query");
-        }
-
-        if (ret == null) {
-            logger.info("decision: not using cache");
-            ret = underlyingStorage.search(context, sqlDigest, returnTupleInfo);
-        } else {
-            logger.info("decision: use cache");
-        }
-
-        if (!queryCacheExists) {
-            //use another nested ITupleIterator to deal with cache
-            final TeeTupleIterator tee = new TeeTupleIterator(ret);
-            tee.addCloseListener(this);
-            return tee;
-        } else {
-            return ret;
-        }
-    }
-
-    @Override
-    public void notify(List<ITuple> duplicated, long createTime) {
-        boolean cacheIt = true;
-        //        long storageQueryTime = System.currentTimeMillis() - createTime;
-        //        long durationThreshold = KylinConfig.getInstanceFromEnv().getQueryDurationCacheThreshold();
-        //        long scancountThreshold = KylinConfig.getInstanceFromEnv().getQueryScanCountCacheThreshold();
-        //
-        //        if (storageQueryTime < durationThreshold) {
-        //            logger.info("Skip storage caching for storage cache because storage query time {} less than {}", storageQueryTime, durationThreshold);
-        //            cacheIt = false;
-        //        }
-        //
-        //        if (duplicated.size() < scancountThreshold) {
-        //            logger.info("Skip storage caching for storage cache because scan count {} less than {}", duplicated.size(), scancountThreshold);
-        //            cacheIt = false;
-        //        }
-
-        if (cacheIt) {
-            StreamSQLResult newCacheEntry = new StreamSQLResult(duplicated, Ranges.<Long> all(), null);
-            CACHE_MANAGER.getCache(this.underlyingStorage.getStorageUUID()).put(new Element(streamSQLDigest.hashCode(), newCacheEntry));
-            logger.info("cache after the query: " + newCacheEntry);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java b/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
index 64d729a..f77cc35 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/exception/ScanOutOfLimitException.java
@@ -14,7 +14,7 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
-*/
+ */
 
 package org.apache.kylin.storage.exception;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
index 997e7b2..e9f0975 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridInstance.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.storage.hybrid;
 
 import java.util.Collections;
@@ -26,7 +43,7 @@ import com.google.common.collect.Lists;
 
 /**
  */
-
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
 public class HybridInstance extends RootPersistentEntity implements IRealization {
 
@@ -265,4 +282,9 @@ public class HybridInstance extends RootPersistentEntity implements IRealization
         }
         return null;
     }
+
+    @Override
+    public int getStorageType() {
+        return ID_HYBRID;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
index 9eb8922..0f00f1a 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.kylin.storage.hybrid;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorage.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorage.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorage.java
new file mode 100644
index 0000000..f669c3d
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorage.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kylin.storage.hybrid;
+
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.storage.IStorage;
+import org.apache.kylin.storage.IStorageQuery;
+
+public class HybridStorage implements IStorage {
+
+    @Override
+    public IStorageQuery createQuery(IRealization realization) {
+        return new HybridStorageQuery((HybridInstance) realization);
+    }
+
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        throw new UnsupportedOperationException();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
deleted file mode 100644
index 499cd48..0000000
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageEngine.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package org.apache.kylin.storage.hybrid;
-
-import java.util.List;
-
-import org.apache.kylin.metadata.realization.IRealization;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.tuple.CompoundTupleIterator;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.storage.IStorageQuery;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.StorageQueryFactory;
-import org.apache.kylin.storage.tuple.TupleInfo;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class HybridStorageEngine implements IStorageQuery {
-
-    private IRealization[] realizations;
-    private IStorageQuery[] storageEngines;
-
-    public HybridStorageEngine(HybridInstance hybridInstance) {
-        this.realizations = hybridInstance.getRealizations();
-        storageEngines = new IStorageQuery[realizations.length];
-        for (int i = 0; i < realizations.length; i++) {
-            storageEngines[i] = StorageQueryFactory.createQuery(realizations[i]);
-        }
-    }
-
-    @Override
-    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
-        List<ITupleIterator> tupleIterators = Lists.newArrayList();
-        for (int i = 0; i < realizations.length; i++) {
-            if (realizations[i].isReady() && realizations[i].isCapable(sqlDigest)) {
-                ITupleIterator dataIterator = storageEngines[i].search(context, sqlDigest, returnTupleInfo);
-                tupleIterators.add(dataIterator);
-            }
-        }
-        // combine tuple iterator
-        return new CompoundTupleIterator(tupleIterators);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
new file mode 100644
index 0000000..49f9fd9
--- /dev/null
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridStorageQuery.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kylin.storage.hybrid;
+
+import java.util.List;
+
+import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.CompoundTupleIterator;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.StorageFactory;
+import org.apache.kylin.storage.tuple.TupleInfo;
+
+import com.google.common.collect.Lists;
+
+/**
+ */
+public class HybridStorageQuery implements IStorageQuery {
+
+    private IRealization[] realizations;
+    private IStorageQuery[] storageEngines;
+
+    public HybridStorageQuery(HybridInstance hybridInstance) {
+        this.realizations = hybridInstance.getRealizations();
+        storageEngines = new IStorageQuery[realizations.length];
+        for (int i = 0; i < realizations.length; i++) {
+            storageEngines[i] = StorageFactory.createQuery(realizations[i]);
+        }
+    }
+
+    @Override
+    public ITupleIterator search(final StorageContext context, final SQLDigest sqlDigest, final TupleInfo returnTupleInfo) {
+        List<ITupleIterator> tupleIterators = Lists.newArrayList();
+        for (int i = 0; i < realizations.length; i++) {
+            if (realizations[i].isReady() && realizations[i].isCapable(sqlDigest)) {
+                ITupleIterator dataIterator = storageEngines[i].search(context, sqlDigest, returnTupleInfo);
+                tupleIterators.add(dataIterator);
+            }
+        }
+        // combine tuple iterator
+        return new CompoundTupleIterator(tupleIterators);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/test/java/org/apache/kylin/storage/cache/DynamicCacheTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/cache/DynamicCacheTest.java b/core-storage/src/test/java/org/apache/kylin/storage/cache/DynamicCacheTest.java
index d2c8e11..309d67f 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/cache/DynamicCacheTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/cache/DynamicCacheTest.java
@@ -86,7 +86,7 @@ public class DynamicCacheTest {
         final AtomicInteger underlyingSEHitCount = new AtomicInteger(0);
         final List<Integer> returnedRowPerSearch = Lists.newArrayList();
 
-        CacheFledgedDynamicStorageEngine dynamicCache = new CacheFledgedDynamicStorageEngine(new ICachableStorageQuery() {
+        CacheFledgedDynamicQuery dynamicCache = new CacheFledgedDynamicQuery(new ICachableStorageQuery() {
             @Override
             public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
                 Range<Long> tsRagneInQuery = TsConditionExtractor.extractTsCondition(partitionCol, sqlDigest.filter);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/core-storage/src/test/java/org/apache/kylin/storage/cache/StaticCacheTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/cache/StaticCacheTest.java b/core-storage/src/test/java/org/apache/kylin/storage/cache/StaticCacheTest.java
index d81331e..e54e3e0 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/cache/StaticCacheTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/cache/StaticCacheTest.java
@@ -44,7 +44,7 @@ public class StaticCacheTest {
 
         final AtomicInteger underlyingSEHitCount = new AtomicInteger(0);
 
-        CacheFledgedStaticStorageEngine cacheFledgedStaticStorageEngine = new CacheFledgedStaticStorageEngine(new ICachableStorageQuery() {
+        CacheFledgedStaticQuery cacheFledgedStaticStorageEngine = new CacheFledgedStaticQuery(new ICachableStorageQuery() {
             @Override
             public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
                 underlyingSEHitCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
index 6b5cfa4..d00f592 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -25,9 +25,13 @@ import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.engine.mr.steps.BaseCuboidJob;
 import org.apache.kylin.engine.mr.steps.NDCuboidJob;
 import org.apache.kylin.job.constant.ExecutableConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class BatchCubingJobBuilder extends JobBuilderSupport {
-
+    
+    private static final Logger logger = LoggerFactory.getLogger(BatchCubingJobBuilder.class);
+    
     private final IMRBatchCubingInputSide inputSide;
     private final IMRBatchCubingOutputSide outputSide;
 
@@ -38,6 +42,8 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
     }
 
     public CubingJob build() {
+        logger.info("MR_V1 new job to BUILD segment " + seg);
+        
         final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
         final String jobId = result.getId();
         final String cuboidRootPath = getCuboidRootPath(jobId);
@@ -81,7 +87,7 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
 
         appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
         appendExecCmdParameters(cmd, "segmentname", seg.getName());
-        appendExecCmdParameters(cmd, "input", ""); // marks flat table input
+        appendExecCmdParameters(cmd, "input", "FLAT_TABLE"); // marks flat table input
         appendExecCmdParameters(cmd, "output", cuboidOutputTempPath[0]);
         appendExecCmdParameters(cmd, "jobname", "Kylin_Base_Cuboid_Builder_" + seg.getCubeInstance().getName());
         appendExecCmdParameters(cmd, "level", "0");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index abdabd8..fe9f1d6 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -25,8 +25,11 @@ import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.engine.mr.steps.InMemCuboidJob;
 import org.apache.kylin.engine.mr.steps.SaveStatisticsStep;
 import org.apache.kylin.job.constant.ExecutableConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class BatchCubingJobBuilder2 extends JobBuilderSupport {
+    private static final Logger logger = LoggerFactory.getLogger(BatchCubingJobBuilder2.class);
 
     private final IMRBatchCubingInputSide inputSide;
     private final IMRBatchCubingOutputSide2 outputSide;
@@ -38,6 +41,8 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
     }
 
     public CubingJob build() {
+        logger.info("MR_V2 new job to BUILD segment " + seg);
+        
         final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
         final String jobId = result.getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
index 3f6201c..bc377ed 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder.java
@@ -26,11 +26,14 @@ import org.apache.kylin.engine.mr.IMROutput.IMRBatchMergeOutputSide;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.engine.mr.steps.MergeCuboidJob;
 import org.apache.kylin.job.constant.ExecutableConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 public class BatchMergeJobBuilder extends JobBuilderSupport {
+    private static final Logger logger = LoggerFactory.getLogger(BatchMergeJobBuilder.class);
 
     private final IMRBatchMergeOutputSide outputSide;
 
@@ -40,6 +43,8 @@ public class BatchMergeJobBuilder extends JobBuilderSupport {
     }
 
     public CubingJob build() {
+        logger.info("MR_V1 new job to MERGE segment " + seg);
+        
         final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
         final String jobId = result.getId();
         final String cuboidRootPath = getCuboidRootPath(jobId);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
index e561ac3..443cf95 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
@@ -26,11 +26,14 @@ import org.apache.kylin.engine.mr.common.MapReduceExecutable;
 import org.apache.kylin.engine.mr.steps.MergeCuboidFromStorageJob;
 import org.apache.kylin.engine.mr.steps.MergeStatisticsStep;
 import org.apache.kylin.job.constant.ExecutableConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 public class BatchMergeJobBuilder2 extends JobBuilderSupport {
+    private static final Logger logger = LoggerFactory.getLogger(BatchMergeJobBuilder2.class);
 
     private final IMROutput2.IMRBatchMergeOutputSide2 outputSide;
 
@@ -40,6 +43,8 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
     }
 
     public CubingJob build() {
+        logger.info("MR_V2 new job to MERGE segment " + seg);
+        
         final CubingJob result = CubingJob.createMergeJob(seg, submitter, config);
         final String jobId = result.getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 5ecfcfd..5f977bc 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -32,8 +32,6 @@ import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterBuildStep;
 import org.apache.kylin.engine.mr.steps.UpdateCubeInfoAfterMergeStep;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
@@ -45,7 +43,6 @@ public class JobBuilderSupport {
     final protected JobEngineConfig config;
     final protected CubeSegment seg;
     final protected String submitter;
-    final protected Logger logger = LoggerFactory.getLogger(this.getClass());
 
     public JobBuilderSupport(CubeSegment seg, String submitter) {
         Preconditions.checkNotNull(seg, "segment cannot be null");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
index 4c8b652..9a1c1f5 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -11,13 +11,13 @@ import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeInputSide2;
 import org.apache.kylin.engine.mr.IMROutput2.IMRBatchMergeOutputSide2;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 import org.apache.kylin.storage.StorageFactory;
 
 public class MRUtil {
 
     public static IMRBatchCubingInputSide getBatchCubingInputSide(CubeSegment seg) {
-        return TableSourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchCubingInputSide(seg);
+        return SourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchCubingInputSide(seg);
     }
 
     public static IMRTableInputFormat getTableInputFormat(String tableName) {
@@ -25,7 +25,7 @@ public class MRUtil {
     }
 
     public static IMRTableInputFormat getTableInputFormat(TableDesc tableDesc) {
-        return TableSourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc);
+        return SourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc);
     }
 
     private static TableDesc getTableDesc(String tableName) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
index a15a086..3edaefb 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
@@ -21,7 +21,6 @@ package org.apache.kylin.engine.mr.steps;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -128,7 +127,7 @@ public class CuboidJob extends AbstractHadoopJob {
     private void configureMapperInputFormat(CubeSegment cubeSeg) throws IOException {
         String input = getOptionValue(OPTION_INPUT_PATH);
 
-        if (StringUtils.isBlank(input)) {
+        if ("FLAT_TABLE".equals(input)) {
             // base cuboid case
             IMRTableInputFormat flatTableInputFormat = MRUtil.getBatchCubingInputSide(cubeSeg).getFlatTableInputFormat();
             flatTableInputFormat.configureJob(job);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
index f68288a..2e65ae6 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingJobBuilder.java
@@ -25,10 +25,13 @@ import org.apache.kylin.engine.mr.IMROutput2;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  */
 public class SparkCubingJobBuilder extends JobBuilderSupport {
+    private static final Logger logger = LoggerFactory.getLogger(SparkCubingJobBuilder.class);
 
     private final IMRInput.IMRBatchCubingInputSide inputSide;
     private final IMROutput2.IMRBatchCubingOutputSide2 outputSide;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
index cb8eb50..5ce11f8 100644
--- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/IIInstance.java
@@ -27,6 +27,7 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.LookupDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
@@ -45,6 +46,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 /**
  * @author honma
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class IIInstance extends RootPersistentEntity implements IRealization {
 
@@ -366,4 +368,9 @@ public class IIInstance extends RootPersistentEntity implements IRealization {
         this.cost = cost;
     }
 
+    @Override
+    public int getStorageType() {
+        return IStorageAware.ID_HBASE;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
index cf4708e..0cc6f71 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/NewBaseCuboidMapper.java
@@ -57,7 +57,7 @@ import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.ReadableTable;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -131,7 +131,7 @@ public class NewBaseCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Text, Text, T
         cube = CubeManager.getInstance(config).getCube(cubeName);
         cubeSegment = cube.getSegment(segmentName, SegmentStatusEnum.NEW);
         cubeDesc = cube.getDescriptor();
-        factTableDesc = metadataManager.getTableDesc(cubeDesc.getFactTable());
+        factTableDesc = cubeDesc.getFactTableDesc();
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
         baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
@@ -171,7 +171,7 @@ public class NewBaseCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Text, Text, T
                 // load lookup tables
                 if (!lookupTables.containsKey(lookupTableName)) {
                     TableDesc tableDesc = metadataManager.getTableDesc(lookupTableName);
-                    ReadableTable htable = TableSourceFactory.createReadableTable(tableDesc);
+                    ReadableTable htable = SourceFactory.createReadableTable(tableDesc);
                     LookupBytesTable btable = new LookupBytesTable(tableDesc, join.getPrimaryKey(), htable);
                     lookupTables.put(lookupTableName, btable);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
index 2156574..d7eb3cf 100644
--- a/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
+++ b/job/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java
@@ -40,7 +40,7 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
-import org.apache.kylin.engine.BuildEngineFactory;
+import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.execution.AbstractExecutable;
@@ -274,7 +274,7 @@ public class BuildCubeWithEngineTest {
 
     private String buildSegment(String cubeName, long startDate, long endDate) throws Exception {
         CubeSegment segment = cubeManager.appendSegments(cubeManager.getCube(cubeName), endDate);
-        DefaultChainedExecutable job = BuildEngineFactory.createBatchCubingJob(segment, "TEST");
+        DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
         return job.getId();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/job/src/test/java/org/apache/kylin/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/DeployUtil.java b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
index 5e95983..351e176 100644
--- a/job/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/job/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -26,14 +26,12 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.ResourceTool;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
-import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeUpdate;
@@ -150,10 +148,9 @@ public class DeployUtil {
     }
 
     public static void prepareTestDataForStreamingCube(long startTime, long endTime, StreamingConfig streamingConfig) throws IOException {
-        MetadataManager metadataManager = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
         CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(streamingConfig.getCubeName());
         List<String> data = StreamingTableDataGenerator.generate(10000, startTime, endTime, cubeInstance.getFactTable());
-        TableDesc tableDesc = metadataManager.getTableDesc(cubeInstance.getFactTable());
+        TableDesc tableDesc = cubeInstance.getFactTableDesc();
 
         //load into kafka
         KafkaDataLoader.loadIntoKafka(streamingConfig, data);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java b/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
index 3d4fbba..0df632a 100644
--- a/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
+++ b/job/src/test/java/org/apache/kylin/source/hive/ITSnapshotManagerTest.java
@@ -27,7 +27,7 @@ import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.source.ReadableTable;
 import org.apache.kylin.source.ReadableTable.TableReader;
-import org.apache.kylin.source.TableSourceFactory;
+import org.apache.kylin.source.SourceFactory;
 import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.Before;
@@ -56,7 +56,7 @@ public class ITSnapshotManagerTest extends HBaseMetadataTestCase {
     public void basicTest() throws Exception {
         String tableName = "EDW.TEST_SITES";
         TableDesc tableDesc = MetadataManager.getInstance(getTestConfig()).getTableDesc(tableName);
-        ReadableTable hiveTable = TableSourceFactory.createReadableTable(tableDesc);
+        ReadableTable hiveTable = SourceFactory.createReadableTable(tableDesc);
         String snapshotPath = snapshotMgr.buildSnapshot(hiveTable, tableDesc).getResourcePath();
 
         snapshotMgr.wipeoutCache();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
index c6c1ce1..38883f0 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
@@ -33,7 +33,7 @@ import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.storage.IStorageQuery;
-import org.apache.kylin.storage.StorageQueryFactory;
+import org.apache.kylin.storage.StorageFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -111,7 +111,7 @@ public class OLAPEnumerator implements Enumerator<Object[]> {
         olapContext.resetSQLDigest();
 
         // query storage engine
-        IStorageQuery storageEngine = StorageQueryFactory.createQuery(olapContext.realization);
+        IStorageQuery storageEngine = StorageFactory.createQuery(olapContext.realization);
         ITupleIterator iterator = storageEngine.search(olapContext.storageContext, olapContext.getSQLDigest(), olapContext.returnTupleInfo);
         if (logger.isDebugEnabled()) {
             logger.debug("return TupleIterator...");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/query/src/main/java/org/apache/kylin/query/routing/RoutingRules/AdjustForWeaklyMatchedRealization.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/routing/RoutingRules/AdjustForWeaklyMatchedRealization.java b/query/src/main/java/org/apache/kylin/query/routing/RoutingRules/AdjustForWeaklyMatchedRealization.java
index 1c1dce5..250f2a6 100644
--- a/query/src/main/java/org/apache/kylin/query/routing/RoutingRules/AdjustForWeaklyMatchedRealization.java
+++ b/query/src/main/java/org/apache/kylin/query/routing/RoutingRules/AdjustForWeaklyMatchedRealization.java
@@ -66,7 +66,7 @@ public class AdjustForWeaklyMatchedRealization extends RoutingRule {
     private static void adjustOLAPContextIfNecessary(IIInstance ii, OLAPContext olapContext) {
         IIDesc iiDesc = ii.getDescriptor();
         Collection<FunctionDesc> iiFuncs = iiDesc.listAllFunctions();
-        convertAggregationToDimension(olapContext, iiFuncs, iiDesc.getFactTableName());
+        convertAggregationToDimension(olapContext, iiFuncs);
     }
 
     private static void adjustOLAPContextIfNecessary(CubeInstance cube, OLAPContext olapContext) {
@@ -75,10 +75,10 @@ public class AdjustForWeaklyMatchedRealization extends RoutingRule {
 
         CubeDesc cubeDesc = cube.getDescriptor();
         Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
-        convertAggregationToDimension(olapContext, cubeFuncs, cubeDesc.getFactTable());
+        convertAggregationToDimension(olapContext, cubeFuncs);
     }
 
-    private static void convertAggregationToDimension(OLAPContext olapContext, Collection<FunctionDesc> availableAggregations, String factTableName) {
+    private static void convertAggregationToDimension(OLAPContext olapContext, Collection<FunctionDesc> availableAggregations) {
         Iterator<FunctionDesc> it = olapContext.aggregations.iterator();
         while (it.hasNext()) {
             FunctionDesc functionDesc = it.next();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/server/src/main/java/org/apache/kylin/rest/controller/QueryController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/QueryController.java b/server/src/main/java/org/apache/kylin/rest/controller/QueryController.java
index 738f904..63e973a 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/QueryController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/QueryController.java
@@ -45,7 +45,7 @@ import org.apache.kylin.rest.request.SaveSqlRequest;
 import org.apache.kylin.rest.response.SQLResponse;
 import org.apache.kylin.rest.service.QueryService;
 import org.apache.kylin.rest.util.QueryUtil;
-import org.apache.kylin.storage.cache.AbstractCacheFledgedStorageEngine;
+import org.apache.kylin.storage.cache.AbstractCacheFledgedQuery;
 import org.apache.kylin.storage.exception.ScanOutOfLimitException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -85,7 +85,7 @@ public class QueryController extends BasicController {
     @PostConstruct
     public void init() throws IOException {
         Preconditions.checkNotNull(cacheManager, "cacheManager is not injected yet");
-        AbstractCacheFledgedStorageEngine.setCacheManager(cacheManager);
+        AbstractCacheFledgedQuery.setCacheManager(cacheManager);
     }
 
     @RequestMapping(value = "/query", method = RequestMethod.POST)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index bf339cc..a5cc08f 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -37,7 +37,7 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.cuboid.CuboidCLI;
 import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.engine.BuildEngineFactory;
+import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
@@ -635,7 +635,7 @@ public class CubeService extends BasicService {
                     if (newSeg != null) {
                         newSeg = getCubeManager().mergeSegments(cube, newSeg.getDateRangeStart(), newSeg.getDateRangeEnd(), true);
                         logger.debug("Will submit merge job on " + newSeg);
-                        DefaultChainedExecutable job = BuildEngineFactory.createBatchMergeJob(newSeg, "SYSTEM");
+                        DefaultChainedExecutable job = EngineFactory.createBatchMergeJob(newSeg, "SYSTEM");
                         getExecutableManager().addJob(job);
                     } else {
                         logger.debug("Not ready for merge on cube " + cubeName);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/server/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/JobService.java b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
index 90992e9..21d7d58 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -30,7 +30,7 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.CubeUpdate;
 import org.apache.kylin.cube.model.CubeBuildTypeEnum;
-import org.apache.kylin.engine.BuildEngineFactory;
+import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.CubingJob;
 import org.apache.kylin.engine.mr.common.HadoopShellExecutable;
 import org.apache.kylin.engine.mr.common.MapReduceExecutable;
@@ -137,10 +137,10 @@ public class JobService extends BasicService {
 
         if (buildType == CubeBuildTypeEnum.BUILD) {
             CubeSegment newSeg = getCubeManager().appendSegments(cube, endDate);
-            job = BuildEngineFactory.createBatchCubingJob(newSeg, submitter);
+            job = EngineFactory.createBatchCubingJob(newSeg, submitter);
         } else if (buildType == CubeBuildTypeEnum.MERGE) {
             CubeSegment newSeg = getCubeManager().mergeSegments(cube, startDate, endDate, forceMergeEmptySeg);
-            job = BuildEngineFactory.createBatchMergeJob(newSeg, submitter);
+            job = EngineFactory.createBatchMergeJob(newSeg, submitter);
         } else if (buildType == CubeBuildTypeEnum.REFRESH) {
             List<CubeSegment> readySegs = cube.getSegment(SegmentStatusEnum.READY);
             boolean segExists = false;
@@ -156,7 +156,7 @@ public class JobService extends BasicService {
             }
 
             CubeSegment refreshSeg = getCubeManager().refreshSegment(cube, startDate, endDate);
-            job = BuildEngineFactory.createBatchCubingJob(refreshSeg, submitter);
+            job = EngineFactory.createBatchCubingJob(refreshSeg, submitter);
         } else {
             throw new JobException("invalid build type:" + buildType);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
index f9f87c6..2dbe071 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveTableSource.java
@@ -20,12 +20,12 @@ package org.apache.kylin.source.hive;
 
 import org.apache.kylin.engine.mr.IMRInput;
 import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.source.ITableSource;
+import org.apache.kylin.source.ISource;
 import org.apache.kylin.source.ReadableTable;
 
 @SuppressWarnings("unused")
 //used by reflection
-public class HiveTableSource implements ITableSource {
+public class HiveTableSource implements ISource {
 
     @SuppressWarnings("unchecked")
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
index 4ad072a..1d8c8c8 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -18,21 +18,85 @@
 
 package org.apache.kylin.storage.hbase;
 
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.engine.mr.IMROutput;
 import org.apache.kylin.engine.mr.IMROutput2;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.PartitionDesc;
+import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealization;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.storage.ICachableStorageQuery;
 import org.apache.kylin.storage.IStorage;
 import org.apache.kylin.storage.IStorageQuery;
+import org.apache.kylin.storage.cache.CacheFledgedDynamicQuery;
+import org.apache.kylin.storage.cache.CacheFledgedStaticQuery;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput2;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.apache.kylin.storage.hybrid.HybridStorageQuery;
+
+import com.google.common.base.Preconditions;
 
 @SuppressWarnings("unused")//used by reflection
 public class HBaseStorage implements IStorage {
 
+    private final static boolean allowStorageLayerCache = true;
+    private final static String defaultCubeStorageQuery = "org.apache.kylin.storage.hbase.cube.v1.CubeStorageQuery";
+    private final static String defaultIIStorageQuery = "org.apache.kylin.storage.hbase.ii.InvertedIndexStorageQuery";
+
     @Override
-    public IStorageQuery createStorageQuery(IRealization realization) {
-        // TODO Auto-generated method stub
-        return null;
+    public IStorageQuery createQuery(IRealization realization) {
+        if (realization.getType() == RealizationType.INVERTED_INDEX) {
+            ICachableStorageQuery ret;
+            try {
+                ret = (ICachableStorageQuery) Class.forName(defaultIIStorageQuery).getConstructor(IIInstance.class).newInstance((IIInstance) realization);
+            } catch (Exception e) {
+                throw new RuntimeException("Failed to initialize storage query for " + defaultIIStorageQuery, e);
+            }
+
+            if (allowStorageLayerCache) {
+                return wrapWithCache(ret, realization);
+            } else {
+                return ret;
+            }
+        } else if (realization.getType() == RealizationType.CUBE) {
+            ICachableStorageQuery ret;
+            try {
+                ret = (ICachableStorageQuery) Class.forName(defaultCubeStorageQuery).getConstructor(CubeInstance.class).newInstance((CubeInstance) realization);
+            } catch (Exception e) {
+                throw new RuntimeException("Failed to initialize storage query for " + defaultCubeStorageQuery, e);
+            }
+
+            if (allowStorageLayerCache) {
+                return wrapWithCache(ret, realization);
+            } else {
+                return ret;
+            }
+        } else {
+            throw new IllegalArgumentException("Unknown realization type " + realization.getType());
+        }
+    }
+    
+    private static IStorageQuery wrapWithCache(ICachableStorageQuery underlyingStorageEngine, IRealization realization) {
+        if (underlyingStorageEngine.isDynamic()) {
+            return new CacheFledgedDynamicQuery(underlyingStorageEngine, getPartitionCol(realization));
+        } else {
+            return new CacheFledgedStaticQuery(underlyingStorageEngine);
+        }
+    }
+
+    private static TblColRef getPartitionCol(IRealization realization) {
+        String modelName = realization.getModelName();
+        DataModelDesc dataModelDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getDataModelDesc(modelName);
+        PartitionDesc partitionDesc = dataModelDesc.getPartitionDesc();
+        Preconditions.checkArgument(partitionDesc != null, "PartitionDesc for " + realization + " is null!");
+        TblColRef partitionColRef = partitionDesc.getPartitionDateColumnRef();
+        Preconditions.checkArgument(partitionColRef != null, "getPartitionDateColumnRef for " + realization + " is null");
+        return partitionColRef;
     }
 
     @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
new file mode 100644
index 0000000..93d6a86
--- /dev/null
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.storage.hbase.steps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+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.client.HBaseAdmin;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.job.cmd.ShellCmdOutput;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * A deprecated job step, used by 0.7 jobs only. Kept here solely for smooth upgrade to 0.8.
+ * 
+ * Drop the resources that is no longer needed, including intermediate hive table 
+ * (after cube build) and hbase tables (after cube merge)
+ */
+public class DeprecatedGCStep extends AbstractExecutable {
+
+    private static final String OLD_HTABLES = "oldHTables";
+
+    private static final String OLD_HIVE_TABLE = "oldHiveTable";
+
+    private static final String OLD_HDFS_PATHS = "oldHdfsPaths";
+
+    private static final Logger logger = LoggerFactory.getLogger(DeprecatedGCStep.class);
+
+    private StringBuffer output;
+
+    public DeprecatedGCStep() {
+        super();
+        output = new StringBuffer();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+
+        try {
+            dropHBaseTable(context);
+            dropHdfsPath(context);
+            dropHiveTable(context);
+        } catch (IOException e) {
+            logger.error("job:" + getId() + " execute finished with exception", e);
+            output.append("\n").append(e.getLocalizedMessage());
+            return new ExecuteResult(ExecuteResult.State.ERROR, output.toString());
+        }
+
+        return new ExecuteResult(ExecuteResult.State.SUCCEED, output.toString());
+    }
+
+    private void dropHiveTable(ExecutableContext context) throws IOException {
+        final String hiveTable = this.getOldHiveTable();
+        if (StringUtils.isNotEmpty(hiveTable)) {
+            final String dropSQL = "DROP TABLE IF EXISTS  " + hiveTable + ";";
+            final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
+            ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
+            context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
+            output.append("Dropped Hive table " + hiveTable + " \n");
+        }
+
+    }
+
+    private void dropHBaseTable(ExecutableContext context) throws IOException {
+        List<String> oldTables = getOldHTables();
+        if (oldTables != null && oldTables.size() > 0) {
+            String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
+            Configuration conf = HBaseConfiguration.create();
+            HBaseAdmin admin = null;
+            try {
+                admin = new HBaseAdmin(conf);
+                for (String table : oldTables) {
+                    if (admin.tableExists(table)) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                        String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
+                        if (metadataUrlPrefix.equalsIgnoreCase(host)) {
+                            if (admin.isTableEnabled(table)) {
+                                admin.disableTable(table);
+                            }
+                            admin.deleteTable(table);
+                            logger.debug("Dropped HBase table " + table);
+                            output.append("Dropped HBase table " + table + " \n");
+                        } else {
+                            logger.debug("Skipped HBase table " + table);
+                            output.append("Skipped HBase table " + table + " \n");
+                        }
+                    }
+                }
+
+            } finally {
+                if (admin != null)
+                    try {
+                        admin.close();
+                    } catch (IOException e) {
+                        logger.error(e.getLocalizedMessage());
+                    }
+            }
+        }
+    }
+
+    private void dropHdfsPath(ExecutableContext context) throws IOException {
+
+        List<String> oldHdfsPaths = this.getOldHdsfPaths();
+        if (oldHdfsPaths != null && oldHdfsPaths.size() > 0) {
+            Configuration hconf = HadoopUtil.getCurrentConfiguration();
+            FileSystem fileSystem = FileSystem.get(hconf);
+            for (String path : oldHdfsPaths) {
+                if (path.endsWith("*"))
+                    path = path.substring(0, path.length() - 1);
+
+                Path oldPath = new Path(path);
+                if (fileSystem.exists(oldPath)) {
+                    fileSystem.delete(oldPath, true);
+                    logger.debug("Dropped HDFS path: " + path);
+                    output.append("Dropped HDFS path  \"" + path + "\" \n");
+                } else {
+                    logger.debug("HDFS path not exists: " + path);
+                    output.append("HDFS path not exists: \"" + path + "\" \n");
+                }
+            }
+
+        }
+    }
+
+    public void setOldHTables(List<String> tables) {
+        setArrayParam(OLD_HTABLES, tables);
+    }
+
+    private List<String> getOldHTables() {
+        return getArrayParam(OLD_HTABLES);
+    }
+
+    public void setOldHdsfPaths(List<String> paths) {
+        setArrayParam(OLD_HDFS_PATHS, paths);
+    }
+
+    private List<String> getOldHdsfPaths() {
+        return getArrayParam(OLD_HDFS_PATHS);
+    }
+
+    private void setArrayParam(String paramKey, List<String> paramValues) {
+        setParam(paramKey, StringUtils.join(paramValues, ","));
+    }
+
+    private List<String> getArrayParam(String paramKey) {
+        final String ids = getParam(paramKey);
+        if (ids != null) {
+            final String[] splitted = StringUtils.split(ids, ",");
+            ArrayList<String> result = Lists.newArrayListWithExpectedSize(splitted.length);
+            for (String id : splitted) {
+                result.add(id);
+            }
+            return result;
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+    public void setOldHiveTable(String hiveTable) {
+        setParam(OLD_HIVE_TABLE, hiveTable);
+    }
+
+    private String getOldHiveTable() {
+        return getParam(OLD_HIVE_TABLE);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/bf83339c/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/ITStorageTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/ITStorageTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/ITStorageTest.java
index 1cc6f89..0b4fd07 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/ITStorageTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/common/ITStorageTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.storage.hbase.common;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 import java.util.Collections;
 import java.util.List;
@@ -34,10 +34,10 @@ import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.StorageQueryFactory;
+import org.apache.kylin.storage.StorageFactory;
 import org.apache.kylin.storage.cache.StorageMockUtils;
-import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
 import org.apache.kylin.storage.exception.ScanOutOfLimitException;
+import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -67,7 +67,7 @@ public class ITStorageTest extends HBaseMetadataTestCase {
         CubeManager cubeMgr = CubeManager.getInstance(getTestConfig());
         cube = cubeMgr.getCube("TEST_KYLIN_CUBE_WITHOUT_SLR_EMPTY");
         Assert.assertNotNull(cube);
-        storageEngine = StorageQueryFactory.createQuery(cube);
+        storageEngine = StorageFactory.createQuery(cube);
         String url = KylinConfig.getInstanceFromEnv().getStorageUrl();
         context = new StorageContext();
         context.setConnUrl(url);


[10/28] incubator-kylin git commit: KYLIN-982 package.sh should grep out "Download*" messages when determining version

Posted by ma...@apache.org.
KYLIN-982 package.sh should grep out "Download*" messages when determining version

Signed-off-by: honma <ho...@ebay.com>


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

Branch: refs/heads/2.x-staging
Commit: d501dc4e3e90083aad2a74a82e3762bbb45afb01
Parents: 730fab0
Author: gaodayue <ga...@meituan.com>
Authored: Tue Sep 1 23:36:10 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Wed Sep 2 10:51:11 2015 +0800

----------------------------------------------------------------------
 script/package.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d501dc4e/script/package.sh
----------------------------------------------------------------------
diff --git a/script/package.sh b/script/package.sh
index a4bda3f..b45de28 100755
--- a/script/package.sh
+++ b/script/package.sh
@@ -31,7 +31,7 @@ fi
 
 dir=$(dirname ${0})
 cd ${dir}/..
-version=`mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=project.version | grep -v '\['`
+version=`mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=project.version | grep -E '^[0-9]+\.[0-9]+\.[0-9]+'`
 echo "kylin version: ${version}"
 export version