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

[01/50] [abbrv] incubator-kylin git commit: KYLIN-956 refine document

Repository: incubator-kylin
Updated Branches:
  refs/heads/master 575eee0d0 -> 43f119741


KYLIN-956 refine document


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

Branch: refs/heads/master
Commit: 9b805e52f511cbcebac1c832820e577ec3b9fd09
Parents: 4e07377
Author: honma <ho...@ebay.com>
Authored: Tue Sep 1 21:21:38 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:37:59 2015 +0800

----------------------------------------------------------------------
 website/_docs/install/advance_settings.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/9b805e52/website/_docs/install/advance_settings.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/advance_settings.md b/website/_docs/install/advance_settings.md
index d876b8e..b52b3d6 100644
--- a/website/_docs/install/advance_settings.md
+++ b/website/_docs/install/advance_settings.md
@@ -9,7 +9,7 @@ since: v0.7.1
 
 ## Enabling LZO compression
 
-LZO compression can be leveraged to compress the output of MR jobs, as well as hbase table storage, reducing the storage overhead. By default we do not enable LZO compression in Kylin because hadoop sandbox venders tend to not include LZO in their distributions due to license(GPL) issues.
+By default Kylin leverages snappy compression to compress the output of MR jobs, as well as hbase table storage, reducing the storage overhead. We do not choose LZO compression in Kylin because hadoop venders tend to not include LZO in their distributions due to license(GPL) issues.
 
 To enable LZO in Kylin, follow these steps:
 


[35/50] [abbrv] incubator-kylin git commit: KYLIN-958 ignore more non-structral change in DataModel

Posted by li...@apache.org.
KYLIN-958 ignore more non-structral change in DataModel

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

Branch: refs/heads/master
Commit: 419798d7b3b14f168b5b49abad0b5d18d1cfbdfc
Parents: c4b4b11
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 9 13:40:23 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 9 13:40:23 2015 +0800

----------------------------------------------------------------------
 .../kylin/metadata/model/DataModelDesc.java     | 85 ++++++++++----------
 1 file changed, 44 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/419798d7/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
index cb1e784..78feec4 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
@@ -18,17 +18,17 @@
 
 package org.apache.kylin.metadata.model;
 
-import java.io.IOException;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
-import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 
@@ -36,12 +36,12 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class DataModelDesc extends RootPersistentEntity {
-    private static final Logger logger = LoggerFactory.getLogger(DataModelDesc.class);
+    private static final Logger logger = LoggerFactory.getLogger(DataModelDesc.class);
 
     public static enum RealizationCapacity {
         SMALL, MEDIUM, LARGE
@@ -219,41 +219,44 @@ public class DataModelDesc extends RootPersistentEntity {
         }
     }
 
-    /**
-     * Check whether two data model are compatible or not. Compatible means
-     * having the same structure. Tow models could be compatible even they
-     * have different UUID or last modified time.
-     * @param that model to compare with
-     * @return true if compatible, false otherwise.
-     */
-    public boolean compatibleWith(DataModelDesc that) {
-        if (this == that)
-            return true;
-
-        if (that == null)
-            return false;
-
-        try {
-            String thisRepr = excludeHeaderInfo(this);
-            String thatRepr = excludeHeaderInfo(that);
-            return StringUtils.equals(thisRepr, thatRepr);
-
-        } catch (IOException e) {
-            logger.error("Failed to serialize DataModelDesc to string", e);
-            return false;
-        }
-    }
-
-    private String excludeHeaderInfo(DataModelDesc modelDesc) throws IOException {
-        // make a copy
-        String repr = JsonUtil.writeValueAsString(modelDesc);
-        DataModelDesc copy = JsonUtil.readValue(repr, DataModelDesc.class);
-
-        copy.setUuid(null);
-        copy.setLastModified(0);
-        return JsonUtil.writeValueAsString(copy);
-    }
-
+    /**
+     * Check whether two data model are compatible or not. Compatible means
+     * having the same structure. Tow models could be compatible even they
+     * have different UUID or last modified time.
+     * @param that model to compare with
+     * @return true if compatible, false otherwise.
+     */
+    public boolean compatibleWith(DataModelDesc that) {
+        if (this == that)
+            return true;
+
+        if (that == null)
+            return false;
+
+        try {
+            String thisRepr = excludeHeaderInfo(this);
+            String thatRepr = excludeHeaderInfo(that);
+            return StringUtils.equals(thisRepr, thatRepr);
+
+        } catch (IOException e) {
+            logger.error("Failed to serialize DataModelDesc to string", e);
+            return false;
+        }
+    }
+
+    private String excludeHeaderInfo(DataModelDesc modelDesc) throws IOException {
+        // make a copy
+        String repr = JsonUtil.writeValueAsString(modelDesc);
+        DataModelDesc copy = JsonUtil.readValue(repr, DataModelDesc.class);
+
+        copy.setUuid(null);
+        copy.setLastModified(0);
+        copy.setCapacity(RealizationCapacity.MEDIUM);
+        copy.setFilterCondition(null);
+        copy.setPartitionDesc(null);
+        return JsonUtil.writeValueAsString(copy);
+    }
+
     @Override
     public String toString() {
         return "DataModelDesc [name=" + name + "]";


[05/50] [abbrv] incubator-kylin git commit: KYLIN-957 Support HBase in a separate cluster

Posted by li...@apache.org.
KYLIN-957 Support HBase in a separate cluster


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

Branch: refs/heads/master
Commit: 2748643a945fc248c06bae301f89b08d0c79fe2d
Parents: 9b805e5
Author: sunyerui <su...@gmail.com>
Authored: Wed Aug 26 18:39:03 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:37:59 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    | 12 +++
 .../common/persistence/HBaseResourceStore.java  |  4 +-
 .../apache/kylin/common/util/HadoopUtil.java    | 49 +++++++++++-
 .../persistence/HBaseResourceStoreTest.java     |  2 +-
 .../kylin/common/util/HadoopUtilTest.java       | 83 ++++++++++++++++++++
 conf/kylin.properties                           |  8 ++
 .../apache/kylin/job/AbstractJobBuilder.java    |  8 +-
 .../apache/kylin/job/cube/CubingJobBuilder.java | 13 ++-
 .../kylin/job/cube/GarbageCollectionStep.java   | 36 +++++----
 .../kylin/job/hadoop/AbstractHadoopJob.java     |  2 +-
 .../kylin/job/hadoop/hbase/BulkLoadJob.java     |  3 +-
 .../kylin/job/hadoop/hbase/CreateHTableJob.java |  3 +-
 .../kylin/job/tools/DeployCoprocessorCLI.java   |  2 +-
 13 files changed, 198 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index deb2eda..76031c2 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -94,6 +94,10 @@ public class KylinConfig {
 
     public static final String KYLIN_HDFS_WORKING_DIR = "kylin.hdfs.working.dir";
 
+    public static final String KYLIN_HADOOP_CLUSTER_FS = "kylin.hadoop.cluster.fs";
+
+    public static final String KYLIN_HBASE_CLUSTER_FS = "kylin.hbase.cluster.fs";
+
     public static final String HIVE_PASSWORD = "hive.password";
 
     public static final String HIVE_USER = "hive.user";
@@ -280,6 +284,14 @@ public class KylinConfig {
         return root + getMetadataUrlPrefix() + "/";
     }
 
+    public String getHadoopClusterFs() {
+        return getOptional(KYLIN_HADOOP_CLUSTER_FS, "");
+    }
+
+    public String getHBaseClusterFs() {
+        return getOptional(KYLIN_HBASE_CLUSTER_FS, "");
+    }
+
     public String getKylinJobLogDir() {
         return getOptional(KYLIN_JOB_LOG_DIR, "/tmp/kylin/logs");
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index e2a4b12..37b8f8d 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -179,7 +179,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] value = r.getValue(B_FAMILY, B_COLUMN);
         if (value.length == 0) {
             Path redirectPath = bigCellHDFSPath(resPath);
-            Configuration hconf = HadoopUtil.getCurrentConfiguration();
+            Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
             FileSystem fileSystem = FileSystem.get(hconf);
 
             return fileSystem.open(redirectPath);
@@ -297,7 +297,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
-        Configuration hconf = HadoopUtil.getCurrentConfiguration();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
 
         if (fileSystem.exists(redirectPath)) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
index fcefcf2..43b2f29 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
@@ -29,8 +29,10 @@ import java.util.regex.Pattern;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.kylin.common.KylinConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,17 +41,40 @@ public class HadoopUtil {
 
     private static ThreadLocal<Configuration> hadoopConfig = new ThreadLocal<>();
 
+    private static ThreadLocal<Configuration> hbaseConfig = new ThreadLocal<>();
+
     public static void setCurrentConfiguration(Configuration conf) {
         hadoopConfig.set(conf);
     }
 
+    public static void setCurrentHBaseConfiguration(Configuration conf) {
+        hbaseConfig.set(conf);
+    }
+
     public static Configuration getCurrentConfiguration() {
         if (hadoopConfig.get() == null) {
-            hadoopConfig.set(new Configuration());
+            Configuration configuration = new Configuration();
+            String hadoopClusterFs = KylinConfig.getInstanceFromEnv().getHadoopClusterFs();
+            if (hadoopClusterFs != null && !hadoopClusterFs.equals("")) {
+                configuration.set(FileSystem.FS_DEFAULT_NAME_KEY, hadoopClusterFs);
+            }
+            hadoopConfig.set(configuration);
         }
         return hadoopConfig.get();
     }
 
+    public static Configuration getCurrentHBaseConfiguration() {
+        if (hbaseConfig.get() == null) {
+            Configuration configuration = HBaseConfiguration.create(new Configuration());
+            String hbaseClusterFs = KylinConfig.getInstanceFromEnv().getHBaseClusterFs();
+            if (hbaseClusterFs != null && !hbaseClusterFs.equals("")) {
+                configuration.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
+            }
+            hbaseConfig.set(configuration);
+        }
+        return hbaseConfig.get();
+    }
+
     public static FileSystem getFileSystem(String path) throws IOException {
         return FileSystem.get(makeURI(path), getCurrentConfiguration());
     }
@@ -62,6 +87,24 @@ public class HadoopUtil {
         }
     }
 
+    public static String makeQualifiedPathInHadoopCluster(String path) {
+        try {
+            FileSystem fs = FileSystem.get(getCurrentConfiguration());
+            return fs.makeQualified(new Path(path)).toString();
+        } catch (IOException e) {
+            throw new IllegalArgumentException("Cannot create FileSystem from current hadoop cluster conf", e);
+        }
+    }
+
+    public static String makeQualifiedPathInHBaseCluster(String path) {
+        try {
+            FileSystem fs = FileSystem.get(getCurrentHBaseConfiguration());
+            return fs.makeQualified(new Path(path)).toString();
+        } catch (IOException e) {
+            throw new IllegalArgumentException("Cannot create FileSystem from current hbase cluster conf", e);
+        }
+    }
+
     /**
      * e.g.
      * 0. hbase (recommended way)
@@ -116,6 +159,10 @@ public class HadoopUtil {
         conf.set(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, "60000");
         // conf.set(ScannerCallable.LOG_SCANNER_ACTIVITY, "true");
 
+        String hbaseClusterFs = KylinConfig.getInstanceFromEnv().getHBaseClusterFs();
+        if (hbaseClusterFs != null && !hbaseClusterFs.equals("")) {
+            conf.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
+        }
         return conf;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java b/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
index c9e8063..75625fb 100644
--- a/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
+++ b/common/src/test/java/org/apache/kylin/common/persistence/HBaseResourceStoreTest.java
@@ -77,7 +77,7 @@ public class HBaseResourceStoreTest extends HBaseMetadataTestCase {
             assertEquals(content, t);
 
             Path redirectPath = ((HBaseResourceStore) store).bigCellHDFSPath(path);
-            Configuration hconf = HadoopUtil.getCurrentConfiguration();
+            Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
             FileSystem fileSystem = FileSystem.get(hconf);
             assertTrue(fileSystem.exists(redirectPath));
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java b/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
new file mode 100644
index 0000000..c380933
--- /dev/null
+++ b/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.kylin.common.KylinConfig;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+import static org.junit.Assert.*;
+
+/**
+ * Created by sunyerui on 15/8/26.
+ * Tests for HadoopUtil
+ */
+public class HadoopUtilTest {
+
+  @BeforeClass
+  public static void beforeClass() {
+    System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox");
+  }
+
+  @After
+  public void after() {
+    HadoopUtil.setCurrentConfiguration(null);
+    HadoopUtil.setCurrentHBaseConfiguration(null);
+  }
+
+  @Test
+  public void testGetCurrentConfiguration() throws Exception {
+    KylinConfig config = KylinConfig.getInstanceFromEnv();
+    config.setProperty(KylinConfig.KYLIN_HADOOP_CLUSTER_FS, "hdfs://hadoop-cluster/");
+
+    Configuration conf = HadoopUtil.getCurrentConfiguration();
+    assertEquals("hdfs://hadoop-cluster/", conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
+  }
+
+  @Test
+  public void testGetCurrentHBaseConfiguration() throws Exception {
+    KylinConfig config = KylinConfig.getInstanceFromEnv();
+    config.setProperty(KylinConfig.KYLIN_HBASE_CLUSTER_FS, "hdfs://hbase-cluster/");
+
+    Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
+    assertEquals("hdfs://hbase-cluster/", conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
+  }
+
+  @Test
+  public void testMakeQualifiedPathInHadoopCluster() throws Exception {
+    KylinConfig config = KylinConfig.getInstanceFromEnv();
+    config.setProperty(KylinConfig.KYLIN_HADOOP_CLUSTER_FS, "file:/");
+
+    String path = HadoopUtil.makeQualifiedPathInHadoopCluster("/path/to/test/hadoop");
+    assertEquals("file:/path/to/test/hadoop", path);
+  }
+
+  @Test
+  public void testMakeQualifiedPathInHBaseCluster() throws Exception {
+    KylinConfig config = KylinConfig.getInstanceFromEnv();
+    config.setProperty(KylinConfig.KYLIN_HBASE_CLUSTER_FS, "file:/");
+
+    String path = HadoopUtil.makeQualifiedPathInHBaseCluster("/path/to/test/hbase");
+    assertEquals("file:/path/to/test/hbase", path);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 8c7c647..af860bd 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -29,6 +29,14 @@ kylin.storage.url=hbase
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
+# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster/
+# leave empty if using default fs configured by local core-site.xml
+kylin.hadoop.cluster.fs=
+
+# HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster/
+# leave empty if hbase running on same cluster with hive and mapreduce
+kylin.hbase.cluster.fs=
+
 kylin.job.mapreduce.default.reduce.input.mb=500
 
 kylin.server.mode=all

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
index e6fde23..96b87c5 100644
--- a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
@@ -20,6 +20,9 @@ package org.apache.kylin.job;
 
 import java.io.IOException;
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -62,6 +65,7 @@ public abstract class AbstractJobBuilder {
     protected AbstractExecutable createIntermediateHiveTableStep(IJoinedFlatTableDesc intermediateTableDesc, String jobId) {
 
         final String useDatabaseHql = "USE " + engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + ";";
+        final String setClusterHql = "-hiveconf " + FileSystem.FS_DEFAULT_NAME_KEY + "=\"" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY) + "\"";
         final String dropTableHql = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc, jobId);
         final String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, getJobWorkingDir(jobId), jobId);
         String insertDataHqls;
@@ -74,7 +78,9 @@ public abstract class AbstractJobBuilder {
 
         ShellExecutable step = new ShellExecutable();
         StringBuffer buf = new StringBuffer();
-        buf.append("hive -e \"");
+        buf.append("hive ");
+        buf.append(setClusterHql);
+        buf.append(" -e \"");
         buf.append(useDatabaseHql + "\n");
         buf.append(dropTableHql + "\n");
         buf.append(createTableHql + "\n");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
index 747ae3c..dd71cd8 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
@@ -25,6 +25,9 @@ import java.util.List;
 import java.util.TimeZone;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.job.AbstractJobBuilder;
@@ -201,7 +204,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
     AbstractExecutable addHTableSteps(CubeSegment seg, String cuboidRootPath, CubingJob result) {
         final String jobId = result.getId();
-        final String cuboidPath = cuboidRootPath + "*";
+        final String cuboidPath = HadoopUtil.makeQualifiedPathInHadoopCluster(cuboidRootPath + "*");
 
         result.addTask(createRangeRowkeyDistributionStep(seg, cuboidPath));
         // create htable step
@@ -240,6 +243,8 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
             if (jobConf != null && jobConf.length() > 0) {
                 builder.append(" -conf ").append(jobConf);
             }
+            String setCluster = " -D" + FileSystem.FS_DEFAULT_NAME_KEY + "=" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY);
+            builder.append(setCluster);
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
@@ -263,15 +268,15 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
     }
 
     private String getRowkeyDistributionOutputPath(CubeSegment seg) {
-        return getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
+        return HadoopUtil.makeQualifiedPathInHBaseCluster(getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats");
     }
 
     private String getFactDistinctColumnsPath(CubeSegment seg, String jobUuid) {
-        return getJobWorkingDir(jobUuid) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns";
+        return HadoopUtil.makeQualifiedPathInHadoopCluster(getJobWorkingDir(jobUuid) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns");
     }
 
     private String getHFilePath(CubeSegment seg, String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/hfile/";
+        return HadoopUtil.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/hfile/");
     }
 
     private MapReduceExecutable createFactDistinctColumnsStep(CubeSegment seg, String intermediateHiveTableName, String jobId) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index 4cb4a80..b4f6e8e 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -83,10 +83,14 @@ public class GarbageCollectionStep extends AbstractExecutable {
     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 + "\"";
+            final String dropSQL = "USE " + KylinConfig.getInstanceFromEnv().getHiveDatabaseForIntermediateTable() + ";" + " DROP TABLE IF EXISTS  " + hiveTable + ";";
+            final String setClusterHql = "-hiveconf " + FileSystem.FS_DEFAULT_NAME_KEY + "=\"" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY) + "\"";
+            final String dropHiveCMD = "hive " + setClusterHql + " -e \"" + dropSQL + "\"";
+            logger.info("executing: " + dropHiveCMD);
             ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
             context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
+            logger.debug("Dropped Hive table " + hiveTable + " \n");
+            output.append(shellCmdOutput.getOutput() + " \n");
             output.append("Dropped Hive table " + hiveTable + " \n");
         }
 
@@ -129,27 +133,31 @@ public class GarbageCollectionStep extends AbstractExecutable {
         }
     }
 
-    private void dropHdfsPath(ExecutableContext context) throws IOException {
+    private void dropFileSystemPath(FileSystem fs, Path p) throws IOException {
+        Path path = fs.makeQualified(p);
+        if (fs.exists(path)) {
+            fs.delete(path, 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");
+        }
+    }
 
+    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);
+            FileSystem hadoopFs = FileSystem.get(HadoopUtil.getCurrentConfiguration());
+            FileSystem hbaseFs = FileSystem.get(HadoopUtil.getCurrentHBaseConfiguration());
             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");
-                }
+                dropFileSystemPath(hadoopFs, oldPath);
+                dropFileSystemPath(hbaseFs, oldPath);
             }
-
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
index 6ad89d6..a995649 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
@@ -295,7 +295,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     }
 
     protected void deletePath(Configuration conf, Path path) throws IOException {
-        FileSystem fs = FileSystem.get(conf);
+        FileSystem fs = FileSystem.get(path.toUri(), conf);
         if (fs.exists(path)) {
             fs.delete(path, true);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
index 2608085..692d53e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -59,7 +60,7 @@ public class BulkLoadJob extends AbstractHadoopJob {
             // end with "/"
             String input = getOptionValue(OPTION_INPUT_PATH);
 
-            Configuration conf = HBaseConfiguration.create(getConf());
+            Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
             FileSystem fs = FileSystem.get(conf);
 
             String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
index f114b5b..027c0ca 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -79,7 +80,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
         tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
         tableDesc.setValue(IRealizationConstants.HTableTag, config.getMetadataUrlPrefix());
 
-        Configuration conf = HBaseConfiguration.create(getConf());
+        Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
         HBaseAdmin admin = new HBaseAdmin(conf);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/2748643a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
index a28477e..89472b2 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
@@ -106,7 +106,7 @@ public class DeployCoprocessorCLI {
 
     private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.getCurrentConfiguration();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
 
         String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();


[39/50] [abbrv] incubator-kylin git commit: minor, enhance BadQueryDetector to report slow query at realtime

Posted by li...@apache.org.
minor, enhance BadQueryDetector to report slow query at realtime


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

Branch: refs/heads/master
Commit: 876079b516d4fcd193815a742612186409736021
Parents: 44309fe
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Sep 11 15:33:14 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Sep 11 15:33:14 2015 +0800

----------------------------------------------------------------------
 .../kylin/rest/service/BadQueryDetector.java    | 29 +++++++++++---------
 1 file changed, 16 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/876079b5/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java b/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
index 32410f4..5f5b247 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/BadQueryDetector.java
@@ -81,14 +81,7 @@ public class BadQueryDetector extends Thread {
     }
 
     public void queryEnd(Thread thread) {
-        Entry e = runningQueries.remove(thread);
-
-        if (e != null) {
-            int runningSec = (int) ((System.currentTimeMillis() - e.startTime) / 1000);
-            if (runningSec >= alertRunningSec) {
-                notify("Slow", runningSec, e.sqlRequest.getSql());
-            }
-        }
+        runningQueries.remove(thread);
     }
 
     private class Entry implements Comparable<Entry> {
@@ -124,18 +117,28 @@ public class BadQueryDetector extends Thread {
     }
 
     private void detectBadQuery() {
+        long now = System.currentTimeMillis();
+        ArrayList<Entry> entries = new ArrayList<Entry>(runningQueries.values());
+        Collections.sort(entries);
+        
+        // report if low memory
         if (getSystemAvailMB() < alertMB) {
-            ArrayList<Entry> entries = new ArrayList<Entry>(runningQueries.values());
-            Collections.sort(entries);
-
             logger.info("System free memory less than " + alertMB + " MB. " + entries.size() + " queries running.");
-            long now = System.currentTimeMillis();
-
             for (int i = 0; i < entries.size(); i++) {
                 Entry e = entries.get(i);
                 notify("Low mem", (int) ((now - e.startTime) / 1000), e.sqlRequest.getSql());
             }
         }
+
+        // report if query running long
+        for (Entry e : entries) {
+            int runningSec = (int) ((now - e.startTime) / 1000);
+            if (runningSec >= alertRunningSec) {
+                notify("Slow", runningSec, e.sqlRequest.getSql());
+            } else {
+                break; // entries are sorted by startTime
+            }
+        }
     }
 
     public static final int ONE_MB = 1024 * 1024;


[02/50] [abbrv] incubator-kylin git commit: KYLIN-956 hbase compression: gzip for test case and snappy for binary

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/capacity-scheduler.xml b/examples/test_case_data/sandbox-hdp21/capacity-scheduler.xml
deleted file mode 100644
index bd2c284..0000000
--- a/examples/test_case_data/sandbox-hdp21/capacity-scheduler.xml
+++ /dev/null
@@ -1,112 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-
-  <property>
-    <name>yarn.scheduler.capacity.maximum-applications</name>
-    <value>10000</value>
-    <description>
-      Maximum number of applications that can be pending and running.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
-    <value>0.1</value>
-    <description>
-      Maximum percent of resources in the cluster which can be used to run 
-      application masters i.e. controls number of concurrent running
-      applications.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.resource-calculator</name>
-    <value>org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator</value>
-    <description>
-      The ResourceCalculator implementation to be used to compare 
-      Resources in the scheduler.
-      The default i.e. DefaultResourceCalculator only uses Memory while
-      DominantResourceCalculator uses dominant-resource to compare 
-      multi-dimensional resources such as Memory, CPU etc.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.queues</name>
-    <value>default</value>
-    <description>
-      The queues at the this level (root is the root queue).
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.capacity</name>
-    <value>100</value>
-    <description>Default queue target capacity.</description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
-    <value>1</value>
-    <description>
-      Default queue user limit a percentage from 0.0 to 1.0.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
-    <value>100</value>
-    <description>
-      The maximum capacity of the default queue. 
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.state</name>
-    <value>RUNNING</value>
-    <description>
-      The state of the default queue. State can be one of RUNNING or STOPPED.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.acl_submit_applications</name>
-    <value>*</value>
-    <description>
-      The ACL of who can submit jobs to the default queue.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.default.acl_administer_queue</name>
-    <value>*</value>
-    <description>
-      The ACL of who can administer jobs on the default queue.
-    </description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.node-locality-delay</name>
-    <value>-1</value>
-    <description>
-      Number of missed scheduling opportunities after which the CapacityScheduler 
-      attempts to schedule rack-local containers. 
-      Typically this should be set to number of racks in the cluster, this 
-      feature is disabled by default, set to -1.
-    </description>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/core-site.xml b/examples/test_case_data/sandbox-hdp21/core-site.xml
deleted file mode 100644
index 4cc666e..0000000
--- a/examples/test_case_data/sandbox-hdp21/core-site.xml
+++ /dev/null
@@ -1,106 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-    <property>
-    <name>hadoop.proxyuser.hcat.groups</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.hcat.hosts</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>fs.trash.interval</name>
-    <value>360</value>
-  </property>
-    <property>
-    <name>io.compression.codecs</name>
-    <value>org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.hive.groups</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.hue.hosts</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>io.file.buffer.size</name>
-    <value>131072</value>
-  </property>
-    <property>
-    <name>ipc.client.connection.maxidletime</name>
-    <value>30000</value>
-  </property>
-    <property>
-    <name>mapreduce.jobtracker.webinterface.trusted</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>hadoop.security.authorization</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.oozie.groups</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>hadoop.security.auth_to_local</name>
-    <value>
-        RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/
-        RULE:[2:$1@$0](jhs@.*)s/.*/mapred/
-        RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/
-        RULE:[2:$1@$0](hm@.*)s/.*/hbase/
-        RULE:[2:$1@$0](rs@.*)s/.*/hbase/
-        DEFAULT</value>
-  </property>
-    <property>
-    <name>hadoop.security.authentication</name>
-    <value>simple</value>
-  </property>
-    <property>
-    <name>ipc.client.idlethreshold</name>
-    <value>8000</value>
-  </property>
-    <property>
-    <name>fs.checkpoint.size</name>
-    <value>0.5</value>
-  </property>
-    <property>
-    <name>fs.defaultFS</name>
-    <value>hdfs://sandbox.hortonworks.com:8020</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.oozie.hosts</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.hive.hosts</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>io.serializations</name>
-    <value>org.apache.hadoop.io.serializer.WritableSerialization</value>
-  </property>
-    <property>
-    <name>hadoop.proxyuser.hue.groups</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>ipc.client.connect.max.retries</name>
-    <value>50</value>
-  </property>
-  </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/hadoop-policy.xml b/examples/test_case_data/sandbox-hdp21/hadoop-policy.xml
deleted file mode 100644
index 491dbe7..0000000
--- a/examples/test_case_data/sandbox-hdp21/hadoop-policy.xml
+++ /dev/null
@@ -1,219 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-
- Copyright 2011 The Apache Software Foundation
- 
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements.  See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership.  The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
-
--->
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-  <property>
-    <name>security.client.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ClientProtocol, which is used by user code
-    via the DistributedFileSystem.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.client.datanode.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ClientDatanodeProtocol, the client-to-datanode protocol
-    for block recovery.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.datanode.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for DatanodeProtocol, which is used by datanodes to
-    communicate with the namenode.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.inter.datanode.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for InterDatanodeProtocol, the inter-datanode protocol
-    for updating generation timestamp.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.namenode.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for NamenodeProtocol, the protocol used by the secondary
-    namenode to communicate with the namenode.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
- <property>
-    <name>security.admin.operations.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for AdminOperationsProtocol. Used for admin commands.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.refresh.usertogroups.mappings.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for RefreshUserMappingsProtocol. Used to refresh
-    users mappings. The ACL is a comma-separated list of user and
-    group names. The user and group list is separated by a blank. For
-    e.g. "alice,bob users,wheel".  A special value of "*" means all
-    users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.refresh.policy.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for RefreshAuthorizationPolicyProtocol, used by the
-    dfsadmin and mradmin commands to refresh the security policy in-effect.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.ha.service.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for HAService protocol used by HAAdmin to manage the
-      active and stand-by states of namenode.</description>
-  </property>
-
-  <property>
-    <name>security.zkfc.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for access to the ZK Failover Controller
-    </description>
-  </property>
-
-  <property>
-    <name>security.qjournal.service.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for QJournalProtocol, used by the NN to communicate with
-    JNs when using the QuorumJournalManager for edit logs.</description>
-  </property>
-
-  <property>
-    <name>security.mrhs.client.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for HSClientProtocol, used by job clients to
-    communciate with the MR History Server job status etc. 
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <!-- YARN Protocols -->
-
-  <property>
-    <name>security.resourcetracker.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ResourceTrackerProtocol, used by the
-    ResourceManager and NodeManager to communicate with each other.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.resourcemanager-administration.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ResourceManagerAdministrationProtocol, for admin commands. 
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.applicationclient.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ApplicationClientProtocol, used by the ResourceManager 
-    and applications submission clients to communicate with each other.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.applicationmaster.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ApplicationMasterProtocol, used by the ResourceManager 
-    and ApplicationMasters to communicate with each other.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.containermanagement.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ContainerManagementProtocol protocol, used by the NodeManager 
-    and ApplicationMasters to communicate with each other.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.resourcelocalizer.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for ResourceLocalizer protocol, used by the NodeManager 
-    and ResourceLocalizer to communicate with each other.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.job.task.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for TaskUmbilicalProtocol, used by the map and reduce
-    tasks to communicate with the parent tasktracker.
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-  <property>
-    <name>security.job.client.protocol.acl</name>
-    <value>*</value>
-    <description>ACL for MRClientProtocol, used by job clients to
-    communciate with the MR ApplicationMaster to query job status etc. 
-    The ACL is a comma-separated list of user and group names. The user and
-    group list is separated by a blank. For e.g. "alice,bob users,wheel".
-    A special value of "*" means all users are allowed.</description>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/hbase-site.xml b/examples/test_case_data/sandbox-hdp21/hbase-site.xml
deleted file mode 100644
index 0e4a814..0000000
--- a/examples/test_case_data/sandbox-hdp21/hbase-site.xml
+++ /dev/null
@@ -1,193 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-
-	<property>
-		<name>hbase.master.info.bindAddress</name>
-		<value>0.0.0.0</value>
-	</property>
-
-	<property>
-		<name>hbase.hstore.flush.retries.number</name>
-		<value>120</value>
-	</property>
-
-	<property>
-		<name>hbase.client.keyvalue.maxsize</name>
-		<value>10485760</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.keytab.file</name>
-		<value>/etc/security/keytabs/hbase.service.keytab</value>
-	</property>
-
-	<property>
-		<name>hbase.hstore.compactionThreshold</name>
-		<value>3</value>
-	</property>
-
-	<property>
-		<name>hbase.zookeeper.property.clientPort</name>
-		<value>2181</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.handler.count</name>
-		<value>60</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.global.memstore.lowerLimit</name>
-		<value>0.38</value>
-	</property>
-
-	<property>
-		<name>hbase.master.kerberos.principal</name>
-		<value>hbase/_HOST@EXAMPLE.COM</value>
-	</property>
-
-	<property>
-		<name>hbase.hregion.memstore.block.multiplier</name>
-		<value>2</value>
-	</property>
-
-	<property>
-		<name>hbase.hregion.memstore.flush.size</name>
-		<value>134217728</value>
-	</property>
-
-	<property>
-		<name>hbase.superuser</name>
-		<value>hbase</value>
-	</property>
-
-	<property>
-		<name>hbase.rootdir</name>
-		<value>hdfs://sandbox.hortonworks.com:8020/apps/hbase/data
-		</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.kerberos.principal</name>
-		<value>hbase/_HOST@EXAMPLE.COM</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.global.memstore.upperLimit</name>
-		<value>0.4</value>
-	</property>
-
-	<property>
-		<name>zookeeper.session.timeout</name>
-		<value>30000</value>
-	</property>
-
-	<property>
-		<name>hbase.client.scanner.caching</name>
-		<value>100</value>
-	</property>
-
-	<property>
-		<name>hbase.tmp.dir</name>
-		<value>/tmp/hbase</value>
-	</property>
-
-	<property>
-		<name>hfile.block.cache.size</name>
-		<value>0.40</value>
-	</property>
-
-	<property>
-		<name>hbase.hregion.max.filesize</name>
-		<value>10737418240</value>
-	</property>
-
-	<property>
-		<name>hbase.security.authentication</name>
-		<value>simple</value>
-	</property>
-
-	<property>
-		<name>hbase.defaults.for.version.skip</name>
-		<value>true</value>
-	</property>
-
-	<property>
-		<name>hbase.master.info.port</name>
-		<value>60010</value>
-	</property>
-
-	<property>
-		<name>hbase.zookeeper.quorum</name>
-		<value>sandbox.hortonworks.com</value>
-	</property>
-
-	<property>
-		<name>hbase.regionserver.info.port</name>
-		<value>60030</value>
-	</property>
-
-	<property>
-		<name>zookeeper.znode.parent</name>
-		<value>/hbase-unsecure</value>
-	</property>
-
-	<property>
-		<name>hbase.hstore.blockingStoreFiles</name>
-		<value>10</value>
-	</property>
-
-	<property>
-		<name>hbase.hregion.majorcompaction</name>
-		<value>86400000</value>
-	</property>
-
-	<property>
-		<name>hbase.security.authorization</name>
-		<value>false</value>
-	</property>
-
-	<property>
-		<name>hbase.master.keytab.file</name>
-		<value>/etc/security/keytabs/hbase.service.keytab</value>
-	</property>
-
-	<property>
-		<name>hbase.local.dir</name>
-		<value>${hbase.tmp.dir}/local</value>
-	</property>
-
-	<property>
-		<name>hbase.cluster.distributed</name>
-		<value>true</value>
-	</property>
-
-	<property>
-		<name>hbase.hregion.memstore.mslab.enabled</name>
-		<value>true</value>
-	</property>
-
-	<property>
-		<name>dfs.domain.socket.path</name>
-		<value>/var/lib/hadoop-hdfs/dn_socket</value>
-	</property>
-
-	<property>
-		<name>hbase.zookeeper.useMulti</name>
-		<value>true</value>
-	</property>
-
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/hdfs-site.xml b/examples/test_case_data/sandbox-hdp21/hdfs-site.xml
deleted file mode 100644
index bb85d10..0000000
--- a/examples/test_case_data/sandbox-hdp21/hdfs-site.xml
+++ /dev/null
@@ -1,188 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-    <property>
-    <name>dfs.namenode.stale.datanode.interval</name>
-    <value>30000</value>
-  </property>
-    <property>
-    <name>dfs.https.port</name>
-    <value>50470</value>
-  </property>
-    <property>
-    <name>dfs.datanode.address</name>
-    <value>0.0.0.0:50010</value>
-  </property>
-    <property>
-    <name>dfs.cluster.administrators</name>
-    <value>hdfs</value>
-  </property>
-    <property>
-    <name>dfs.namenode.checkpoint.dir</name>
-    <value>/hadoop/hdfs/namesecondary</value>
-  </property>
-    <property>
-    <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
-    <value>NEVER</value>
-  </property>
-    <property>
-    <name>dfs.client.read.shortcircuit</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.blockreport.initialDelay</name>
-    <value>120</value>
-  </property>
-    <property>
-    <name>dfs.namenode.avoid.read.stale.datanode</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.namenode.checkpoint.period</name>
-    <value>21600</value>
-  </property>
-    <property>
-    <name>dfs.namenode.secondary.http-address</name>
-    <value>sandbox.hortonworks.com:50090</value>
-  </property>
-    <property>
-    <name>dfs.support.append</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.heartbeat.interval</name>
-    <value>3</value>
-  </property>
-    <property>
-    <name>dfs.datanode.http.address</name>
-    <value>0.0.0.0:50075</value>
-  </property>
-    <property>
-    <name>dfs.replication</name>
-    <value>3</value>
-  </property>
-    <property>
-    <name>dfs.datanode.du.reserved</name>
-    <value>1073741824</value>
-  </property>
-    <property>
-    <name>dfs.permissions.superusergroup</name>
-    <value>hdfs</value>
-  </property>
-    <property>
-    <name>dfs.datanode.balance.bandwidthPerSec</name>
-    <value>6250000</value>
-  </property>
-    <property>
-    <name>dfs.blocksize</name>
-    <value>134217728</value>
-  </property>
-    <property>
-    <name>dfs.namenode.name.dir</name>
-    <value>/hadoop/hdfs/namenode</value>
-  </property>
-    <property>
-    <name>dfs.namenode.checkpoint.edits.dir</name>
-    <value>${dfs.namenode.checkpoint.dir}</value>
-  </property>
-    <property>
-    <name>dfs.namenode.accesstime.precision</name>
-    <value>0</value>
-  </property>
-    <property>
-    <name>dfs.datanode.ipc.address</name>
-    <value>0.0.0.0:8010</value>
-  </property>
-    <property>
-    <name>dfs.datanode.data.dir.perm</name>
-    <value>750</value>
-  </property>
-    <property>
-    <name>dfs.webhdfs.enabled</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.namenode.handler.count</name>
-    <value>5</value>
-  </property>
-    <property>
-    <name>dfs.client.read.shortcircuit.streams.cache.size</name>
-    <value>4096</value>
-  </property>
-    <property>
-    <name>dfs.replication.max</name>
-    <value>50</value>
-  </property>
-    <property>
-    <name>dfs.namenode.write.stale.datanode.ratio</name>
-    <value>1.0f</value>
-  </property>
-    <property>
-    <name>dfs.datanode.max.transfer.threads</name>
-    <value>1024</value>
-  </property>
-    <property>
-    <name>dfs.namenode.https-address</name>
-    <value>sandbox.hortonworks.com:50470</value>
-  </property>
-  <property>
-    <name>dfs.permissions</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>dfs.domain.socket.path</name>
-    <value>/var/lib/hadoop-hdfs/dn_socket</value>
-  </property>
-    <property>
-    <name>dfs.namenode.avoid.write.stale.datanode</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.block.access.token.enable</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>dfs.hosts.exclude</name>
-    <value>/etc/hadoop/conf/dfs.exclude</value>
-  </property>
-    <property>
-    <name>dfs.journalnode.http-address</name>
-    <value>0.0.0.0:8480</value>
-  </property>
-    <property>
-    <name>fs.permissions.umask-mode</name>
-    <value>022</value>
-  </property>
-    <property>
-    <name>dfs.datanode.data.dir</name>
-    <value>/hadoop/hdfs/data</value>
-  </property>
-    <property>
-    <name>dfs.journalnode.edits.dir</name>
-    <value>/grid/0/hdfs/journal</value>
-  </property>
-    <property>
-    <name>dfs.namenode.safemode.threshold-pct</name>
-    <value>1.0f</value>
-  </property>
-    <property>
-    <name>dfs.namenode.http-address</name>
-    <value>sandbox.hortonworks.com:50070</value>
-  </property>
-    <property>
-    <name>dfs.datanode.failed.volumes.tolerated</name>
-    <value>0</value>
-  </property>
-  </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/hive-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/hive-site.xml b/examples/test_case_data/sandbox-hdp21/hive-site.xml
deleted file mode 100644
index 590a7fb..0000000
--- a/examples/test_case_data/sandbox-hdp21/hive-site.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-  <property>
-    <name>hive.metastore.uris</name>
-    <value>thrift://sandbox.hortonworks.com:9083</value>
-  </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/httpfs-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/httpfs-site.xml b/examples/test_case_data/sandbox-hdp21/httpfs-site.xml
deleted file mode 100644
index 4a718e1..0000000
--- a/examples/test_case_data/sandbox-hdp21/httpfs-site.xml
+++ /dev/null
@@ -1,17 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<configuration>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/kylin.properties b/examples/test_case_data/sandbox-hdp21/kylin.properties
deleted file mode 100644
index bb1719e..0000000
--- a/examples/test_case_data/sandbox-hdp21/kylin.properties
+++ /dev/null
@@ -1,84 +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.
-#
-
-## Config for Kylin Engine ##
-
-# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.rest.servers=localhost:7070
-
-# The metadata store in hbase
-kylin.metadata.url=kylin_metadata_qa@hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# The storage for final cube file in hbase
-kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# Temp folder in hdfs, make sure user has the right access to the hdfs directory
-kylin.hdfs.working.dir=/kylin
-
-kylin.job.mapreduce.default.reduce.input.mb=500
-
-# If true, job engine will not assume that hadoop CLI reside on the same server as it self
-# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
-kylin.job.run.as.remote.cmd=false
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.hostname=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.username=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.password=
-
-# Used by test cases to prepare synthetic data for sample cube
-kylin.job.remote.cli.working.dir=/tmp/kylin
-
-# Max count of concurrent jobs running
-kylin.job.concurrent.max.limit=10
-
-# Time interval to check hadoop job status
-kylin.job.yarn.app.rest.check.interval.seconds=10
-
-
-## Config for Restful APP ##
-# database connection settings:
-ldap.server=
-ldap.username=
-ldap.password=
-ldap.user.searchBase=
-ldap.user.searchPattern=
-ldap.user.groupSearchBase=
-ldap.service.searchBase=OU=
-ldap.service.searchPattern=
-ldap.service.groupSearchBase=
-acl.adminRole=
-acl.defaultRole=
-ganglia.group=
-ganglia.port=8664
-
-## Config for mail service
-
-# If true, will send email notification;
-mail.enabled=false
-mail.host=
-mail.username=
-mail.password=
-mail.sender=
-
-###########################config info for sandbox#######################
-kylin.sandbox=true
-

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_disabled.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_disabled.xml b/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_disabled.xml
deleted file mode 100644
index 62aff4a..0000000
--- a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_disabled.xml
+++ /dev/null
@@ -1,74 +0,0 @@
-<?xml version="1.0"?>
-<!--
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-
-    <property>
-        <name>mapreduce.job.split.metainfo.maxsize</name>
-        <value>-1</value>
-        <description>The maximum permissible size of the split metainfo file.
-            The JobTracker won't attempt to read split metainfo files bigger than
-            the configured value. No limits if set to -1.
-        </description>
-    </property>
-
-    <!-- uncomment the following 5 properties to enable lzo compressing
-
-	<property>
-		<name>mapred.compress.map.output</name>
-		<value>true</value>
-		<description>Compress map outputs</description>
-	</property>
-
-	<property>
-		<name>mapred.map.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for map outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compress</name>
-		<value>true</value>
-		<description>Compress the output of a MapReduce job</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for job outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.type</name>
-		<value>BLOCK</value>
-		<description>The compression type to use for job outputs</description>
-	</property>
-
-	!-->
-
-    <property>
-        <name>mapreduce.job.max.split.locations</name>
-        <value>2000</value>
-        <description>No description</description>
-    </property>
-
-    <property>
-        <name>dfs.replication</name>
-        <value>2</value>
-        <description>Block replication</description>
-    </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_enabled.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_enabled.xml b/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_enabled.xml
deleted file mode 100644
index 62aff4a..0000000
--- a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.lzo_enabled.xml
+++ /dev/null
@@ -1,74 +0,0 @@
-<?xml version="1.0"?>
-<!--
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-
-    <property>
-        <name>mapreduce.job.split.metainfo.maxsize</name>
-        <value>-1</value>
-        <description>The maximum permissible size of the split metainfo file.
-            The JobTracker won't attempt to read split metainfo files bigger than
-            the configured value. No limits if set to -1.
-        </description>
-    </property>
-
-    <!-- uncomment the following 5 properties to enable lzo compressing
-
-	<property>
-		<name>mapred.compress.map.output</name>
-		<value>true</value>
-		<description>Compress map outputs</description>
-	</property>
-
-	<property>
-		<name>mapred.map.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for map outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compress</name>
-		<value>true</value>
-		<description>Compress the output of a MapReduce job</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for job outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.type</name>
-		<value>BLOCK</value>
-		<description>The compression type to use for job outputs</description>
-	</property>
-
-	!-->
-
-    <property>
-        <name>mapreduce.job.max.split.locations</name>
-        <value>2000</value>
-        <description>No description</description>
-    </property>
-
-    <property>
-        <name>dfs.replication</name>
-        <value>2</value>
-        <description>Block replication</description>
-    </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.xml b/examples/test_case_data/sandbox-hdp21/kylin_job_conf.xml
deleted file mode 100644
index 62aff4a..0000000
--- a/examples/test_case_data/sandbox-hdp21/kylin_job_conf.xml
+++ /dev/null
@@ -1,74 +0,0 @@
-<?xml version="1.0"?>
-<!--
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-
-    <property>
-        <name>mapreduce.job.split.metainfo.maxsize</name>
-        <value>-1</value>
-        <description>The maximum permissible size of the split metainfo file.
-            The JobTracker won't attempt to read split metainfo files bigger than
-            the configured value. No limits if set to -1.
-        </description>
-    </property>
-
-    <!-- uncomment the following 5 properties to enable lzo compressing
-
-	<property>
-		<name>mapred.compress.map.output</name>
-		<value>true</value>
-		<description>Compress map outputs</description>
-	</property>
-
-	<property>
-		<name>mapred.map.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for map outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compress</name>
-		<value>true</value>
-		<description>Compress the output of a MapReduce job</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for job outputs
-		</description>
-	</property>
-
-	<property>
-		<name>mapred.output.compression.type</name>
-		<value>BLOCK</value>
-		<description>The compression type to use for job outputs</description>
-	</property>
-
-	!-->
-
-    <property>
-        <name>mapreduce.job.max.split.locations</name>
-        <value>2000</value>
-        <description>No description</description>
-    </property>
-
-    <property>
-        <name>dfs.replication</name>
-        <value>2</value>
-        <description>Block replication</description>
-    </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/mapred-site.xml b/examples/test_case_data/sandbox-hdp21/mapred-site.xml
deleted file mode 100644
index 14e1dd9..0000000
--- a/examples/test_case_data/sandbox-hdp21/mapred-site.xml
+++ /dev/null
@@ -1,169 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-    <property>
-    <name>mapreduce.jobhistory.webapp.address</name>
-    <value>sandbox.hortonworks.com:19888</value>
-  </property>
-    <property>
-    <name>mapreduce.shuffle.port</name>
-    <value>13562</value>
-  </property>
-    <property>
-    <name>mapreduce.cluster.administrators</name>
-    <value> hadoop</value>
-  </property>
-    <property>
-    <name>mapreduce.task.timeout</name>
-    <value>300000</value>
-  </property>
-    <property>
-    <name>mapreduce.map.sort.spill.percent</name>
-    <value>0.7</value>
-  </property>
-    <property>
-    <name>mapreduce.admin.user.env</name>
-    <value>LD_LIBRARY_PATH=/usr/lib/hadoop/lib/native:/usr/lib/hadoop/lib/native/`$JAVA_HOME/bin/java -d32 -version &amp;&gt; /dev/null;if [ $? -eq 0 ]; then echo Linux-i386-32; else echo Linux-amd64-64;fi`</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.shuffle.input.buffer.percent</name>
-    <value>0.7</value>
-  </property>
-    <property>
-    <name>mapreduce.admin.reduce.child.java.opts</name>
-    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.input.buffer.percent</name>
-    <value>0.0</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.shuffle.parallelcopies</name>
-    <value>30</value>
-  </property>
-    <property>
-    <name>mapreduce.output.fileoutputformat.compress.type</name>
-    <value>BLOCK</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.memory.mb</name>
-    <value>1024</value>
-  </property>
-    <property>
-    <name>mapreduce.task.io.sort.mb</name>
-    <value>200</value>
-  </property>
-    <property>
-    <name>mapreduce.job.reduce.slowstart.completedmaps</name>
-    <value>0.05</value>
-  </property>
-    <property>
-    <name>yarn.app.mapreduce.am.log.level</name>
-    <value>INFO</value>
-  </property>
-    <property>
-    <name>mapreduce.map.java.opts</name>
-    <value>-Xmx512m</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.log.level</name>
-    <value>INFO</value>
-  </property>
-    <property>
-    <name>mapreduce.map.log.level</name>
-    <value>INFO</value>
-  </property>
-    <property>
-    <name>mapreduce.am.max-attempts</name>
-    <value>2</value>
-  </property>
-    <property>
-    <name>yarn.app.mapreduce.am.staging-dir</name>
-    <value>/user</value>
-  </property>
-    <property>
-    <name>mapreduce.task.io.sort.factor</name>
-    <value>100</value>
-  </property>
-    <property>
-    <name>mapreduce.framework.name</name>
-    <value>yarn</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.shuffle.merge.percent</name>
-    <value>0.66</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.speculative</name>
-    <value>false</value>
-  </property>
-  <property>
-    <name>mapreduce.application.classpath</name>
-    <value>/usr/lib/hive/conf/,/usr/lib/hbase/lib/*,/usr/lib/hive-hcatalog/share/hcatalog/*,/usr/lib/hive/lib/*</value>
-  </property>
-  
-  <property>
-    <name>mapreduce.map.output.compress</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>mapreduce.map.speculative</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>mapreduce.map.memory.mb</name>
-    <value>1024</value>
-  </property>
-    <property>
-    <name>yarn.app.mapreduce.am.admin-command-opts</name>
-    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
-  </property>
-    <property>
-    <name>mapreduce.jobhistory.address</name>
-    <value>sandbox.hortonworks.com:10020</value>
-  </property>
-    <property>
-    <name>mapreduce.jobhistory.done-dir</name>
-    <value>/mr-history/done</value>
-  </property>
-    <property>
-    <name>mapreduce.output.fileoutputformat.compress</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>yarn.app.mapreduce.am.command-opts</name>
-    <value>-Xmx312m</value>
-  </property>
-    <property>
-    <name>mapreduce.reduce.java.opts</name>
-    <value>-Xmx512m</value>
-  </property>
-    <property>
-    <name>yarn.app.mapreduce.am.resource.mb</name>
-    <value>250</value>
-  </property>
-    <property>
-    <name>mapreduce.admin.map.child.java.opts</name>
-    <value>-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN</value>
-  </property>
-    <property>
-    <name>mapreduce.jobhistory.intermediate-done-dir</name>
-    <value>/mr-history/tmp</value>
-  </property>
-  <property>
-	<name>mapreduce.app-submission.cross-platform</name>
-	<value>true</value>
-  </property>
-  </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/sandbox-hdp21/yarn-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox-hdp21/yarn-site.xml b/examples/test_case_data/sandbox-hdp21/yarn-site.xml
deleted file mode 100644
index 1b4c8be..0000000
--- a/examples/test_case_data/sandbox-hdp21/yarn-site.xml
+++ /dev/null
@@ -1,160 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<configuration>
-    <property>
-    <name>yarn.nodemanager.aux-services</name>
-    <value>mapreduce_shuffle</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.aux-services.mapreduce_shuffle.class</name>
-    <value>org.apache.hadoop.mapred.ShuffleHandler</value>
-  </property>
-    <property>
-    <name>yarn.log-aggregation.retain-seconds</name>
-    <value>2592000</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.address</name>
-    <value>0.0.0.0:45454</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.log-aggregation.compression-type</name>
-    <value>gz</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.resource.memory-mb</name>
-    <value>8000</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.container-executor.class</name>
-    <value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.linux-container-executor.group</name>
-    <value>hadoop</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.resource-tracker.address</name>
-    <value>sandbox.hortonworks.com:8025</value>
-  </property>
-  <property>
-    <name>yarn.application.classpath</name>
-    <value>/etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.log.retain-second</name>
-    <value>604800</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.hostname</name>
-    <value>sandbox.hortonworks.com</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.admin-env</name>
-    <value>MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX</value>
-  </property>
-    <property>
-    <name>yarn.scheduler.minimum-allocation-mb</name>
-    <value>64</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.health-checker.script.timeout-ms</name>
-    <value>60000</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.health-checker.interval-ms</name>
-    <value>135000</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.container-monitor.interval-ms</name>
-    <value>3000</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.remote-app-log-dir-suffix</name>
-    <value>logs</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.scheduler.class</name>
-    <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler</value>
-  </property>
-    <property>
-    <name>yarn.admin.acl</name>
-    <value>*</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.disk-health-checker.min-healthy-disks</name>
-    <value>0.25</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.remote-app-log-dir</name>
-    <value>/app-logs</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.local-dirs</name>
-    <value>/hadoop/yarn/local</value>
-  </property>
-    <property>
-    <name>yarn.scheduler.maximum-allocation-mb</name>
-    <value>4096</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.log-dirs</name>
-    <value>/hadoop/yarn/log</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.admin.address</name>
-    <value>sandbox.hortonworks.com:8141</value>
-  </property>
-    <property>
-    <name>yarn.log.server.url</name>
-    <value>http://sandbox.hortonworks.com:19888/jobhistory/logs</value>
-  </property>
-    <property>
-    <name>yarn.acl.enable</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.vmem-pmem-ratio</name>
-    <value>10</value>
-  </property>
-    <property>
-    <name>yarn.log-aggregation-enable</name>
-    <value>true</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.address</name>
-    <value>sandbox.hortonworks.com:8050</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.scheduler.address</name>
-    <value>sandbox.hortonworks.com:8030</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.webapp.address</name>
-    <value>sandbox.hortonworks.com:8088</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.vmem-check-enabled</name>
-    <value>false</value>
-  </property>
-    <property>
-    <name>yarn.resourcemanager.am.max-attempts</name>
-    <value>2</value>
-  </property>
-    <property>
-    <name>yarn.nodemanager.delete.debug-delay-sec</name>
-    <value>0</value>
-  </property>
-  </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/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 767dcb2..359174d 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -56,6 +56,9 @@ kylin.job.yarn.app.rest.check.interval.seconds=10
 # Hive database name for putting the intermediate flat tables
 kylin.job.hive.database.for.intermediatetable=default
 
+#default compression codec for htable,snappy,lzo,gzip,lz4
+kylin.hbase.default.compression.codec=gzip
+
 ## Config for Restful APP ##
 # database connection settings:
 ldap.server=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/website/_docs/install/advance_settings.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/advance_settings.md b/website/_docs/install/advance_settings.md
index 17288f0..d876b8e 100644
--- a/website/_docs/install/advance_settings.md
+++ b/website/_docs/install/advance_settings.md
@@ -15,7 +15,7 @@ To enable LZO in Kylin, follow these steps:
 
 ### Make sure LZO is working in your environment
 
-We have a simple tool to test whether LZO is well installed on EVERY SERVER in hbase cluster (http://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.1.2/bk_installing_manually_book/content/rpm-chap2-3.html), and restart the cluster.
+We have a simple tool to test whether LZO is well installed on EVERY SERVER in hbase cluster ( http://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.2.4/bk_installing_manually_book/content/ch_install_hdfs_yarn_chapter.html#install-snappy-man-install ), and restart the cluster.
 To test it on the hadoop CLI that you deployed Kylin, Just run
 
 {% highlight Groff markup %}


[40/50] [abbrv] incubator-kylin git commit: KYLIN-948 Hive connection resouce not close

Posted by li...@apache.org.
KYLIN-948 Hive connection resouce not close

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/c0ef677c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/c0ef677c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/c0ef677c

Branch: refs/heads/master
Commit: c0ef677cbfaf19351ac8beeef839aec79138fe12
Parents: 876079b
Author: wangxiaoyu8 <wa...@jd.com>
Authored: Wed Sep 9 17:22:18 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Fri Sep 11 18:44:03 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/monitor/HiveJdbcClient.java    | 45 ++++++++++++++------
 1 file changed, 33 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c0ef677c/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
index 2eda2e8..ffe3cc0 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
@@ -28,16 +28,16 @@ import java.sql.Statement;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
 
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
 import org.apache.log4j.Logger;
 import org.datanucleus.util.StringUtils;
 import org.w3c.dom.Document;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
 /**
  * @author jiazhong
  */
@@ -93,20 +93,20 @@ public class HiveJdbcClient {
 
         MAP_QUEUE_NAME = getQueueName();
 
-        if(org.apache.commons.lang.StringUtils.isEmpty(MAP_QUEUE_NAME)){
+        if (org.apache.commons.lang.StringUtils.isEmpty(MAP_QUEUE_NAME)) {
             MAP_QUEUE_NAME = monitorConfig.getKylinMapJobQueue();
         }
 
-        logger.info("mapred.job.queue.name:"+MAP_QUEUE_NAME);
+        logger.info("mapred.job.queue.name:" + MAP_QUEUE_NAME);
 
         String CON_URL = monitorConfig.getHiveJdbcConUrl();
 
         String USER_NAME = monitorConfig.getHiveJdbcConUserName();
         String PASSWD = monitorConfig.getHiveJdbcConPasswd();
 
-        Connection con = DriverManager.getConnection(CON_URL,USER_NAME,PASSWD);
+        Connection con = DriverManager.getConnection(CON_URL, USER_NAME, PASSWD);
         Statement stmt = con.createStatement();
-        stmt.execute("set mapred.job.queue.name="+MAP_QUEUE_NAME);
+        stmt.execute("set mapred.job.queue.name=" + MAP_QUEUE_NAME);
         ResultSet res = null;
 
         SQL_GENERATE_QUERY_LOG_TABLE = generateQueryLogSql();
@@ -187,7 +187,27 @@ public class HiveJdbcClient {
             FileUtils.appendResultToHdfs(each_day_percentile_file, new String[] { res.getString(1), res.getString(2), res.getString(3) });
             logger.info(res.getString(1) + "," + res.getString(2) + "," + res.getString(3));
         }
-
+        if (res != null) {
+            try {
+                res.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        }
+        if (stmt != null) {
+            try {
+                stmt.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        }
+        if (con != null) {
+            try {
+                con.close();
+            } catch (SQLException e) {
+                logger.error("failed to close", e);
+            }
+        }
     }
 
     public String generateQueryLogSql() {
@@ -228,8 +248,8 @@ public class HiveJdbcClient {
 
     public String getQueueName() throws IOException {
         String queueName = "";
-        InputStream stream =  this.getClass().getClassLoader().getResourceAsStream(KYLIN_JOB_CONF_XML);
-        if (stream!=null) {
+        InputStream stream = this.getClass().getClassLoader().getResourceAsStream(KYLIN_JOB_CONF_XML);
+        if (stream != null) {
             DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
             DocumentBuilder builder;
             Document doc;
@@ -253,6 +273,7 @@ public class HiveJdbcClient {
         }
         return queueName;
     }
+
     public String monthStasticSqlConvert(String sql) {
 
         SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd");
@@ -264,4 +285,4 @@ public class HiveJdbcClient {
         return sql.replace("[START_DATE]", "'" + startDate + "'").replace("[END_DATE]", "'" + endDate + "'");
     }
 
-}
\ No newline at end of file
+}


[32/50] [abbrv] incubator-kylin git commit: update website blog layout

Posted by li...@apache.org.
update website blog layout


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

Branch: refs/heads/master
Commit: 3991b639e5506ae31b7b1cf93f36769fd4a52eb6
Parents: 277e152
Author: Luke Han <lu...@apache.org>
Authored: Tue Sep 8 09:53:26 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Tue Sep 8 09:53:26 2015 +0800

----------------------------------------------------------------------
 .gitignore                     |  7 +++++++
 website/_data/contributors.yml |  3 ++-
 website/blog/index.md          | 10 ++--------
 3 files changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3991b639/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 6742b8a..b016aa2 100644
--- a/.gitignore
+++ b/.gitignore
@@ -91,3 +91,10 @@ tomcat
 lib
 webapp/app/components/*
 commit.sha1
+
+# website 
+website/_site
+website/.sass-cache
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3991b639/website/_data/contributors.yml
----------------------------------------------------------------------
diff --git a/website/_data/contributors.yml b/website/_data/contributors.yml
index e731648..56c5b53 100644
--- a/website/_data/contributors.yml
+++ b/website/_data/contributors.yml
@@ -87,7 +87,7 @@
   role: PMC
 - name: Dayue Gao (高大月)
   apacheId: 
-  githubId: 
+  githubId: gaodayue
   org: Meituan
   role: committer
 - name: Hua Huang (黄桦)
@@ -97,3 +97,4 @@
   role: committer 
 
 
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3991b639/website/blog/index.md
----------------------------------------------------------------------
diff --git a/website/blog/index.md b/website/blog/index.md
index 5574ce6..6417344 100644
--- a/website/blog/index.md
+++ b/website/blog/index.md
@@ -11,20 +11,14 @@ title: Blog
          <!-- second-->
           <div id="content-container" class="animated fadeIn">
             <div >
-            <!--
-            <p class="aboutkylin" style="font-size:1.2em">Comming Soon...</p>
-            -->
              <ul class="post-list">
             {% for category in site.categories %}     <!-- categories -->
             {% if category[0]  == 'blog' %}
             {% for post in category[1] %}
             <li>
-            <!--
-            <span align="left" class="content-header">{{ post.date | date: "%b %-d, %Y" }}</span>
-            -->
         <h2 align="left">
-          <a class="post-link" href="{{ post.url | prepend: site.baseurl }}">{{ post.title }}</a>
-        </h2>
+          <a class="post-link" href="{{ post.url | prepend: site.baseurl }}">{{ post.title }}</a></h2><div align="left" class="post-meta" >posted: {{ post.date | date: "%b %-d, %Y" }}</div>
+        
       </li>
     {% endfor %}
     {% endif %}


[20/50] [abbrv] incubator-kylin git commit: KYLIN-957 fix a path issue (“//path”) when kylin.hadoop.cluster.fs wasn’t set.

Posted by li...@apache.org.
KYLIN-957 fix a path issue (“//path”) when kylin.hadoop.cluster.fs wasn’t set.

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

Branch: refs/heads/master
Commit: a47fc5d986d143678f2dd4c43536deccd896fe80
Parents: c9f819e
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 10:02:13 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 10:02:13 2015 +0800

----------------------------------------------------------------------
 bin/check-env.sh      | 4 ++--
 conf/kylin.properties | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a47fc5d9/bin/check-env.sh
----------------------------------------------------------------------
diff --git a/bin/check-env.sh b/bin/check-env.sh
index 89a4aed..9efb393 100644
--- a/bin/check-env.sh
+++ b/bin/check-env.sh
@@ -47,9 +47,9 @@ fi
 
 WORKING_DIR=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hdfs.working.dir`
 HADOOP_FS=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hadoop.cluster.fs`
-hadoop fs -mkdir -p $HADOOP_FS/$WORKING_DIR
+hadoop fs -mkdir -p $HADOOP_FS$WORKING_DIR
 if [ $? != 0 ]
 then
-    echo "failed to create $HADOOP_FS/$WORKING_DIR, Please make sure the user has right to access $HADOOP_FS/$WORKING_DIR"
+    echo "failed to create $HADOOP_FS$WORKING_DIR, Please make sure the user has right to access $HADOOP_FS$WORKING_DIR"
     exit 1
 fi

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a47fc5d9/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 98eeafc..3850005 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -29,11 +29,11 @@ kylin.storage.url=hbase
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
-# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster/
+# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster/ (end with /)
 # leave empty if using default fs configured by local core-site.xml
 kylin.hadoop.cluster.fs=
 
-# HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster/
+# HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster/ (end with /)
 # leave empty if hbase running on same cluster with hive and mapreduce
 kylin.hbase.cluster.fs=
 


[45/50] [abbrv] incubator-kylin git commit: KYLIN-1026 Error message for git check is not correct in package.sh

Posted by li...@apache.org.
KYLIN-1026 Error message for git check is not correct in package.sh

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/b0039214
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/b0039214
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/b0039214

Branch: refs/heads/master
Commit: b003921468318b84b9c5ad3ef131877c728225df
Parents: f929892
Author: zkld123 <zk...@sjtu.edu.cn>
Authored: Tue Sep 15 20:27:42 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Tue Sep 15 20:38:57 2015 +0800

----------------------------------------------------------------------
 script/package.sh | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b0039214/script/package.sh
----------------------------------------------------------------------
diff --git a/script/package.sh b/script/package.sh
index 759c082..820dce2 100755
--- a/script/package.sh
+++ b/script/package.sh
@@ -26,9 +26,11 @@ else
     echo "maven check passed"
 fi
 
+echo "Checking git..."
+
 if [ -z "$(command -v git)" ]
 then
-    echo "Please install maven first so that Kylin packaging can proceed"
+    echo "Please install git first so that Kylin packaging can proceed"
     exit 1
 else
     echo "git check passed"


[46/50] [abbrv] incubator-kylin git commit: KYLIN-968 CubeSegment.lastBuildJobID is null in new instance but used for rowkey_stats path

Posted by li...@apache.org.
KYLIN-968 CubeSegment.lastBuildJobID is null in new instance but used for rowkey_stats path

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/8fd1404d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/8fd1404d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/8fd1404d

Branch: refs/heads/master
Commit: 8fd1404dd748b6761ef2e51c197ba5ef4c56c11c
Parents: b003921
Author: sunyerui <su...@gmail.com>
Authored: Thu Sep 10 11:23:16 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 16 21:01:59 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/job/cube/CubingJobBuilder.java | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8fd1404d/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
index 5c3c277..ff79286 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
@@ -206,9 +206,9 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         final String jobId = result.getId();
         final String cuboidPath = cuboidRootPath + "*";
 
-        result.addTask(createRangeRowkeyDistributionStep(seg, cuboidPath));
+        result.addTask(createRangeRowkeyDistributionStep(seg, cuboidPath, jobId));
         // create htable step
-        result.addTask(createCreateHTableStep(seg));
+        result.addTask(createCreateHTableStep(seg, jobId));
         // generate hfiles step
         final MapReduceExecutable convertCuboidToHfileStep = createConvertCuboidToHfileStep(seg, cuboidPath, jobId);
         result.addTask(convertCuboidToHfileStep);
@@ -265,8 +265,8 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         return getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/cuboid/*";
     }
 
-    private String getRowkeyDistributionOutputPath(CubeSegment seg) {
-        return getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
+    private String getRowkeyDistributionOutputPath(CubeSegment seg, String jobId) {
+        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
     }
 
     private String getFactDistinctColumnsPath(CubeSegment seg, String jobUuid) {
@@ -347,14 +347,14 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         return ndCuboidStep;
     }
 
-    private MapReduceExecutable createRangeRowkeyDistributionStep(CubeSegment seg, String inputPath) {
+    private MapReduceExecutable createRangeRowkeyDistributionStep(CubeSegment seg, String inputPath, String jobId) {
         MapReduceExecutable rowkeyDistributionStep = new MapReduceExecutable();
         rowkeyDistributionStep.setName(ExecutableConstants.STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION);
         StringBuilder cmd = new StringBuilder();
 
         appendMapReduceParameters(cmd, seg);
         appendExecCmdParameters(cmd, "input", inputPath);
-        appendExecCmdParameters(cmd, "output", getRowkeyDistributionOutputPath(seg));
+        appendExecCmdParameters(cmd, "output", getRowkeyDistributionOutputPath(seg, jobId));
         appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
         appendExecCmdParameters(cmd, "jobname", "Kylin_Region_Splits_Calculator_" + seg.getCubeInstance().getName() + "_Step");
 
@@ -363,12 +363,12 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         return rowkeyDistributionStep;
     }
 
-    private HadoopShellExecutable createCreateHTableStep(CubeSegment seg) {
+    private HadoopShellExecutable createCreateHTableStep(CubeSegment seg, String jobId) {
         HadoopShellExecutable createHtableStep = new HadoopShellExecutable();
         createHtableStep.setName(ExecutableConstants.STEP_NAME_CREATE_HBASE_TABLE);
         StringBuilder cmd = new StringBuilder();
         appendExecCmdParameters(cmd, "cubename", seg.getCubeInstance().getName());
-        appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath(seg) + "/part-r-00000");
+        appendExecCmdParameters(cmd, "input", getRowkeyDistributionOutputPath(seg, jobId) + "/part-r-00000");
         appendExecCmdParameters(cmd, "htablename", seg.getStorageLocationIdentifier());
 
         createHtableStep.setJobParams(cmd.toString());


[25/50] [abbrv] incubator-kylin git commit: add v1.0 release announcement blog

Posted by li...@apache.org.
add v1.0 release announcement blog


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

Branch: refs/heads/master
Commit: 60a95f3b8bcffa59f7f9692486578ff7ed1aaad6
Parents: 33f1a8b
Author: Luke Han <lu...@apache.org>
Authored: Mon Sep 7 16:07:51 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Mon Sep 7 16:07:51 2015 +0800

----------------------------------------------------------------------
 website/_data/contributors.yml                  | 33 +++++++++------
 .../blog/2015-09-06-release-v1.0-incubating.md  | 44 ++++++++++++++++++++
 website/community/index.md                      | 17 +++++++-
 3 files changed, 81 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/60a95f3b/website/_data/contributors.yml
----------------------------------------------------------------------
diff --git a/website/_data/contributors.yml b/website/_data/contributors.yml
index 776caa6..e731648 100644
--- a/website/_data/contributors.yml
+++ b/website/_data/contributors.yml
@@ -27,27 +27,27 @@
 - name: Hongbin Ma (马洪宾)
   apacheId: mahongbin
   githubId: binmahone
-  org: 
+  org: eBay
   role: PMC
 - name: Jason Zhong (仲俭)
   apacheId: zhongjian
   githubId: janzhongi
-  org: 
+  org: eBay
   role: PMC
 - name: Julian Hyde
   apacheId: jhyde
   githubId: julianhyde
-  org: 
+  org: Hortonworks
   role: Mentor
 - name: Luke Han (韩卿) 
   apacheId: lukehan
   githubId: lukehan
-  org: 
+  org: eBay
   role: PMC
 - name: Owen O'Malley
   apacheId: omalley
   githubId: omalley
-  org: 
+  org: Hortonworks
   role: Mentor
 - name: P. Taylor Goetz
   apacheId: ptgoetz
@@ -57,34 +57,43 @@
 - name: Qianhao Zhou (周千昊)
   apacheId: qhzhou
   githubId: qhzhou
-  org: 
+  org: eBay
   role: PMC
 - name: Ted Dunning
   apacheId: tdunning
   githubId: tdunning
-  org: 
+  org: MapR
   role: Champion
   avatar: https://www.mapr.com/sites/default/files/otherpageimages/ted-circle-80.png
 - name: Shaofeng Shi (史少锋) 
   apacheId: shaofengshi
   githubId: shaofengshi
-  org: 
+  org: eBay
   role: PMC
 - name: Xiaodong Duo (朵晓东)
   apacheId: xduo
   githubId: xduo
-  org: 
+  org: Alipay
   role: PMC
 - name: Xu Jiang (蒋旭)
   apacheId: jiangxu
   githubId: jiangxuchina
-  org: 
+  org: Alibaba
   role: PMC
 - name: Yang Li (李扬)
   apacheId: liyang
   githubId: liyang-gmt8
-  org: 
+  org: eBay
   role: PMC
-
+- name: Dayue Gao (高大月)
+  apacheId: 
+  githubId: 
+  org: Meituan
+  role: committer
+- name: Hua Huang (黄桦)
+  apacheId: 
+  githubId: superhua
+  org: MiningLAMP
+  role: committer 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/60a95f3b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
new file mode 100644
index 0000000..4861eea
--- /dev/null
+++ b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
@@ -0,0 +1,44 @@
+---
+layout: post-blog
+title:  Apache Kylin 1.0 (incubating) Release Announcement
+date:   2015-09-06 17:28:00
+author: Luke Han
+categories: blog
+---
+
+The Apache Kylin team is pleased to announce the release of Apache Kylin v1.0 (incubating). Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets.
+
+To download Apache Kylin v1.0 (incubating) visit the [download](http://kylin.incubator.apache.org/download) page.
+
+This is a majority release which brings more stable, robust and well management version, Apache Kylin team resolved about 47 issues including bug fixes, improvements, and few new features.
+
+## Change Highlights
+
+__Kylin Core Improvement__
+
+* Dynamic Data Model has been supported for new added or removed column in data model without rebuild cube from the beginning [KYLIN-867](https://issues.apache.org/jira/browse/KYLIN-867)
+* Upgraded Apache Calcite to 1.3 for more bug fixes and new SQL functions [KYLIN-881](https://issues.apache.org/jira/browse/KYLIN-881) 
+* Cleanup job enhanced to make sure there’s no garbage files left in OS and HDFS/HBase after job build [KYLIN-926](https://issues.apache.org/jira/browse/KYLIN-926)
+* Added setting option for Hive intermediate tables created by Kylin [KYLIN-883](https://issues.apache.org/jira/browse/KYLIN-883) 
+* HBase Corprocessor enhanced to imrpove query performance [KYLIN-857](https://issues.apache.org/jira/browse/KYLIN-857)
+* Kylin System Dashboard for usage, storage, performance [KYLIN-792](https://issues.apache.org/jira/browse/KYLIN-792)
+
+__Main Bug Fixes__
+
+* Can’t get source record size, especially when using other Hadoop distribution rather than HDP, like Cloudera and MapR [KYLIN-404](https://issues.apache.org/jira/browse/KYLIN-404)
+* Front-end cache cleanup issue [KYLIN-757](https://issues.apache.org/jira/browse/KYLIN-757)
+* Useless hive intermediate tables and HBase tables will be dropped after cube build/merge [KYLIN-805](https://issues.apache.org/jira/browse/KYLIN-805)
+* Support More than one HDFS files of lookup table, especially for large lookup table [KYLIN-889](https://issues.apache.org/jira/browse/KYLIN-889)
+* JDBC driver bug fixes [KYLIN-945](https://issues.apache.org/jira/browse/KYLIN-945), [KYLIN-626](https://issues.apache.org/jira/browse/KYLIN-626)
+* UI Bug fixes [KYLIN-946](https://issues.apache.org/jira/browse/KYLIN-946), [KYLIN-935](https://issues.apache.org/jira/browse/KYLIN-935)
+
+__Zeppelin Integration__
+
+[Apache Zeppelin](http://zeppelin.incubator.apache.org/) is a web-based notebook that enables interactive data analytics. The Apache Kylin team has contributed Kylin Interpreter which enable Zeppelin interactive with Kylin from notebook using ANSI SQL, this interpreter could be found from Zeppelin master code repo [here](https://github.com/apache/incubator-zeppelin/tree/master/kylin).
+
+__Upgrade__
+
+We recommend to upgrade to this version from v0.7.x or even more early version for better performance, stablility and more clear one (most of the intermediate files will be cleaned up automatically). Also to keep up to date with community with latest features and supports.
+Any issue or question during upgrade, please send to Apache Kylin dev mailing list: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
+
+_Great thanks to everyone who contributed!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/60a95f3b/website/community/index.md
----------------------------------------------------------------------
diff --git a/website/community/index.md b/website/community/index.md
index e7f7e8b..c573e08 100644
--- a/website/community/index.md
+++ b/website/community/index.md
@@ -13,7 +13,7 @@ These are the mailing lists that have been established for this project. For eac
 | User Mailing List | [Subscribe](mailto:user-subscribe@kylin.incubator.apache.org) | [Unsubscribe](mailto:user-unsubscribe@kylin.incubator.apache.org) | [Post](mailto:user@kylin.incubator.apache.org) | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-user/) |
 | Developers Mailing List | [Subscribe](mailto:dev-subscribe@kylin.incubator.apache.org) | [Unsubscribe](mailto:dev-unsubscribe@kylin.incubator.apache.org) | [Post](mailto:dev@kylin.incubator.apache.org) | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-dev/) |
 | Issues Mailing List | [Subscribe](mailto:issues-subscribe@kylin.incubator.apache.org) | [Unsubscribe](mailto:issues-unsubscribe@kylin.incubator.apache.org) | N/A | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-issues/) |
-| Commits Mailing List | [Subscribe](mailto:commits-subscribe@kylin.incubator.apache.org) | [Unsubscribe](mailto:commits-unsubscribe@kylin.incubator.apache.org) | [Post](mailto:commits@kylin.incubator.apache.org) | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-commits/) |
+| Commits Mailing List | [Subscribe](mailto:commits-subscribe@kylin.incubator.apache.org) | [Unsubscribe](mailto:commits-unsubscribe@kylin.incubator.apache.org) | N/A | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-commits/) |
 
 ### Mailing List Archives
 For convenience, there's a forum style mailing list archives which not part of offical Apache archives:
@@ -33,6 +33,21 @@ A successful project requires many people to play many roles. Some members write
 {% for c in site.data.contributors %}  | {{ c.name }} | <a href="http://people.apache.org/committer-index#{{ c.apacheId }}">{{ c.apacheId }}</a> | <a href="http://github.com/{{ c.githubId }}"><img width="48" src="{% unless c.avatar %}http://github.com/{{ c.githubId }}.png{% else %}{{ c.avatar }}{% endunless %}"></a> |  {{ c.role }} |
 {% endfor %}
 
+### Contributors
+Contributors has commited code could be found [here](https://github.com/apache/incubator-kylin/graphs/contributors).
+__Other contributors__
+
+| Name  | Github    |   |
+|:----- |:----------|:------|
+|Rui Feng | [fengrui129](https://github.com/fengrui129) | Website Design, Kylin Logo|
+|Luffy Xiao | [luffy-xiao](http://github.com/luffy-xiao) | Kylin Web application, REST service |
+|Kejia Wang |  [Kejia-Wang](https://github.com/Kejia-Wang)  | Web aplication, Website|
+|Yue Yang |  | Web aplication UI design |
+
+### Credits
+
+* Thanks [eBay Inc.](https://www.ebayinc.com/) to donated this project to open source community, first announement at [eBay Techblog](http://www.ebaytechblog.com/2014/10/20/announcing-kylin-extreme-olap-engine-for-big-data/).  
+* Thanks [JetBrains](https://www.jetbrains.com/) for providing us a free license of [IntelliJ IDEA](https://www.jetbrains.com/idea/).
 
 
 


[07/50] [abbrv] incubator-kylin git commit: KYLIN-985 Fix support of AVG()

Posted by li...@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/cc23122d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/cc23122d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/cc23122d

Branch: refs/heads/master
Commit: cc23122d2e00f96fbf623449a6b0a0b4e5f8fde8
Parents: e839bc9
Author: Yang Li <li...@apache.org>
Authored: Fri Sep 4 20:39:56 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 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/cc23122d/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 1b74040..aaedab4 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/cc23122d/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 


[06/50] [abbrv] incubator-kylin git commit: KYLIN-1001 Kylin generates wrong HDFS path in creating intermediate table

Posted by li...@apache.org.
KYLIN-1001 Kylin generates wrong HDFS path in creating intermediate table


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

Branch: refs/heads/master
Commit: 412a150488e0130209d5dfcd637457f955dbdc6b
Parents: cc23122
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 10:48:32 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/AbstractJobBuilder.java   |  2 +-
 job/src/main/java/org/apache/kylin/job/JobInstance.java |  2 +-
 .../apache/kylin/job/hadoop/cube/StorageCleanupJob.java |  2 +-
 .../org/apache/kylin/rest/service/PerformService.java   | 12 ++++++------
 4 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/412a1504/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
index 96b87c5..ffbfe98 100644
--- a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
@@ -94,6 +94,6 @@ public abstract class AbstractJobBuilder {
     }
 
     protected String getJobWorkingDir(String uuid) {
-        return engineConfig.getHdfsWorkingDirectory() + "/" + JOB_WORKING_DIR_PREFIX + uuid;
+        return engineConfig.getHdfsWorkingDirectory() + JOB_WORKING_DIR_PREFIX + uuid;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/412a1504/job/src/main/java/org/apache/kylin/job/JobInstance.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/JobInstance.java b/job/src/main/java/org/apache/kylin/job/JobInstance.java
index 6a07aca..82d4753 100644
--- a/job/src/main/java/org/apache/kylin/job/JobInstance.java
+++ b/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/412a1504/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 fda1275..46f0849 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
@@ -169,7 +169,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/412a1504/server/src/main/java/org/apache/kylin/rest/service/PerformService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/PerformService.java b/server/src/main/java/org/apache/kylin/rest/service/PerformService.java
index 9df9d4a..8d6ed7b 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/PerformService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/PerformService.java
@@ -45,7 +45,7 @@ public class PerformService extends BasicService {
      * @return all query user
      */
     public List<String[]> getTotalQueryUser() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/total_query_user.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/total_query_user.csv";
         List<String[]> res = readHdfsFile(filePath);
         logger.info("Total Query User:" + res.get(0)[0]);
         return res;
@@ -55,7 +55,7 @@ public class PerformService extends BasicService {
     * @return last 30 daily query num
     */
     public List<String[]> dailyQueryCount() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/last_30_daily_query_count.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/last_30_daily_query_count.csv";
         List<String[]> res = readHdfsFile(filePath);
         return res;
     }
@@ -64,7 +64,7 @@ public class PerformService extends BasicService {
      * @return average query count every day
      */
     public List<String[]> avgDayQuery() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/avg_day_query.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/avg_day_query.csv";
         List<String[]> res = readHdfsFile(filePath);
         logger.info("Avg Day Query:" + res.get(0)[0]);
         return res;
@@ -74,7 +74,7 @@ public class PerformService extends BasicService {
      *@return average latency every day
      */
     public List<String[]> last30DayPercentile() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/last_30_day_90_percentile_latency.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/last_30_day_90_percentile_latency.csv";
         List<String[]> res = readHdfsFile(filePath);
         return res;
     }
@@ -83,7 +83,7 @@ public class PerformService extends BasicService {
      *@return average latency for every cube
      */
     public List<String[]> eachDayPercentile() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/each_day_90_95_percentile_latency.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/each_day_90_95_percentile_latency.csv";
         List<String[]> res = readHdfsFile(filePath);
         return res;
     }
@@ -92,7 +92,7 @@ public class PerformService extends BasicService {
      *@return average latency for every cube
      */
     public List<String[]> projectPercentile() throws IOException {
-        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "/performance/metadata/project_90_95_percentile_latency.csv";
+        String filePath = KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory() + "performance/metadata/project_90_95_percentile_latency.csv";
         List<String[]> res = readHdfsFile(filePath);
         return res;
     }


[21/50] [abbrv] incubator-kylin git commit: KYLIN-983 Query sql offset keyword bug

Posted by li...@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/66bd59e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/66bd59e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/66bd59e7

Branch: refs/heads/master
Commit: 66bd59e765f1d6a560ab948aa5d0ff3af918c7d7
Parents: a47fc5d
Author: wangxiaoyu8 <wa...@jd.com>
Authored: Mon Sep 7 09:39:08 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Sep 7 10:49:24 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/query/relnode/OLAPLimitRel.java |  5 +++++
 .../apache/kylin/query/test/KylinQueryTest.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 +
 .../org/apache/kylin/storage/StorageContext.java |  9 +++++++++
 .../hbase/SerializedHBaseTupleIterator.java      |  2 +-
 14 files changed, 112 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/66bd59e7/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 c4833c4..caf9e98 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/66bd59e7/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
index 0048e53..84f1042 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinQueryTest.java
@@ -170,9 +170,8 @@ public class KylinQueryTest 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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/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/66bd59e7/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index 9f5011e..ae2436d 100644
--- a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -44,6 +44,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;
@@ -112,6 +113,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/66bd59e7/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
index a204d62..a115753 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
@@ -89,7 +89,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     @Override
     public boolean hasNext() {
         // 1. check limit
-        if (context.isLimitEnabled() && scanCount >= context.getLimit()) {
+        if (context.isLimitEnabled() && scanCount >= context.getLimit() + context.getOffset()) {
             return false;
         }
         // 2. check partial result


[50/50] [abbrv] incubator-kylin git commit: Merge branch 'master' into 1.x-staging

Posted by li...@apache.org.
Merge branch 'master' into 1.x-staging


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

Branch: refs/heads/master
Commit: 43f119741ac07d632867dbbdfd0c5de688832202
Parents: faf401a 575eee0
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 19 07:46:05 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 19 07:46:05 2015 +0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[19/50] [abbrv] incubator-kylin git commit: KYLIN-965 keep default setting for small/medium/large cubes

Posted by li...@apache.org.
KYLIN-965 keep default setting for small/medium/large cubes

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

Branch: refs/heads/master
Commit: c9f819e1de326ef343e4d576535d9080ddd518a2
Parents: 50e0875
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 09:03:35 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 09:03:35 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/common/KylinConfig.java | 13 ++++++++++++-
 1 file changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c9f819e1/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 59337d8..5566fa2 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -658,7 +658,18 @@ public class KylinConfig {
     
     public int getHBaseRegionSizeGB(String capacity) {
         String key = HBASE_REGION_SIZE + "." + capacity.toLowerCase();
-        return Integer.valueOf(getOptional(key, "10"));
+
+        int cut = 20;
+        if (kylinConfig.containsKey(key)) {
+            cut = kylinConfig.getInt(key);
+        } else if ("small".equalsIgnoreCase(capacity)) {
+            cut = 10;
+        } else if ("medium".equalsIgnoreCase(capacity)) {
+            cut = 20;
+        } else if ("large".equalsIgnoreCase(capacity)) {
+            cut = 100;
+        }
+        return cut;
     }
     
     public int getHBaseMaxRegionCount() {


[24/50] [abbrv] incubator-kylin git commit: update website for 1.0 release

Posted by li...@apache.org.
update website for 1.0 release


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

Branch: refs/heads/master
Commit: 33f1a8b82cd87ca9d2f59811743b5e3f1df6593a
Parents: e73c822
Author: Luke Han <lu...@apache.org>
Authored: Mon Sep 7 15:27:06 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Mon Sep 7 15:27:06 2015 +0800

----------------------------------------------------------------------
 website/_docs/release_notes.md | 34 +++++++++++++++--------------
 website/download/index.cn.md   | 41 +++++++++++++----------------------
 website/download/index.md      | 43 +++++++++++++++----------------------
 website/index.cn.md            |  2 +-
 website/index.md               |  2 +-
 5 files changed, 52 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/33f1a8b8/website/_docs/release_notes.md
----------------------------------------------------------------------
diff --git a/website/_docs/release_notes.md b/website/_docs/release_notes.md
index 1e9c072..c100ae1 100644
--- a/website/_docs/release_notes.md
+++ b/website/_docs/release_notes.md
@@ -7,17 +7,15 @@ version: v1.0
 since: v0.7.1
 ---
 
-# Apache Kylin Release Notes
-
 To download latest release, please visit: [http://kylin.incubator.apache.org/download/](http://kylin.incubator.apache.org/download/), there are source code package, binary package, ODBC driver and installation guide avaliable.
 
-Any problem or issue, please send to Apache Kylin mailing list:   
-* User relative: [user@kylin.incubator.apache.org](mailto:user@kylin.incubator.apache.org)  
-* Development relative: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
+Any problem or issue, please report to Apache Kylin JIRA project: [https://issues.apache.org/jira/browse/KYLIN](https://issues.apache.org/jira/browse/KYLIN)
 
-Or, report to Apache Kylin JIRA project: [https://issues.apache.org/jira/browse/KYLIN](https://issues.apache.org/jira/browse/KYLIN)
+or send to Apache Kylin mailing list:   
+* User relative: [user@kylin.incubator.apache.org](mailto:user@kylin.incubator.apache.org)
+* Development relative: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
 
-## v1.0-incubating - releasing
+## v1.0-incubating - 2015-09-06
 _Tag:_ [kylin-1.0-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.0-incubating)
 
 __New Feature__
@@ -31,16 +29,24 @@ __Bug__
     * [KYLIN-751] - Max on negative double values is not working
     * [KYLIN-757] - Cache wasn't flushed in cluster mode
     * [KYLIN-780] - Upgrade Calcite to 1.0
+    * [KYLIN-805] - Drop useless Hive intermediate table and HBase tables in the last step of cube build/merge
     * [KYLIN-889] - Support more than one HDFS files of lookup table
     * [KYLIN-897] - Update CubeMigrationCLI to copy data model info
+    * [KYLIN-898] - "CUBOID_CACHE" in Cuboid.java never flushes
+    * [KYLIN-911] - NEW segments not DELETED when cancel BuildAndMerge Job
+    * [KYLIN-912] - $KYLIN_HOME/tomcat/temp folder takes much disk space after long run
     * [KYLIN-914] - Scripts shebang should use /bin/bash
+    * [KYLIN-915] - appendDBName in CubeMetadataUpgrade will return null
     * [KYLIN-921] - Dimension with all nulls cause BuildDimensionDictionary failed due to FileNotFoundException
+    * [KYLIN-923] - FetcherRunner will never run again if encountered exception during running
     * [KYLIN-929] - can not sort cubes by [Source Records] at cubes list page
     * [KYLIN-934] - Negative number in SUM result and Kylin results not matching exactly Hive results
     * [KYLIN-935] - always loading when try to view the log of the sub-step of cube build job
     * [KYLIN-936] - can not see job step log 
     * [KYLIN-940] - NPE when close the null resouce
+    * [KYLIN-945] - Kylin JDBC - Get Connection from DataSource results in NullPointerException
     * [KYLIN-946] - [UI] refresh page show no results when Project selected as [--Select All--]
+    * [KYLIN-949] - Query cache doesn't work properly for prepareStatement queries
 
 __Improvement__
 
@@ -52,8 +58,10 @@ __Improvement__
     * [KYLIN-844] - add backdoor toggles to control query behavior 
     * [KYLIN-857] - backport coprocessor improvement in 0.8 to 0.7
     * [KYLIN-866] - Confirm with user when he selects empty segments to merge
+    * [KYLIN-867] - Hybrid model for multiple realizations/cubes
     * [KYLIN-880] -  Kylin should change the default folder from /tmp to user configurable destination
     * [KYLIN-881] - Upgrade Calcite to 1.3.0
+    * [KYLIN-883] - Using configurable option for Hive intermediate tables created by Kylin job
     * [KYLIN-893] - Remove the dependency on quartz and metrics
     * [KYLIN-922] - Enforce same code style for both intellij and eclipse user
     * [KYLIN-926] - Make sure Kylin leaves no garbage files in local OS and HDFS/HBASE
@@ -63,9 +71,9 @@ __Improvement__
 __Task__
 
     * [KYLIN-884] - Restructure docs and website
-    
-__Sub-task__
-
+    * [KYLIN-907] - Improve Kylin community development experience
+    * [KYLIN-954] - Release v1.0 (formerly v0.7.3)
+    * [KYLIN-863] - create empty segment when there is no data in one single streaming batch
     * [KYLIN-908] - Help community developer to setup develop/debug environment
     * [KYLIN-931] - Port KYLIN-921 to 0.8 branch
 
@@ -123,9 +131,6 @@ __Task__
 
     * [KYLIN-7] - Enable maven checkstyle plugin
     * [KYLIN-885] - Release v0.7.2
-
-__Sub-task__
-
     * [KYLIN-812] - Upgrade to Calcite 0.9.2
 
 ## v0.7.1-incubating (First Apache Release) - 2015-06-10  
@@ -161,9 +166,6 @@ __Task__
     * [KYLIN-624] - Add dimension and metric info into DataModel
     * [KYLIN-650] - Move all document from github wiki to code repository (using md file)
     * [KYLIN-669] - Release v0.7.1 as first apache release
-
-__Sub-task__
-
     * [KYLIN-670] - Update pom with "incubating" in version number
     * [KYLIN-737] - Generate and sign release package for review and vote
     * [KYLIN-795] - Release after success vote

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/33f1a8b8/website/download/index.cn.md
----------------------------------------------------------------------
diff --git a/website/download/index.cn.md b/website/download/index.cn.md
index 992500a..e372fdc 100644
--- a/website/download/index.cn.md
+++ b/website/download/index.cn.md
@@ -6,36 +6,25 @@ title: 下载
 __最新发布 (源代码)__  
 最新发布的Apache Kylin可以从ASF网站下载::
 
-The latest release of Apache Kylin can be downloaded from the ASF:
-  * [Apache-Kylin-0.7.2-Incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-0.7.2-incubating)
-  * [Release Notes](../docs/release_notes.html)
-  * Git Tag: [kylin-0.7.2-incubating](https://github.com/apache/incubator-kylin/tree/kylin-0.7.2-incubating)
-  * Git Commit: [f7a28422645bc209b2c38bb05b24a340444f9066](https://github.com/apache/incubator-kylin/commit/f7a28422645bc209b2c38bb05b24a340444f9066)
+* [Apache Kylin v1.0-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.0-incubating/)
+* [发布日志](http://kylin.incubator.apache.org/docs/release_notes.html)
+* Git 标签: [kylin-1.0-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.0-incubating)
+* Git Commit: [ef067de4ae43303c015118ad39a9aba7172bc250](https://github.com/apache/incubator-kylin/commit/ef067de4ae43303c015118ad39a9aba7172bc250)
 
+__二进制包__
+为方便实用,我们也提供预打包的二进制安装包:
 
-__之前的发布__  
-  * [Apache-Kylin-0.7.1-Incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-0.7.1-incubating)
-  * [Release Notes](http://kylin.incubator.apache.org/docs/release_notes.html)
-  * Git Tag: [kylin-0.7.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-0.7.1-incubating)
-  * Git Commit: [6a7d07dd79ffc00ba1ece330010275b6f0715de8](https://github.com/apache/incubator-kylin/commit/6a7d07dd79ffc00ba1ece330010275b6f0715de8)
+  * Apache Kylin v1.0-indubating [二进制包](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.0-incubating/apache-kylin-1.0-incubating-bin.tar.gz)
+  * [安装帮助](http://kylin.incubator.apache.org/docs/install)
 
-__二进制包__  
-预打包的二进制包以方便部署:
+__构建二进制包__
+可以从各个版本或当前最新的开发分支中生成二进制包,请参考这篇[帮助文档](http://kylin.incubator.apache.org/docs/howto/howto_package.html)
 
-  * [Kylin 二进制包](kylin-0.7.2-incubating.tar.gz)
-  * [Installation Guide](../docs/install)
-  * 如何从源文件生成二进制包,请参考这篇[教程](http://kylin.incubator.apache.org/docs/howto/howto_package.html)
+__以前的版本__
+Apache Kylin的旧版本可以从[归档](https://dist.apache.org/repos/dist/release/incubator/kylin/)中下载。
 
 __ODBC 驱动__  
-Apache Kylin ODBC驱动:
-
-  * [Kylin ODBC Driver](KylinODBCDriver.zip)
-
-__开发版本二进制包(v0.7.3)__
-
-最新的开发版二进制包:
-
-  * [kylin-v0.7.3-incubating-SNAPSHOT](kylin-0.7.3-incubating-SNAPSHOT.tar.gz)
-  * 更新日期: 2015-08-03
-  * Commit ID: a880dd4fb6e4443d2bcad0d57db3ae09bac65c4d
+Kylin ODBC 驱动要求首先安装[Microsoft Visual C++ 2012 Redistributable]()。 
+Kylin ODBC 驱动可以从这里下载:
 
+  * [Kylin ODBC 驱动](http://kylin.incubator.apache.org/download/KylinODBCDriver.zip)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/33f1a8b8/website/download/index.md
----------------------------------------------------------------------
diff --git a/website/download/index.md b/website/download/index.md
index aab1e22..e7db862 100644
--- a/website/download/index.md
+++ b/website/download/index.md
@@ -6,37 +6,28 @@ permalink: /download/index.html
 
 __Latest Release (Source Code)__  
 The latest release of Apache Kylin can be downloaded from the ASF:
-  * [Apache-Kylin-0.7.2-Incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-0.7.2-incubating)
-  * [Release Notes](../docs/release_notes.html)
-  * Git Tag: [kylin-0.7.2-incubating](https://github.com/apache/incubator-kylin/tree/kylin-0.7.2-incubating)
-  * Git Commit: [f7a28422645bc209b2c38bb05b24a340444f9066](https://github.com/apache/incubator-kylin/commit/f7a28422645bc209b2c38bb05b24a340444f9066)
 
-__Previous Release__  
-  * [Apache-Kylin-0.7.1-Incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-0.7.1-incubating)
-  * [Release Notes](../docs/release_notes.html)
-  * Git Tag: [kylin-0.7.1-incubating](https://github.com/apache/incubator-kylin/tree/kylin-0.7.1-incubating)
-  * Git Commit: [6a7d07dd79ffc00ba1ece330010275b6f0715de8](https://github.com/apache/incubator-kylin/commit/6a7d07dd79ffc00ba1ece330010275b6f0715de8)
-
-__Binary Package__  
-For convenience, there's binary package also available:
-
-  * [Kylin Binary Package for Deployment](kylin-0.7.2-incubating.tar.gz)
-  * [Installation Guide](../docs/install)
-  * Generate binary package from source, please refer to this [guide](../docs/howto/howto_package.html)
-    
-__ODBC Driver__  
-Kylin ODBC requires you first install Microsoft Visual C++ 2012 Redistributable first, it can be downloaded at:http://www.microsoft.com/en-us/download/details.aspx?id=30679
+* [Apache Kylin v1.0-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.0-incubating/)
+* [Release Notes](http://kylin.incubator.apache.org/docs/release_notes.html)
+* Git Tag: [kylin-1.0-incubating](https://github.com/apache/incubator-kylin/tree/kylin-1.0-incubating)
+* Git Commit: [ef067de4ae43303c015118ad39a9aba7172bc250](https://github.com/apache/incubator-kylin/commit/ef067de4ae43303c015118ad39a9aba7172bc250)
 
-And Kylin ODBC Driver could be find here:
+__Binary Package__
+For convenience, there’s binary package also available: 
 
-  * [Kylin ODBC Driver](KylinODBCDriver.zip)
+* Apache Kylin v1.0-indubating [Binary Package](https://dist.apache.org/repos/dist/release/incubator/kylin/apache-kylin-1.0-incubating/apache-kylin-1.0-incubating-bin.tar.gz)
+* [Installation Guide](http://kylin.incubator.apache.org/docs/install)
 
-__Development Version (v0.7.3)__  
-Here's latest development version binary package:
+__Build Binary Package__
+To build binary package from any version even latest development branch, please refer to this [guide](http://kylin.incubator.apache.org/docs/howto/howto_package.html)
 
-  * [kylin-v1.0-incubating-SNAPSHOT](kylin-1.0-incubating-SNAPSHOT-bin.tar.gz)
-  * Updated Date: 2015-08-14
-  * Commit ID: b449590cc5c6aaa3eea9d19f922473bff6fce4b0
+__Previous Release__  
+ Older releases may be found in the [archives](https://dist.apache.org/repos/dist/release/incubator/kylin/).
+    
+__ODBC Driver__  
+Kylin ODBC driver requires [Microsoft Visual C++ 2012 Redistributable](http://www.microsoft.com/en-us/download/details.aspx?id=30679) installed first. 
+And Kylin ODBC Driver could be downloaded here: 
 
+* [Kylin ODBC Driver](http://kylin.incubator.apache.org/download/KylinODBCDriver.zip)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/33f1a8b8/website/index.cn.md
----------------------------------------------------------------------
diff --git a/website/index.cn.md b/website/index.cn.md
index b0d19ee..e840db9 100644
--- a/website/index.cn.md
+++ b/website/index.cn.md
@@ -14,7 +14,7 @@ title: 首页
           <div class="row" style="margin-top:-20px;">
             <div class="col-sm-12 col-md-12">
               <p class="title_text"> Kylin 于2014年11月25日被接受会Apache孵化器项目</p>
-              <p class="title_text"> Apache Kylin 是由eBay开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据</p>
+              <p class="title_text"> Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。</p>
               <img id="diagram" src="{{ "/assets/images/kylin_diagram.png"| prepend: site.baseurl }}"> </div>
           </div>
         </div>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/33f1a8b8/website/index.md
----------------------------------------------------------------------
diff --git a/website/index.md b/website/index.md
index f18a7c3..8efac9a 100644
--- a/website/index.md
+++ b/website/index.md
@@ -14,7 +14,7 @@ title: Home
           <div class="row" style="margin-top:-20px;">
             <div class="col-sm-12 col-md-12">
               <p class="title_text"> Kylin has been accepted as Apache Incubator Project on Nov 25, 2014.</p>
-              <p class="title_text"> Apache Kylin is an open source Distributed Analytics Engine from eBay Inc. that provides SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets</p>
+              <p class="title_text"> Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.</p>
               <img id="diagram" src="assets/images/kylin_diagram.png"> </div>
           </div>
         </div>


[43/50] [abbrv] incubator-kylin git commit: publish kylin on spark blog

Posted by li...@apache.org.
publish kylin on spark blog


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

Branch: refs/heads/master
Commit: f929892dc9a190d2f9a728afa1b1f054ea91203d
Parents: df00623
Author: Luke Han <lu...@apache.org>
Authored: Tue Sep 15 17:54:56 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Tue Sep 15 17:55:10 2015 +0800

----------------------------------------------------------------------
 website/_docs/howto/howto_backup_hbase.md                 |  2 +-
 website/_docs/howto/howto_backup_metadata.md              |  2 +-
 website/_docs/howto/howto_build_cube_with_restapi.md      |  2 +-
 website/_docs/howto/howto_use_restapi.md                  |  2 +-
 website/_docs/howto/howto_use_restapi_in_js.md            |  2 +-
 website/_posts/blog/2015-09-06-release-v1.0-incubating.md | 10 +++++-----
 website/_posts/blog/2015-09-09-fast-cubing-on-spark.md    |  9 +++++++--
 7 files changed, 17 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_docs/howto/howto_backup_hbase.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_backup_hbase.md b/website/_docs/howto/howto_backup_hbase.md
index cc43f27..8714132 100644
--- a/website/_docs/howto/howto_backup_hbase.md
+++ b/website/_docs/howto/howto_backup_hbase.md
@@ -3,7 +3,7 @@ layout: docs
 title:  How to Clean/Backup HBase Tables
 categories: howto
 permalink: /docs/howto/howto_backup_hbase.html
-version: v0.7.2
+version: v1.0
 since: v0.7.1
 ---
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_docs/howto/howto_backup_metadata.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_backup_metadata.md b/website/_docs/howto/howto_backup_metadata.md
index c1eb760..1934a60 100644
--- a/website/_docs/howto/howto_backup_metadata.md
+++ b/website/_docs/howto/howto_backup_metadata.md
@@ -3,7 +3,7 @@ layout: docs
 title:  How to Backup Metadata
 categories: howto
 permalink: /docs/howto/howto_backup_metadata.html
-version: v0.8
+version: v1.0
 since: v0.7.1
 ---
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_docs/howto/howto_build_cube_with_restapi.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_build_cube_with_restapi.md b/website/_docs/howto/howto_build_cube_with_restapi.md
index 0fcdeac..ee4759c 100644
--- a/website/_docs/howto/howto_build_cube_with_restapi.md
+++ b/website/_docs/howto/howto_build_cube_with_restapi.md
@@ -3,7 +3,7 @@ layout: docs
 title:  How to Build Cube with Restful API
 categories: howto
 permalink: /docs/howto/howto_build_cube_with_restapi.html
-version: v0.8
+version: v1.0
 since: v0.7.1
 ---
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_docs/howto/howto_use_restapi.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_use_restapi.md b/website/_docs/howto/howto_use_restapi.md
index cb19556..667334d 100644
--- a/website/_docs/howto/howto_use_restapi.md
+++ b/website/_docs/howto/howto_use_restapi.md
@@ -3,7 +3,7 @@ layout: docs
 title:  How to Use Restful API
 categories: howto
 permalink: /docs/howto/howto_use_restapi.html
-version: v0.8
+version: v1.0
 since: v0.7.1
 ---
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_docs/howto/howto_use_restapi_in_js.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_use_restapi_in_js.md b/website/_docs/howto/howto_use_restapi_in_js.md
index 45c8974..d9fa273 100644
--- a/website/_docs/howto/howto_use_restapi_in_js.md
+++ b/website/_docs/howto/howto_use_restapi_in_js.md
@@ -3,7 +3,7 @@ layout: docs
 title:  How to Use Restful API in Javascript
 categories: howto
 permalink: /docs/howto/howto_use_restapi_in_js.html
-version: v0.8
+version: v1.0
 since: v0.7.1
 ---
 Kypin securicty authorization is based on basic access authorization, so when you want to use API in your javascript,you need to involve the authorization info in http headers.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
index 4861eea..085c7a1 100644
--- a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
+++ b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
@@ -10,17 +10,17 @@ The Apache Kylin team is pleased to announce the release of Apache Kylin v1.0 (i
 
 To download Apache Kylin v1.0 (incubating) visit the [download](http://kylin.incubator.apache.org/download) page.
 
-This is a majority release which brings more stable, robust and well management version, Apache Kylin team resolved about 47 issues including bug fixes, improvements, and few new features.
+This is a major release which brings more stable, robust and well management version, Apache Kylin team resolved about 47 issues including bug fixes, improvements, and few new features.
 
 ## Change Highlights
 
 __Kylin Core Improvement__
 
-* Dynamic Data Model has been supported for new added or removed column in data model without rebuild cube from the beginning [KYLIN-867](https://issues.apache.org/jira/browse/KYLIN-867)
+* Dynamic Data Model has been added to supporting adding or removing column in data model without rebuild cube from the beginning [KYLIN-867](https://issues.apache.org/jira/browse/KYLIN-867)
 * Upgraded Apache Calcite to 1.3 for more bug fixes and new SQL functions [KYLIN-881](https://issues.apache.org/jira/browse/KYLIN-881) 
 * Cleanup job enhanced to make sure there’s no garbage files left in OS and HDFS/HBase after job build [KYLIN-926](https://issues.apache.org/jira/browse/KYLIN-926)
 * Added setting option for Hive intermediate tables created by Kylin [KYLIN-883](https://issues.apache.org/jira/browse/KYLIN-883) 
-* HBase Corprocessor enhanced to imrpove query performance [KYLIN-857](https://issues.apache.org/jira/browse/KYLIN-857)
+* HBase coprocessor enhanced to imrpove query performance [KYLIN-857](https://issues.apache.org/jira/browse/KYLIN-857)
 * Kylin System Dashboard for usage, storage, performance [KYLIN-792](https://issues.apache.org/jira/browse/KYLIN-792)
 
 __Main Bug Fixes__
@@ -34,11 +34,11 @@ __Main Bug Fixes__
 
 __Zeppelin Integration__
 
-[Apache Zeppelin](http://zeppelin.incubator.apache.org/) is a web-based notebook that enables interactive data analytics. The Apache Kylin team has contributed Kylin Interpreter which enable Zeppelin interactive with Kylin from notebook using ANSI SQL, this interpreter could be found from Zeppelin master code repo [here](https://github.com/apache/incubator-zeppelin/tree/master/kylin).
+[Apache Zeppelin](http://zeppelin.incubator.apache.org/) is a web-based notebook that enables interactive data analytics. The Apache Kylin team has contributed Kylin Interpreter which enables Zeppelin interaction with Kylin from notebook using ANSI SQL, this interpreter could be found from Zeppelin master code repo [here](https://github.com/apache/incubator-zeppelin/tree/master/kylin).
 
 __Upgrade__
 
-We recommend to upgrade to this version from v0.7.x or even more early version for better performance, stablility and more clear one (most of the intermediate files will be cleaned up automatically). Also to keep up to date with community with latest features and supports.
+We recommend to upgrade to this version from v0.7.x or even more early version for better performance, stablility and clear one (most of the intermediate files will be cleaned up automatically). Also to keep up to date with community with latest features and supports.
 Any issue or question during upgrade, please send to Apache Kylin dev mailing list: [dev@kylin.incubator.apache.org](mailto:dev@kylin.incubator.apache.org)
 
 _Great thanks to everyone who contributed!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f929892d/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md b/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
index fd9e56d..10ee46a 100644
--- a/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
+++ b/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
@@ -1,5 +1,10 @@
-# Fast Cubing on Spark in Apache Kylin
-
+---
+layout: post-blog
+title:  Fast Cubing on Spark in Apache Kylin
+date:   2015-09-09 15:28:00
+author: Qianhao Zhou
+categories: blog
+---
 
 ## Preparation
 


[04/50] [abbrv] incubator-kylin git commit: Clean the dependency versions in monitor/pom.xml

Posted by li...@apache.org.
Clean the dependency versions in monitor/pom.xml


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

Branch: refs/heads/master
Commit: 0eb83834b20c7e66fed3ebf14d2ae786d09e3efc
Parents: 154e600
Author: shaofengshi <sh...@apache.org>
Authored: Tue Sep 1 13:36:25 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:37:59 2015 +0800

----------------------------------------------------------------------
 monitor/pom.xml | 14 --------------
 pom.xml         |  7 ++++++-
 server/pom.xml  |  1 -
 3 files changed, 6 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0eb83834/monitor/pom.xml
----------------------------------------------------------------------
diff --git a/monitor/pom.xml b/monitor/pom.xml
index 4a3babf..566c873 100644
--- a/monitor/pom.xml
+++ b/monitor/pom.xml
@@ -46,55 +46,46 @@
         <dependency>
             <groupId>log4j</groupId>
             <artifactId>log4j</artifactId>
-            <version>${log4j.version}</version>
         </dependency>
         <dependency>
             <groupId>net.sf.opencsv</groupId>
             <artifactId>opencsv</artifactId>
-            <version>2.3</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-hdfs</artifactId>
-            <version>${hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-auth</artifactId>
-            <version>${hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
-            <version>${hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <!--hbase dependency-->
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-hadoop2-compat</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-common</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-server</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
 
@@ -102,32 +93,27 @@
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-hadoop2-compat</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-common</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-server</artifactId>
-            <version>${hbase-hadoop2.version}</version>
             <scope>provided</scope>
         </dependency>
         <!-- Hive dependencies -->
         <dependency>
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-jdbc</artifactId>
-            <version>${hive.version}</version>
             <scope>provided</scope>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0eb83834/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 23eb729..16786d4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
         <compress-lzf.version>1.0.3</compress-lzf.version>
         <extendedset.version>1.3.4</extendedset.version>
         <jetty.version>9.2.7.v20150116</jetty.version>
+        <opencsv.version>2.3</opencsv.version>
 
         <!-- REST Service -->
         <spring.framework.version>3.1.2.RELEASE</spring.framework.version>
@@ -390,7 +391,11 @@
                 <artifactId>curator-recipes</artifactId>
                 <version>${curator.version}</version>
             </dependency>
-
+            <dependency>
+                <groupId>net.sf.opencsv</groupId>
+                <artifactId>opencsv</artifactId>
+                <version>${opencsv.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/0eb83834/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index f4c7e28..adeb5ba 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -364,7 +364,6 @@
         <dependency>
             <groupId>net.sf.opencsv</groupId>
             <artifactId>opencsv</artifactId>
-            <version>2.3</version>
         </dependency>
 
         <!-- Tomcat Env -->


[48/50] [abbrv] incubator-kylin git commit: KYLIN-971, set default value for [kylin.rest.timezone] in kylin.properties

Posted by li...@apache.org.
KYLIN-971, set default value for [kylin.rest.timezone] in kylin.properties


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

Branch: refs/heads/master
Commit: dab3798c5161f967b5150411d7ef0bfe315e9f20
Parents: fd17282
Author: jiazhong <ji...@ebay.com>
Authored: Thu Sep 17 10:50:18 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Thu Sep 17 10:50:18 2015 +0800

----------------------------------------------------------------------
 conf/kylin.properties                            | 3 +++
 examples/test_case_data/sandbox/kylin.properties | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dab3798c/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 84a1d46..2b8441a 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -20,6 +20,9 @@
 # List of web servers in use, this enables one web server instance to sync up with other servers.
 kylin.rest.servers=localhost:7070
 
+#set display timezone on UI,format like[GMT+N or GMT-N]
+kylin.rest.timezone=GMT-8
+
 # The metadata store in hbase
 kylin.metadata.url=kylin_metadata@hbase
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/dab3798c/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 b3ec4d0..e4b20fe 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -20,6 +20,9 @@
 # List of web servers in use, this enables one web server instance to sync up with other servers.
 kylin.rest.servers=localhost:7070
 
+#set display timezone on UI,format like[GMT+N or GMT-N]
+kylin.rest.timezone=GMT-8
+
 # The metadata store in hbase
 kylin.metadata.url=kylin_metadata@hbase
 


[23/50] [abbrv] incubator-kylin git commit: minor, fix dev_env.md about how to download JS

Posted by li...@apache.org.
minor, fix dev_env.md about how to download JS


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

Branch: refs/heads/master
Commit: e73c8220905eb3acba271f4432bbbada1545f42f
Parents: ff9189c
Author: Li, Yang <ya...@ebay.com>
Authored: Mon Sep 7 13:58:21 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Mon Sep 7 13:59:04 2015 +0800

----------------------------------------------------------------------
 website/_dev/dev_env.md | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e73c8220/website/_dev/dev_env.md
----------------------------------------------------------------------
diff --git a/website/_dev/dev_env.md b/website/_dev/dev_env.md
index fedeceb..20bb69f 100644
--- a/website/_dev/dev_env.md
+++ b/website/_dev/dev_env.md
@@ -75,18 +75,14 @@ Run a end-to-end cube building test, these special test cases will populate some
 It might take a while (maybe one hour), please keep patient.
  
 {% highlight Groff markup %}
-
-mvn test -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -DfailIfNoTests=false -P sandbox
-mvn test -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -DfailIfNoTests=false -P sandbox
-	
+	mvn test -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -DfailIfNoTests=false -P sandbox
+	mvn test -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -DfailIfNoTests=false -P sandbox
 {% endhighlight %}
 	
 Run other tests, the end-to-end cube building test is exclueded
 
 {% highlight Groff markup %}
-
 	mvn test -fae -P sandbox
-
 {% endhighlight %}
 
 ### Launch Kylin Web Server
@@ -94,7 +90,15 @@ Run other tests, the end-to-end cube building test is exclueded
 Copy server/src/main/webapp/WEB-INF to webapp/app/WEB-INF 
 
 {% highlight Groff markup %}
-cp -r server/src/main/webapp/WEB-INF webapp/app/WEB-INF 
+	cp -r server/src/main/webapp/WEB-INF webapp/app/WEB-INF 
+{% endhighlight %}
+
+Download JS for Kylin web GUI. `npm` is part of `Node.js`, please search about how to install it on your OS.
+
+{% highlight Groff markup %}
+	cd webapp
+	npm install -g bower
+	bower --allow-root install
 {% endhighlight %}
 
 In IDE, launch `org.apache.kylin.rest.DebugTomcat` with working directory set to the /server folder. (By default Kylin server will listen on 7070 port; If you want to use another port, please specify it as a parameter when run `DebugTomcat)


[31/50] [abbrv] incubator-kylin git commit: KYLIN-998 small update in StorageCleanupJob

Posted by li...@apache.org.
KYLIN-998 small update in StorageCleanupJob

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

Branch: refs/heads/master
Commit: 277e1524f5be92ba03447ee33010f10b8de5ca75
Parents: 7a2ef17
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 21:41:35 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 21:41:35 2015 +0800

----------------------------------------------------------------------
 .../job/hadoop/cube/StorageCleanupJob.java      | 51 +++++++-------------
 1 file changed, 18 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/277e1524/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 ae684fe..0f6f9cb 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,13 +18,6 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-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;
@@ -57,6 +50,13 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 /**
  * @author ysong1
  */
@@ -225,8 +225,6 @@ public class StorageCleanupJob extends AbstractHadoopJob {
     }
 
     private void cleanUnusedIntermediateHiveTable(Configuration conf) throws IOException {
-        FileSystem fs = FileSystem.get(conf);
-        //JobEngineConfig engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
         int uuidLength = 36;
 
         StringBuilder buf = new StringBuilder();
@@ -243,7 +241,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             e.printStackTrace();
         }
 
-        if(output == null)
+        if (output == null)
             return;
         String outputStr = output.getOutput();
         BufferedReader reader = new BufferedReader(new StringReader(outputStr));
@@ -258,44 +256,31 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
             if (!state.isFinalState()) {
                 workingJobList.add(jobId);
-                log.info("Remove intermediate hive table with job id " + jobId + " with job status " + state);
+                log.info("Exclude intermediate hive table with job id " + jobId + " with job status " + state);
             }
         }
 
         while ((line = reader.readLine()) != null) {
-            if(line.startsWith("kylin_intermediate_")){
-                boolean isNeedDel = true;
+            if (line.startsWith("kylin_intermediate_")) {
+                boolean isNeedDel = false;
                 String uuid = line.substring(line.length() - uuidLength, line.length());
                 uuid = uuid.replace("_", "-");
                 //Check whether it's a hive table in use
-                if(workingJobList.contains(uuid)){
-                    isNeedDel = false;
+                if (allJobs.contains(uuid) && !workingJobList.contains(uuid)) {
+                    isNeedDel = true;
                 }
-                else{
-                    log.info("Hive table with uuid " + uuid + " is in use.");
-                }
-
-                //Check whether the hive table belongs to current Kylin instance
-                String hdfsPath = JobInstance.getJobWorkingDir(uuid, KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory());
-                Path p = new Path(hdfsPath);
 
-                if (fs.exists(p) == false) {
-                    isNeedDel = false;
-                }
-                else{
-                    log.info("Hive table with uuid " + uuid + " belongs to a different Kylin instance.");
-                }
-
-                if(isNeedDel)
+                if (isNeedDel) {
                     allHiveTablesNeedToBeDeleted.add(line);
+                }
             }
         }
-        
+
         if (delete == true) {
             buf.delete(0, buf.length());
             buf.append("hive -e \"");
 
-            for(String delHive : allHiveTablesNeedToBeDeleted){
+            for (String delHive : allHiveTablesNeedToBeDeleted) {
                 buf.append("drop table if exists " + delHive + "; ");
                 log.info("Remove " + delHive + " from hive tables.");
             }
@@ -315,7 +300,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             System.out.println("----------------------------------------------------");
         }
 
-        if(reader != null)
+        if (reader != null)
             reader.close();
     }
 


[38/50] [abbrv] incubator-kylin git commit: KYLIN-998 allow configurable hive database in IIFlattenHiveJob

Posted by li...@apache.org.
KYLIN-998 allow configurable hive database in IIFlattenHiveJob

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

Branch: refs/heads/master
Commit: 44309fed91c130a486ef329b31b9522095ae8dd8
Parents: b338dc3
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 9 16:52:53 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 9 16:52:53 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/44309fed/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java
index 9e60823..e9d8a4a 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IIFlattenHiveJob.java
@@ -59,6 +59,7 @@ public class IIFlattenHiveJob extends AbstractHadoopJob {
             String jobUUID = "00bf87b5-c7b5-4420-a12a-07f6b37b3187";
             JobEngineConfig engineConfig = new JobEngineConfig(config);
             IJoinedFlatTableDesc intermediateTableDesc = new IIJoinedFlatTableDesc(iidesc);
+            final String useDatabaseHql = "USE " + engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + ";";
             String dropTableHql = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc, jobUUID);
             String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, //
                     JobInstance.getJobWorkingDir(jobUUID, engineConfig.getHdfsWorkingDirectory()), jobUUID);
@@ -66,6 +67,7 @@ public class IIFlattenHiveJob extends AbstractHadoopJob {
 
             StringBuffer buf = new StringBuffer();
             buf.append("hive -e \"");
+            buf.append(useDatabaseHql + "\n");
             buf.append(dropTableHql + "\n");
             buf.append(createTableHql + "\n");
             buf.append(insertDataHqls + "\n");


[49/50] [abbrv] incubator-kylin git commit: minor doc update, add Recommended Hadoop Versions in Installation guide

Posted by li...@apache.org.
minor doc update, add Recommended Hadoop Versions in Installation guide


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

Branch: refs/heads/master
Commit: faf401a0ff4fce487336d46ec01eadabba17e505
Parents: dab3798
Author: Li, Yang <ya...@ebay.com>
Authored: Fri Sep 18 15:44:11 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Fri Sep 18 15:44:11 2015 +0800

----------------------------------------------------------------------
 website/_docs/install/index.md | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/faf401a0/website/_docs/install/index.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/index.md b/website/_docs/install/index.md
index 66ad738..e900071 100644
--- a/website/_docs/install/index.md
+++ b/website/_docs/install/index.md
@@ -11,12 +11,13 @@ since: v0.7.1
 
 Kylin requires a properly setup hadoop environment to run. Following are the minimal request to run Kylin, for more detial, please check this reference: [Hadoop Environment](hadoop_env.html).
 
-## Prerequisites on Hadoop
+## Recommended Hadoop Versions
+
+* Hadoop: 2.4 - 2.7
+* Hive: 0.13 - 0.14
+* HBase: 0.98 - 0.99
+* JDK: 1.7+
 
-* Hadoop: 2.4+
-* Hive: 0.13+
-* HBase: 0.98+
-* JDK: 1.7+  
 _Tested with Hortonworks HDP 2.2 and Cloudera Quickstart VM 5.1_
 
 


[33/50] [abbrv] incubator-kylin git commit: KYLIN-1015 Hive dependency jars appeared twice

Posted by li...@apache.org.
KYLIN-1015 Hive dependency jars appeared twice


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

Branch: refs/heads/master
Commit: 204b517c97358d206f887d6e935c0fd9727dc9c2
Parents: 3991b63
Author: shaofengshi <sh...@apache.org>
Authored: Tue Sep 8 13:54:29 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 8 13:54:42 2015 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/204b517c/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
index a995649..9b3c58f 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
@@ -162,7 +162,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
             classpath = classpath + "," + kylinHiveDependency;
         }
 
-        jobConf.set(MAP_REDUCE_CLASSPATH, classpath + "," + kylinHiveDependency);
+        jobConf.set(MAP_REDUCE_CLASSPATH, classpath);
         logger.info("Hadoop job classpath is: " + job.getConfiguration().get(MAP_REDUCE_CLASSPATH));
     }
 


[34/50] [abbrv] incubator-kylin git commit: KYLIN-998 support non-default hive database

Posted by li...@apache.org.
KYLIN-998 support non-default hive database

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

Branch: refs/heads/master
Commit: c4b4b11dc5d952906019ece31f10aec1ecd5d9fd
Parents: 204b517
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 9 09:49:02 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 9 09:50:14 2015 +0800

----------------------------------------------------------------------
 job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java     | 4 ----
 .../org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java     | 5 +++--
 2 files changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c4b4b11d/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java b/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
index 03119c3..eb6d27b 100644
--- a/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
+++ b/job/src/main/java/org/apache/kylin/job/JoinedFlatTable.java
@@ -52,10 +52,6 @@ import org.xml.sax.SAXException;
 
 public class JoinedFlatTable {
 
-    public static String getTableDir(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir, String jobUUID) {
-        return storageDfsDir + "/" + intermediateTableDesc.getTableName(jobUUID);
-    }
-
     public static String generateCreateTableStatement(IJoinedFlatTableDesc intermediateTableDesc, String storageDfsDir, String jobUUID) {
         StringBuilder ddl = new StringBuilder();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/c4b4b11d/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 0f6f9cb..3b25ee1 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
@@ -226,9 +226,10 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
     private void cleanUnusedIntermediateHiveTable(Configuration conf) throws IOException {
         int uuidLength = 36;
-
+        final String useDatabaseHql = "USE " + KylinConfig.getInstanceFromEnv().getHiveDatabaseForIntermediateTable() + ";";
         StringBuilder buf = new StringBuilder();
         buf.append("hive -e \"");
+        buf.append(useDatabaseHql);
         buf.append("show tables " + "\'kylin_intermediate_*\'" + "; ");
         buf.append("\"");
 
@@ -279,7 +280,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         if (delete == true) {
             buf.delete(0, buf.length());
             buf.append("hive -e \"");
-
+            buf.append(useDatabaseHql);
             for (String delHive : allHiveTablesNeedToBeDeleted) {
                 buf.append("drop table if exists " + delHive + "; ");
                 log.info("Remove " + delHive + " from hive tables.");


[37/50] [abbrv] incubator-kylin git commit: KYLIN-1020 remove HARD_THRESHOLD from StorageContext

Posted by li...@apache.org.
KYLIN-1020 remove HARD_THRESHOLD from StorageContext


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

Branch: refs/heads/master
Commit: b338dc3eacb874c661e58eeb8fba13cbe91adb37
Parents: fbe3395
Author: shaofengshi <sh...@apache.org>
Authored: Wed Sep 9 16:35:43 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 9 16:36:09 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/storage/StorageContext.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/b338dc3e/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java b/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
index ae2436d..536e09d 100644
--- a/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
+++ b/storage/src/main/java/org/apache/kylin/storage/StorageContext.java
@@ -34,7 +34,6 @@ import com.google.common.collect.HashBiMap;
  */
 public class StorageContext {
 
-    public static final int HARD_THRESHOLD = 4000000;
     public static final int DEFAULT_THRESHOLD = 1000000;
 
     public enum OrderEnum {
@@ -102,7 +101,7 @@ public class StorageContext {
     }
 
     public void setThreshold(int t) {
-        threshold = Math.min(t, HARD_THRESHOLD);
+        threshold = t;
     }
 
     public int getLimit() {


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

Posted by li...@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/a6b0df91
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/a6b0df91
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/a6b0df91

Branch: refs/heads/master
Commit: a6b0df914af710b7d3a39639496f79e7af5cc5dd
Parents: 7f36763
Author: Li, Yang <ya...@ebay.com>
Authored: Sun Sep 6 15:29:26 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Sun Sep 6 15:30:06 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/DictionaryManager.java    | 12 +--
 .../org/apache/kylin/dict/TrieDictionary.java   | 87 +++++++-------------
 2 files changed, 32 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a6b0df91/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index a51bb8a..13da78e 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -253,8 +253,7 @@ public class DictionaryManager {
             if (model.isFactTable(col.getTable())) {
                 TblColRef pkCol = model.findPKByFK(col, "inner");
                 if (pkCol != null)
-                    col = pkCol; // scan the counterparty PK on lookup table
-                // instead
+                    col = pkCol; // scan the counterparty PK on lookup table instead
             }
             srcTable = col.getTable();
             srcCol = col.getName();
@@ -317,10 +316,7 @@ public class DictionaryManager {
 
         TableSignature input = dictInfo.getInput();
         for (String existing : existings) {
-            DictionaryInfo existingInfo = load(existing, false); // skip cache,
-            // direct
-            // load from
-            // store
+            DictionaryInfo existingInfo = load(existing, false); // skip cache, direct load from store
             if (input.equals(existingInfo.getInput()))
                 return existing;
         }
@@ -336,9 +332,7 @@ public class DictionaryManager {
 
         for (String existing : existings) {
             logger.info("Checking dup dict :" + existing);
-            DictionaryInfo existingInfo = load(existing, true); // skip cache,
-            // direct load
-            // from store
+            DictionaryInfo existingInfo = load(existing, true); // skip cache, direct load from store
             if (existingInfo == null)
                 logger.info("existingInfo is null");
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a6b0df91/dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index a0cfed3..1ecdf32 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -144,9 +144,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
     @Override
     final protected int getIdFromValueImpl(T value, int roundingFlag) {
         if (enableCache && roundingFlag == 0) {
-            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache
-                                                  // gracefully when short of
-                                                  // memory
+            HashMap cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
             if (cache != null) {
                 Integer id = null;
                 id = (Integer) cache.get(value);
@@ -190,7 +188,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
      *            found, might be 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
@@ -199,11 +197,8 @@ public class TrieDictionary<T> extends Dictionary<T> {
             // match the current node, note [0] of node's value has been matched
             // when this node is selected by its parent
             int p = n + firstByteOffset; // start of node's value
-            int end = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1); // end of
-                                                                       // node's
-                                                                       // value
-            for (p++; p < end && o < inpEnd; p++, o++) { // note matching start
-                                                         // from [1]
+            int end = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1); // end of node's value
+            for (p++; p < end && o < inpEnd; p++, o++) { // note matching start from [1]
                 if (trieBytes[p] != inp[o]) {
                     int comp = BytesUtil.compareByteUnsigned(trieBytes[p], inp[o]);
                     if (comp < 0) {
@@ -216,9 +211,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
             // node completely matched, is input all consumed?
             boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
             if (o == inpEnd) {
-                return p == end && isEndOfValue ? seq : roundSeqNo(roundingFlag, seq - 1, -1, seq); // input
-                                                                                                    // all
-                                                                                                    // matched
+                return p == end && isEndOfValue ? seq : roundSeqNo(roundingFlag, seq - 1, -1, seq); // input all matched
             }
             if (isEndOfValue)
                 seq++;
@@ -226,9 +219,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
             // find a child to continue
             int c = headSize + (BytesUtil.readUnsigned(trieBytes, n, sizeChildOffset) & childOffsetMask);
             if (c == headSize) // has no children
-                return roundSeqNo(roundingFlag, seq - 1, -1, seq); // input only
-                                                                   // partially
-                                                                   // matched
+                return roundSeqNo(roundingFlag, seq - 1, -1, seq); // input only partially matched
             byte inpByte = inp[o];
             int comp;
             while (true) {
@@ -242,26 +233,10 @@ public class TrieDictionary<T> extends Dictionary<T> {
                 } else if (comp < 0) { // try next child
                     seq += BytesUtil.readUnsigned(trieBytes, c + sizeChildOffset, sizeNoValuesBeneath);
                     if (checkFlag(c, BIT_IS_LAST_CHILD))
-                        return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no
-                                                                           // child
-                                                                           // can
-                                                                           // match
-                                                                           // the
-                                                                           // next
-                                                                           // byte
-                                                                           // of
-                                                                           // input
+                        return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no child can match the next byte of input
                     c = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1);
                 } else { // children are ordered by their first value byte
-                    return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no
-                                                                       // child
-                                                                       // can
-                                                                       // match
-                                                                       // the
-                                                                       // next
-                                                                       // byte
-                                                                       // of
-                                                                       // input
+                    return roundSeqNo(roundingFlag, seq - 1, -1, seq); // no child can match the next byte of input
                 }
             }
         }
@@ -279,9 +254,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
     @Override
     final protected T getValueFromIdImpl(int id) {
         if (enableCache) {
-            Object[] cache = idToValueCache.get(); // SoftReference to skip
-                                                   // cache gracefully when
-                                                   // short of memory
+            Object[] cache = idToValueCache.get(); // SoftReference to skip cache gracefully when short of memory
             if (cache != null) {
                 int seq = calcSeqNoFromId(id);
                 if (seq < 0 || seq >= nValues)
@@ -347,8 +320,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
             int nValuesBeneath;
             while (true) {
                 nValuesBeneath = BytesUtil.readUnsigned(trieBytes, c + sizeChildOffset, sizeNoValuesBeneath);
-                if (seq - nValuesBeneath < 0) { // value is under this child,
-                                                // reset n and loop again
+                if (seq - nValuesBeneath < 0) { // value is under this child, reset n and loop again
                     n = c;
                     break;
                 } else { // go to next child
@@ -430,27 +402,26 @@ public class TrieDictionary<T> extends Dictionary<T> {
 
     public static void main(String[] args) throws Exception {
         TrieDictionaryBuilder<String> b = new TrieDictionaryBuilder<String>(new StringBytesConverter());
-        // b.addValue("part");
-        // b.print();
-        // b.addValue("part");
-        // b.print();
-        // b.addValue("par");
-        // b.print();
-        // b.addValue("partition");
-        // b.print();
-        // b.addValue("party");
-        // b.print();
-        // b.addValue("parties");
-        // b.print();
-        // b.addValue("paint");
-        // b.print();
-        b.addValue("-000000.41");
-        b.addValue("0000101.81");
-        b.addValue("6779331");
-        String t = "0000001.6131";
+        b.addValue("");
+        b.print();
+        b.addValue("part");
+        b.print();
+        b.addValue("part");
+        b.print();
+        b.addValue("par");
+        b.print();
+        b.addValue("partition");
+        b.print();
+        b.addValue("party");
+        b.print();
+        b.addValue("parties");
+        b.print();
+        b.addValue("paint");
+        b.print();
         TrieDictionary<String> dict = b.build(0);
 
-        System.out.println(dict.getIdFromValue(t, -1));
-        System.out.println(dict.getIdFromValue(t, 1));
+        dict.dump(System.out);
+        
+        dict.getIdFromValueBytes(new byte[10], 0, 0);
     }
 }


[17/50] [abbrv] incubator-kylin git commit: KYLIN-957 fix file permission error on generated file

Posted by li...@apache.org.
KYLIN-957 fix file permission error on generated file


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

Branch: refs/heads/master
Commit: 8c8765c683d139485ab94c8525341d464368495e
Parents: 78dba12
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 16:18:33 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 16:18:57 2015 +0800

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


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/8c8765c6/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
index 692d53e..1dc634e 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/BulkLoadJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/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)) {


[16/50] [abbrv] incubator-kylin git commit: Merge branch '0.7-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 0.7-staging

Posted by li...@apache.org.
Merge branch '0.7-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 0.7-staging

Conflicts:
	website/_docs/howto/howto_backup_hbase.md
	website/_docs/install/advance_settings.md


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

Branch: refs/heads/master
Commit: 78dba126cc3c35bbe3a61b6592c7c23bcd44f018
Parents: 412a150 a6b0df9
Author: Luke Han <lu...@apache.org>
Authored: Sun Sep 6 15:57:28 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 15:57:28 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/dict/DictionaryManager.java    | 12 +--
 .../org/apache/kylin/dict/TrieDictionary.java   | 87 +++++++-------------
 website/_data/docs.yml                          |  2 +-
 website/_docs/howto/howto_backup_hbase.md       |  9 +-
 website/_docs/howto/howto_cleanup_storage.md    | 28 +++++++
 website/_docs/install/advance_settings.md       | 11 ++-
 6 files changed, 77 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/78dba126/website/_docs/howto/howto_backup_hbase.md
----------------------------------------------------------------------
diff --cc website/_docs/howto/howto_backup_hbase.md
index 5c54ca5,0000000..cc43f27
mode 100644,000000..100644
--- a/website/_docs/howto/howto_backup_hbase.md
+++ b/website/_docs/howto/howto_backup_hbase.md
@@@ -1,26 -1,0 +1,29 @@@
 +---
 +layout: docs
 +title:  How to Clean/Backup HBase Tables
 +categories: howto
 +permalink: /docs/howto/howto_backup_hbase.html
 +version: v0.7.2
 +since: v0.7.1
 +---
 +
- Kylin persists all data (meta data and cube) in HBase; You may want to export the data sometimes for whatever purposes (backup, migration, troubleshotting etc); This page describes the steps to do this and also there is a Java app for you to do this easily.
++Kylin persists all data (meta data and cube) in HBase; You may want to export the data sometimes for whatever purposes 
++(backup, migration, troubleshotting etc); This page describes the steps to do this and also there is a Java app for you to do this easily.
 +
 +Steps:
 +
 +1. Cleanup unused cubes to save storage space (be cautious on production!): run the following command in hbase CLI: 
 +{% highlight Groff markup %}
 +hbase org.apache.hadoop.util.RunJar /${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete true
 +{% endhighlight %}
- 2. List all HBase tables, iterate and then export each Kylin table to HDFS; See [https://hbase.apache.org/book/ops_mgt.html#export](https://hbase.apache.org/book/ops_mgt.html#export)
++2. List all HBase tables, iterate and then export each Kylin table to HDFS; 
++See [https://hbase.apache.org/book/ops_mgt.html#export](https://hbase.apache.org/book/ops_mgt.html#export)
 +
 +3. Copy the export folder from HDFS to local file system, and then archive it;
 +
 +4. (optional) Download the archive from Hadoop CLI to local;
 +
 +5. Cleanup the export folder from CLI HDFS and local file system;
 +
- Kylin provide the "ExportHBaseData.java" (currently only exist in "minicluster" branch) for you to do the step 2-5 in one run; Please ensure the correct path of "kylin.properties" has been set in the sys env; This Java uses the sandbox config by default;
++Kylin provide the "ExportHBaseData.java" (currently only exist in "minicluster" branch) for you to do the 
++step 2-5 in one run; Please ensure the correct path of "kylin.properties" has been set in the sys env; This Java uses the sandbox config by default;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/78dba126/website/_docs/install/advance_settings.md
----------------------------------------------------------------------
diff --cc website/_docs/install/advance_settings.md
index b52b3d6,5eb9e90..f9704ea
--- a/website/_docs/install/advance_settings.md
+++ b/website/_docs/install/advance_settings.md
@@@ -29,12 -29,19 +29,21 @@@ To test if the hbase cluster is ready t
  create 'lzoTable', {NAME => 'colFam',COMPRESSION => 'LZO'}
  {% endhighlight %}
  
- ### Modify kylin_job_conf.xml
+ ### Use LZO for HBase compression
  
 +You'll need to stop Kylin first by running `./kylin.sh stop`, and then modify $KYLIN_HOME/conf/kylin_job_conf.xml by uncommenting some configuration entries related to LZO compression. 
 +After this, you need to run `./kylin.sh start` to start Kylin again. Now Kylin will use LZO to compress MR outputs and hbase tables.
 +
+ Goto $KYLIN_HOME/conf/kylin.properties, change kylin.hbase.default.compression.codec=snappy to kylin.hbase.default.compression.codec=lzo
+ 
+ ### Use LZO for MR jobs
+ 
+ Modify $KYLIN_HOME/conf/kylin_job_conf.xml by changing all org.apache.hadoop.io.compress.SnappyCodec to com.hadoop.compression.lzo.LzoCodec. 
+ 
+ ### Restart Kylin
+ 
+ Start Kylin again. Now Kylin will use LZO to compress MR outputs and HBase tables.
  
 -
  ## Enable LDAP authentication
  
  Kylin supports LDAP authentication for enterprise or production deployment; This is implemented based on Spring Security framework; Before enable LDAP, please contact your LDAP administrator to get necessary information, like LDAP server URL, username/password, search patterns, etc;


[03/50] [abbrv] incubator-kylin git commit: KYLIN-956 hbase compression: gzip for test case and snappy for binary

Posted by li...@apache.org.
KYLIN-956 hbase compression: gzip for test case and snappy for binary


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

Branch: refs/heads/master
Commit: 4e073775de6ae8b4bc864fe159ef2797458d1607
Parents: 0eb8383
Author: honma <ho...@ebay.com>
Authored: Tue Sep 1 17:59:46 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:37:59 2015 +0800

----------------------------------------------------------------------
 conf/kylin.properties                           |   2 +-
 conf/kylin_job_conf.xml                         |  58 ++-
 .../test_kylin_cube_with_slr_1_new_segment.json |  43 --
 .../cube/test_kylin_cube_with_slr_empty.json    |  10 -
 ...est_kylin_cube_with_slr_left_join_empty.json |  11 -
 ...est_kylin_cube_with_slr_left_join_ready.json |  43 --
 .../cube/test_kylin_cube_with_slr_ready.json    |  45 --
 ...st_kylin_cube_with_slr_ready_2_segments.json |  73 ----
 .../cube/test_kylin_cube_without_slr_empty.json |  10 -
 ..._kylin_cube_without_slr_left_join_empty.json |  11 -
 ..._kylin_cube_without_slr_left_join_ready.json |  43 --
 ..._without_slr_left_join_ready_2_segments.json |  73 ----
 .../cube/test_kylin_cube_without_slr_ready.json |  43 --
 .../test_kylin_cube_with_slr_desc.json          | 176 --------
 ...test_kylin_cube_with_slr_left_join_desc.json | 186 ---------
 .../test_kylin_cube_without_slr_desc.json       | 285 -------------
 ...t_kylin_cube_without_slr_left_join_desc.json | 283 -------------
 .../localmeta_v1/kylin.properties               |  77 ----
 .../localmeta_v1/project/default.json           |  12 -
 .../localmeta_v1/project/onlyinner.json         |   8 -
 .../localmeta_v1/project/onlyleft.json          |   8 -
 .../localmeta_v1/table/TEST_CAL_DT.json         | 407 -------------------
 .../table/TEST_CATEGORY_GROUPINGS.json          | 151 -------
 .../localmeta_v1/table/TEST_KYLIN_FACT.json     |  43 --
 .../table/TEST_SELLER_TYPE_DIM.json             |  43 --
 .../localmeta_v1/table/TEST_SITES.json          |  47 ---
 .../localmeta_v1/table_exd/TEST_KYLIN_FACT.json |  16 -
 .../sandbox-hdp21/capacity-scheduler.xml        | 112 -----
 .../test_case_data/sandbox-hdp21/core-site.xml  | 106 -----
 .../sandbox-hdp21/hadoop-policy.xml             | 219 ----------
 .../test_case_data/sandbox-hdp21/hbase-site.xml | 193 ---------
 .../test_case_data/sandbox-hdp21/hdfs-site.xml  | 188 ---------
 .../test_case_data/sandbox-hdp21/hive-site.xml  |  21 -
 .../sandbox-hdp21/httpfs-site.xml               |  17 -
 .../sandbox-hdp21/kylin.properties              |  84 ----
 .../kylin_job_conf.lzo_disabled.xml             |  74 ----
 .../kylin_job_conf.lzo_enabled.xml              |  74 ----
 .../sandbox-hdp21/kylin_job_conf.xml            |  74 ----
 .../sandbox-hdp21/mapred-site.xml               | 169 --------
 .../test_case_data/sandbox-hdp21/yarn-site.xml  | 160 --------
 .../test_case_data/sandbox/kylin.properties     |   3 +
 website/_docs/install/advance_settings.md       |   2 +-
 42 files changed, 32 insertions(+), 3671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index e61aa59..8c7c647 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -61,7 +61,7 @@ kylin.job.yarn.app.rest.check.interval.seconds=10
 kylin.job.hive.database.for.intermediatetable=default
 
 #default compression codec for htable,snappy,lzo,gzip,lz4
-#kylin.hbase.default.compression.codec=snappy
+kylin.hbase.default.compression.codec=snappy
 
 ## Config for Restful APP ##
 # database connection settings:

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/conf/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/conf/kylin_job_conf.xml b/conf/kylin_job_conf.xml
index 81704f2..1b718a0 100644
--- a/conf/kylin_job_conf.xml
+++ b/conf/kylin_job_conf.xml
@@ -24,41 +24,37 @@ limitations under the License. See accompanying LICENSE file.
         </description>
     </property>
 
-    <!-- uncomment the following 5 properties to enable lzo compressing
-
-	<property>
-		<name>mapred.compress.map.output</name>
-		<value>true</value>
-		<description>Compress map outputs</description>
-	</property>
-
-	<property>
-		<name>mapred.map.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for map outputs
-		</description>
-	</property>
+    <property>
+        <name>mapred.compress.map.output</name>
+        <value>true</value>
+        <description>Compress map outputs</description>
+    </property>
 
-	<property>
-		<name>mapred.output.compress</name>
-		<value>true</value>
-		<description>Compress the output of a MapReduce job</description>
-	</property>
+    <property>
+        <name>mapred.map.output.compression.codec</name>
+        <value>org.apache.hadoop.io.compress.SnappyCodec</value>
+        <description>The compression codec to use for map outputs
+        </description>
+    </property>
 
-	<property>
-		<name>mapred.output.compression.codec</name>
-		<value>com.hadoop.compression.lzo.LzoCodec</value>
-		<description>The compression codec to use for job outputs
-		</description>
-	</property>
+    <property>
+        <name>mapred.output.compress</name>
+        <value>true</value>
+        <description>Compress the output of a MapReduce job</description>
+    </property>
 
-	<property>
-		<name>mapred.output.compression.type</name>
-		<value>BLOCK</value>
-		<description>The compression type to use for job outputs</description>
-	</property>
+    <property>
+        <name>mapred.output.compression.codec</name>
+        <value>org.apache.hadoop.io.compress.SnappyCodec</value>
+        <description>The compression codec to use for job outputs
+        </description>
+    </property>
 
-	!-->
+    <property>
+        <name>mapred.output.compression.type</name>
+        <value>BLOCK</value>
+        <description>The compression type to use for job outputs</description>
+    </property>
 
     <property>
         <name>mapreduce.job.max.split.locations</name>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/localmeta_v1/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta_v1/kylin.properties b/examples/test_case_data/localmeta_v1/kylin.properties
deleted file mode 100644
index ff9ece4..0000000
--- a/examples/test_case_data/localmeta_v1/kylin.properties
+++ /dev/null
@@ -1,77 +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.
-#
-
-## Config for Kylin Engine ##
-
-# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.rest.servers=localhost:7070
-
-# The metadata store in hbase
-kylin.metadata.url=
-
-# The storage for final cube file in hbase
-kylin.storage.url=hbase:sandbox.hortonworks.com:2181:/hbase-unsecure
-
-# Temp folder in hdfs, make sure user has the right access to the hdfs directory
-kylin.hdfs.working.dir=/kylin
-
-kylin.job.mapreduce.default.reduce.input.mb=500
-
-# If true, job engine will not assume that hadoop CLI reside on the same server as it self
-# you will have to specify kylin.job.remote.cli.hostname, kylin.job.remote.cli.username and kylin.job.remote.cli.password
-kylin.job.run.as.remote.cmd=false
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.hostname=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.username=
-
-# Only necessary when kylin.job.run.as.remote.cmd=true
-kylin.job.remote.cli.password=
-
-# Used by test cases to prepare synthetic data for sample cube
-kylin.job.remote.cli.working.dir=/tmp/kylin
-
-# Max count of concurrent jobs running
-kylin.job.concurrent.max.limit=10
-
-# Time interval to check hadoop job status
-kylin.job.yarn.app.rest.check.interval.seconds=10
-
-
-
-## Config for Restful APP ##
-# database connection settings:
-ldap.server=
-ldap.username=
-ldap.password=
-ldap.user.searchBase=
-ldap.user.searchPattern=
-ldap.user.groupSearchBase=
-ldap.service.searchBase=OU=
-ldap.service.searchPattern=
-ldap.service.groupSearchBase=
-acl.adminRole=
-acl.defaultRole=
-ganglia.group=
-ganglia.port=8664
-
-###########################config info for sandbox#######################
-kylin.sandbox=true
-
-

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/localmeta_v1/project/default.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta_v1/project/default.json b/examples/test_case_data/localmeta_v1/project/default.json
deleted file mode 100644
index a164af2..0000000
--- a/examples/test_case_data/localmeta_v1/project/default.json
+++ /dev/null
@@ -1,12 +0,0 @@
-{
-"uuid" : "1eaca32a-a33e-4b69-83dd-0bb8b1f8c91b",
-  "name": "default",
-  "cubes": [
-    "test_kylin_cube_with_slr_empty",
-    "test_kylin_cube_without_slr_empty",
-    "test_kylin_cube_with_slr_left_join_empty",
-    "test_kylin_cube_without_slr_left_join_empty"
-  ],
-    "last_modified" : 1418290489904,
-  "create_time" : "2014-10-22 23:06:27 PDT"
-}
\ No newline at end of file

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4e073775/examples/test_case_data/localmeta_v1/table_exd/TEST_KYLIN_FACT.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta_v1/table_exd/TEST_KYLIN_FACT.json b/examples/test_case_data/localmeta_v1/table_exd/TEST_KYLIN_FACT.json
deleted file mode 100644
index 7cc5b5e..0000000
--- a/examples/test_case_data/localmeta_v1/table_exd/TEST_KYLIN_FACT.json
+++ /dev/null
@@ -1,16 +0,0 @@
-{
-  "minFileSize" : "21745785",
-  "totalNumberFiles" : "1",
-  "location" : "/user/hive/warehouse/test_kylin_fact",
-  "lastUpdateTime" : "1416847983199",
-  "lastAccessTime" : "1416847980624",
-  "columns" : "",
-  "partitionColumns" : "",
-  "maxFileSize" : "56146493",
-  "inputformat" : "",
-  "partitioned" : "false",
-  "tableName" : "test_kylin_fact",
-  "owner" : "",
-  "totalFileSize" : "538720920",
-  "outputformat" : ""
-}
\ No newline at end of file


[29/50] [abbrv] incubator-kylin git commit: KYLIN-957 remove “kylin.hadoop.cluster.fs” as Kylin should always run in the MR cluster, not need to configure that;

Posted by li...@apache.org.
KYLIN-957 remove “kylin.hadoop.cluster.fs” as Kylin should always run in the MR cluster, not need to configure that;


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

Branch: refs/heads/master
Commit: 83a38acf9eea42aa1fce2e1978e3030ea83becaa
Parents: b60823b
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 17:00:47 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 17:33:00 2015 +0800

----------------------------------------------------------------------
 bin/check-env.sh                                | 11 ++-----
 bin/sample.sh                                   |  5 ---
 .../org/apache/kylin/common/KylinConfig.java    |  6 ----
 .../apache/kylin/common/util/HadoopUtil.java    | 11 ++-----
 .../kylin/common/util/HadoopUtilTest.java       | 18 -----------
 conf/kylin.properties                           |  4 ---
 .../apache/kylin/job/AbstractJobBuilder.java    |  2 --
 .../apache/kylin/job/cube/CubingJobBuilder.java |  8 ++---
 .../kylin/job/cube/GarbageCollectionStep.java   | 33 +++++++++-----------
 9 files changed, 22 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/bin/check-env.sh
----------------------------------------------------------------------
diff --git a/bin/check-env.sh b/bin/check-env.sh
index 56f2436..d3cd709 100644
--- a/bin/check-env.sh
+++ b/bin/check-env.sh
@@ -46,17 +46,10 @@ then
 fi
 
 WORKING_DIR=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hdfs.working.dir`
-HADOOP_FS=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hadoop.cluster.fs`
-
-if [ "$HADOOP_FS" ]
-then
-  hadoop fs -Dfs.defaultFS=$HADOOP_FS -mkdir -p $WORKING_DIR
-else
-  hadoop fs -mkdir -p $WORKING_DIR
-fi
+hadoop fs -mkdir -p $WORKING_DIR
 
 if [ $? != 0 ]
 then
-    echo "failed to create $HADOOP_FS$WORKING_DIR, Please make sure the user has right to access $HADOOP_FS$WORKING_DIR"
+    echo "failed to create $WORKING_DIR, Please make sure the user has right to access $WORKING_DIR"
     exit 1
 fi

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/bin/sample.sh
----------------------------------------------------------------------
diff --git a/bin/sample.sh b/bin/sample.sh
index 7c4999f..d53393c 100644
--- a/bin/sample.sh
+++ b/bin/sample.sh
@@ -20,14 +20,9 @@
 dir=$(dirname ${0})
 source ${dir}/check-env.sh
 job_jar=`find ${KYLIN_HOME}/lib/ -name kylin-job*.jar`
-HADOOP_FS=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hadoop.cluster.fs`
 echo "Going to create sample tables in hive..."
 cd ${KYLIN_HOME}/sample_cube/data
-if [ -z $HADOOP_FS ];then
 hive -f ${KYLIN_HOME}/sample_cube/create_sample_tables.sql  || { exit 1; }
-else
-hive -hiveconf fs.defaultFS=${HADOOP_FS} -f ${KYLIN_HOME}/sample_cube/create_sample_tables.sql  || { exit 1; }
-fi
 
 echo "Sample hive tables are created successfully; Going to create sample cube..."
 cd ${KYLIN_HOME}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index f1a8e92..d3220ee 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -94,8 +94,6 @@ public class KylinConfig {
 
     public static final String KYLIN_HDFS_WORKING_DIR = "kylin.hdfs.working.dir";
 
-    public static final String KYLIN_HADOOP_CLUSTER_FS = "kylin.hadoop.cluster.fs";
-
     public static final String KYLIN_HBASE_CLUSTER_FS = "kylin.hbase.cluster.fs";
 
     public static final String HIVE_PASSWORD = "hive.password";
@@ -291,10 +289,6 @@ public class KylinConfig {
         return root + getMetadataUrlPrefix() + "/";
     }
 
-    public String getHadoopClusterFs() {
-        return getOptional(KYLIN_HADOOP_CLUSTER_FS, "");
-    }
-
     public String getHBaseClusterFs() {
         return getOptional(KYLIN_HBASE_CLUSTER_FS, "");
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
index 43b2f29..b67b343 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
@@ -53,12 +53,7 @@ public class HadoopUtil {
 
     public static Configuration getCurrentConfiguration() {
         if (hadoopConfig.get() == null) {
-            Configuration configuration = new Configuration();
-            String hadoopClusterFs = KylinConfig.getInstanceFromEnv().getHadoopClusterFs();
-            if (hadoopClusterFs != null && !hadoopClusterFs.equals("")) {
-                configuration.set(FileSystem.FS_DEFAULT_NAME_KEY, hadoopClusterFs);
-            }
-            hadoopConfig.set(configuration);
+            hadoopConfig.set(new Configuration());
         }
         return hadoopConfig.get();
     }
@@ -67,7 +62,7 @@ public class HadoopUtil {
         if (hbaseConfig.get() == null) {
             Configuration configuration = HBaseConfiguration.create(new Configuration());
             String hbaseClusterFs = KylinConfig.getInstanceFromEnv().getHBaseClusterFs();
-            if (hbaseClusterFs != null && !hbaseClusterFs.equals("")) {
+            if (StringUtils.isNotEmpty(hbaseClusterFs)) {
                 configuration.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
             }
             hbaseConfig.set(configuration);
@@ -160,7 +155,7 @@ public class HadoopUtil {
         // conf.set(ScannerCallable.LOG_SCANNER_ACTIVITY, "true");
 
         String hbaseClusterFs = KylinConfig.getInstanceFromEnv().getHBaseClusterFs();
-        if (hbaseClusterFs != null && !hbaseClusterFs.equals("")) {
+        if (StringUtils.isNotEmpty(hbaseClusterFs)) {
             conf.set(FileSystem.FS_DEFAULT_NAME_KEY, hbaseClusterFs);
         }
         return conf;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java b/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
index c380933..8587683 100644
--- a/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
+++ b/common/src/test/java/org/apache/kylin/common/util/HadoopUtilTest.java
@@ -46,15 +46,6 @@ public class HadoopUtilTest {
   }
 
   @Test
-  public void testGetCurrentConfiguration() throws Exception {
-    KylinConfig config = KylinConfig.getInstanceFromEnv();
-    config.setProperty(KylinConfig.KYLIN_HADOOP_CLUSTER_FS, "hdfs://hadoop-cluster/");
-
-    Configuration conf = HadoopUtil.getCurrentConfiguration();
-    assertEquals("hdfs://hadoop-cluster/", conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
-  }
-
-  @Test
   public void testGetCurrentHBaseConfiguration() throws Exception {
     KylinConfig config = KylinConfig.getInstanceFromEnv();
     config.setProperty(KylinConfig.KYLIN_HBASE_CLUSTER_FS, "hdfs://hbase-cluster/");
@@ -64,15 +55,6 @@ public class HadoopUtilTest {
   }
 
   @Test
-  public void testMakeQualifiedPathInHadoopCluster() throws Exception {
-    KylinConfig config = KylinConfig.getInstanceFromEnv();
-    config.setProperty(KylinConfig.KYLIN_HADOOP_CLUSTER_FS, "file:/");
-
-    String path = HadoopUtil.makeQualifiedPathInHadoopCluster("/path/to/test/hadoop");
-    assertEquals("file:/path/to/test/hadoop", path);
-  }
-
-  @Test
   public void testMakeQualifiedPathInHBaseCluster() throws Exception {
     KylinConfig config = KylinConfig.getInstanceFromEnv();
     config.setProperty(KylinConfig.KYLIN_HBASE_CLUSTER_FS, "file:/");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 0aa2898..84a1d46 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -29,10 +29,6 @@ kylin.storage.url=hbase
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
-# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster:8020
-# leave empty if using default fs configured by local core-site.xml
-kylin.hadoop.cluster.fs=
-
 # HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster:8020
 # leave empty if hbase running on same cluster with hive and mapreduce
 kylin.hbase.cluster.fs=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
index ffbfe98..87c4705 100644
--- a/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/AbstractJobBuilder.java
@@ -65,7 +65,6 @@ public abstract class AbstractJobBuilder {
     protected AbstractExecutable createIntermediateHiveTableStep(IJoinedFlatTableDesc intermediateTableDesc, String jobId) {
 
         final String useDatabaseHql = "USE " + engineConfig.getConfig().getHiveDatabaseForIntermediateTable() + ";";
-        final String setClusterHql = "-hiveconf " + FileSystem.FS_DEFAULT_NAME_KEY + "=\"" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY) + "\"";
         final String dropTableHql = JoinedFlatTable.generateDropTableStatement(intermediateTableDesc, jobId);
         final String createTableHql = JoinedFlatTable.generateCreateTableStatement(intermediateTableDesc, getJobWorkingDir(jobId), jobId);
         String insertDataHqls;
@@ -79,7 +78,6 @@ public abstract class AbstractJobBuilder {
         ShellExecutable step = new ShellExecutable();
         StringBuffer buf = new StringBuffer();
         buf.append("hive ");
-        buf.append(setClusterHql);
         buf.append(" -e \"");
         buf.append(useDatabaseHql + "\n");
         buf.append(dropTableHql + "\n");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
index dd71cd8..5c3c277 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
@@ -204,7 +204,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
     AbstractExecutable addHTableSteps(CubeSegment seg, String cuboidRootPath, CubingJob result) {
         final String jobId = result.getId();
-        final String cuboidPath = HadoopUtil.makeQualifiedPathInHadoopCluster(cuboidRootPath + "*");
+        final String cuboidPath = cuboidRootPath + "*";
 
         result.addTask(createRangeRowkeyDistributionStep(seg, cuboidPath));
         // create htable step
@@ -243,8 +243,6 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
             if (jobConf != null && jobConf.length() > 0) {
                 builder.append(" -conf ").append(jobConf);
             }
-            String setCluster = " -D" + FileSystem.FS_DEFAULT_NAME_KEY + "=" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY);
-            builder.append(setCluster);
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
@@ -268,11 +266,11 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
     }
 
     private String getRowkeyDistributionOutputPath(CubeSegment seg) {
-        return HadoopUtil.makeQualifiedPathInHBaseCluster(getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats");
+        return getJobWorkingDir(seg.getLastBuildJobID()) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
     }
 
     private String getFactDistinctColumnsPath(CubeSegment seg, String jobUuid) {
-        return HadoopUtil.makeQualifiedPathInHadoopCluster(getJobWorkingDir(jobUuid) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns");
+        return getJobWorkingDir(jobUuid) + "/" + seg.getCubeInstance().getName() + "/fact_distinct_columns";
     }
 
     private String getHFilePath(CubeSegment seg, String jobId) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/83a38acf/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index b4f6e8e..72cad96 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -84,8 +84,7 @@ public class GarbageCollectionStep extends AbstractExecutable {
         final String hiveTable = this.getOldHiveTable();
         if (StringUtils.isNotEmpty(hiveTable)) {
             final String dropSQL = "USE " + KylinConfig.getInstanceFromEnv().getHiveDatabaseForIntermediateTable() + ";" + " DROP TABLE IF EXISTS  " + hiveTable + ";";
-            final String setClusterHql = "-hiveconf " + FileSystem.FS_DEFAULT_NAME_KEY + "=\"" + HadoopUtil.getCurrentConfiguration().get(FileSystem.FS_DEFAULT_NAME_KEY) + "\"";
-            final String dropHiveCMD = "hive " + setClusterHql + " -e \"" + dropSQL + "\"";
+            final String dropHiveCMD = "hive -e \"" + dropSQL + "\"";
             logger.info("executing: " + dropHiveCMD);
             ShellCmdOutput shellCmdOutput = new ShellCmdOutput();
             context.getConfig().getCliCommandExecutor().execute(dropHiveCMD, shellCmdOutput);
@@ -132,32 +131,28 @@ public class GarbageCollectionStep extends AbstractExecutable {
             }
         }
     }
-
-    private void dropFileSystemPath(FileSystem fs, Path p) throws IOException {
-        Path path = fs.makeQualified(p);
-        if (fs.exists(path)) {
-            fs.delete(path, 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");
-        }
-    }
-
+    
     private void dropHdfsPath(ExecutableContext context) throws IOException {
+
         List<String> oldHdfsPaths = this.getOldHdsfPaths();
         if (oldHdfsPaths != null && oldHdfsPaths.size() > 0) {
-            FileSystem hadoopFs = FileSystem.get(HadoopUtil.getCurrentConfiguration());
-            FileSystem hbaseFs = FileSystem.get(HadoopUtil.getCurrentHBaseConfiguration());
+            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);
-                dropFileSystemPath(hadoopFs, oldPath);
-                dropFileSystemPath(hbaseFs, oldPath);
+                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");
+                }
             }
+
         }
     }
 


[30/50] [abbrv] incubator-kylin git commit: Finish the hive intermediate table clean up job in org.apache.kylin.job.hadoop.cube.StorageCleanupJob

Posted by li...@apache.org.
Finish the hive intermediate table clean up job in org.apache.kylin.job.hadoop.cube.StorageCleanupJob

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/7a2ef176
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/7a2ef176
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/7a2ef176

Branch: refs/heads/master
Commit: 7a2ef1766144783444ccf362b319b0d604e8c628
Parents: 83a38ac
Author: nichunen@mininglamp.com <ni...@mininglamp.com>
Authored: Mon Sep 7 04:01:22 2015 -0400
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 21:23:25 2015 +0800

----------------------------------------------------------------------
 .../job/hadoop/cube/StorageCleanupJob.java      | 98 +++++++++++++++++++-
 1 file changed, 97 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7a2ef176/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 46f0849..ae684fe 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,7 +18,9 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
+import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -44,7 +46,10 @@ import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
 import org.apache.kylin.job.JobInstance;
+import org.apache.kylin.job.cmd.ICommandOutput;
+import org.apache.kylin.job.cmd.ShellCmd;
 import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.exception.JobException;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.job.manager.ExecutableManager;
@@ -86,9 +91,9 @@ public class StorageCleanupJob extends AbstractHadoopJob {
 
             Configuration conf = HBaseConfiguration.create(getConf());
 
+            cleanUnusedIntermediateHiveTable(conf);
             cleanUnusedHdfsFiles(conf);
             cleanUnusedHBaseTables(conf);
-            cleanUnusedIntermediateHiveTable(conf);
 
             return 0;
         } catch (Exception e) {
@@ -220,7 +225,98 @@ public class StorageCleanupJob extends AbstractHadoopJob {
     }
 
     private void cleanUnusedIntermediateHiveTable(Configuration conf) throws IOException {
+        FileSystem fs = FileSystem.get(conf);
+        //JobEngineConfig engineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+        int uuidLength = 36;
+
+        StringBuilder buf = new StringBuilder();
+        buf.append("hive -e \"");
+        buf.append("show tables " + "\'kylin_intermediate_*\'" + "; ");
+        buf.append("\"");
+
+        ShellCmd cmd = new ShellCmd(buf.toString(), null, null, null, false);
+        ICommandOutput output = null;
+
+        try {
+            output = cmd.execute();
+        } catch (JobException e) {
+            e.printStackTrace();
+        }
+
+        if(output == null)
+            return;
+        String outputStr = output.getOutput();
+        BufferedReader reader = new BufferedReader(new StringReader(outputStr));
+        String line = null;
+        List<String> allJobs = executableManager.getAllJobIds();
+        List<String> allHiveTablesNeedToBeDeleted = new ArrayList<String>();
+        List<String> workingJobList = new ArrayList<String>();
+
+        for (String jobId : allJobs) {
+            // only remove FINISHED and DISCARDED job intermediate table
+            final ExecutableState state = executableManager.getOutput(jobId).getState();
+
+            if (!state.isFinalState()) {
+                workingJobList.add(jobId);
+                log.info("Remove intermediate hive table with job id " + jobId + " with job status " + state);
+            }
+        }
+
+        while ((line = reader.readLine()) != null) {
+            if(line.startsWith("kylin_intermediate_")){
+                boolean isNeedDel = true;
+                String uuid = line.substring(line.length() - uuidLength, line.length());
+                uuid = uuid.replace("_", "-");
+                //Check whether it's a hive table in use
+                if(workingJobList.contains(uuid)){
+                    isNeedDel = false;
+                }
+                else{
+                    log.info("Hive table with uuid " + uuid + " is in use.");
+                }
+
+                //Check whether the hive table belongs to current Kylin instance
+                String hdfsPath = JobInstance.getJobWorkingDir(uuid, KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory());
+                Path p = new Path(hdfsPath);
+
+                if (fs.exists(p) == false) {
+                    isNeedDel = false;
+                }
+                else{
+                    log.info("Hive table with uuid " + uuid + " belongs to a different Kylin instance.");
+                }
+
+                if(isNeedDel)
+                    allHiveTablesNeedToBeDeleted.add(line);
+            }
+        }
+        
+        if (delete == true) {
+            buf.delete(0, buf.length());
+            buf.append("hive -e \"");
+
+            for(String delHive : allHiveTablesNeedToBeDeleted){
+                buf.append("drop table if exists " + delHive + "; ");
+                log.info("Remove " + delHive + " from hive tables.");
+            }
+            buf.append("\"");
+            cmd = new ShellCmd(buf.toString(), null, null, null, false);
+
+            try {
+                cmd.execute();
+            } catch (JobException e) {
+                e.printStackTrace();
+            }
+        } else {
+            System.out.println("------ Intermediate Hive Tables To Be Dropped ------");
+            for (String hiveTable : allHiveTablesNeedToBeDeleted) {
+                System.out.println(hiveTable);
+            }
+            System.out.println("----------------------------------------------------");
+        }
 
+        if(reader != null)
+            reader.close();
     }
 
     public static void main(String[] args) throws Exception {


[44/50] [abbrv] incubator-kylin git commit: update pom settings

Posted by li...@apache.org.
update pom 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/df00623f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/df00623f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/df00623f

Branch: refs/heads/master
Commit: df00623f555f3e35a8e743e766275e2b461db5d8
Parents: e0f5e39
Author: Luke Han <lu...@apache.org>
Authored: Fri Sep 11 13:52:51 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Tue Sep 15 17:55:10 2015 +0800

----------------------------------------------------------------------
 pom.xml | 177 ++++++++++++++++++++++++++++++-----------------------------
 1 file changed, 91 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/df00623f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cdce89e..c614478 100644
--- a/pom.xml
+++ b/pom.xml
@@ -596,6 +596,96 @@
                 </plugin>
             </plugins>
         </pluginManagement>
+        <plugins>
+            <!-- Apache-RAT checks for files without headers.
+                         If run on a messy developer's sandbox, it will fail.
+                         This serves as a reminder to only build a release in a clean
+                         sandbox! -->
+                    <plugin>
+                        <groupId>org.apache.rat</groupId>
+                        <artifactId>apache-rat-plugin</artifactId>
+                        <configuration>
+                            <numUnapprovedLicenses>0</numUnapprovedLicenses>
+                            <excludes>
+                                <exclude>DEPENDENCIES</exclude>
+                                <exclude>.idea/**</exclude>
+                                <exclude>.git/**</exclude>
+                                <exclude>.settings/**</exclude>
+                                <!-- text files without comments -->
+                                <exclude>**/*.csv</exclude>
+                                <exclude>**/*.json</exclude>
+                                <exclude>**/*.md</exclude>
+                                <!-- binary files -->
+                                <exclude>**/*.dict</exclude>
+                                <exclude>**/*.dic</exclude>
+                                <exclude>**/*.snapshot</exclude>
+                                <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>website/**</exclude>
+
+                                <!-- tomcat package -->
+                                <exclude>tomcat/**</exclude>
+                                <!-- front end libary and generated files -->
+                                <exclude>webapp/node_modules/**</exclude>
+                                <exclude>webapp/dist/**</exclude>
+                                <exclude>webapp/app/components/**</exclude>
+                                <!-- json configuration file-->
+                                <exclude>webapp/.bowerrc</exclude>
+                                <exclude>webapp/.jshintrc</exclude>
+                                <!-- generated dict files -->
+                                <exclude>dictionary/metastore_db/**</exclude>
+
+                                <!-- MIT license -->
+                                <exclude>webapp/app/css/AdminLTE-fonts.css</exclude>
+                                <exclude>webapp/app/css/AdminLTE.css</exclude>
+
+                                <!-- jdbc log -->
+                                <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>
+                        <executions>
+                            <execution>
+                                <phase>verify</phase>
+                                <goals>
+                                    <goal>check</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.maven.doxia</groupId>
+                                <artifactId>doxia-core</artifactId>
+                                <version>1.6</version>
+                                <exclusions>
+                                    <exclusion>
+                                        <groupId>xerces</groupId>
+                                        <artifactId>xercesImpl</artifactId>
+                                    </exclusion>
+                                </exclusions>
+                            </dependency>
+                        </dependencies>
+                    </plugin>
+
+            
+        </plugins>
     </build>
 
     <modules>
@@ -782,92 +872,7 @@
                             <argLine>-Xms1G -Xmx4G -XX:PermSize=128M -XX:MaxPermSize=512M</argLine>
                         </configuration>
                     </plugin>
-                    <!-- Apache-RAT checks for files without headers.
-                         If run on a messy developer's sandbox, it will fail.
-                         This serves as a reminder to only build a release in a clean
-                         sandbox! -->
-                    <plugin>
-                        <groupId>org.apache.rat</groupId>
-                        <artifactId>apache-rat-plugin</artifactId>
-                        <configuration>
-                            <numUnapprovedLicenses>0</numUnapprovedLicenses>
-                            <excludes>
-                                <exclude>DEPENDENCIES</exclude>
-                                <exclude>.idea/**</exclude>
-                                <exclude>.git/**</exclude>
-                                <exclude>.settings/**</exclude>
-                                <!-- text files without comments -->
-                                <exclude>**/*.csv</exclude>
-                                <exclude>**/*.json</exclude>
-                                <exclude>**/*.md</exclude>
-                                <!-- binary files -->
-                                <exclude>**/*.dict</exclude>
-                                <exclude>**/*.dic</exclude>
-                                <exclude>**/*.snapshot</exclude>
-                                <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>website/**</exclude>
-
-                                <!-- tomcat package -->
-                                <exclude>tomcat/**</exclude>
-                                <!-- front end libary and generated files -->
-                                <exclude>webapp/node_modules/**</exclude>
-                                <exclude>webapp/dist/**</exclude>
-                                <exclude>webapp/app/components/**</exclude>
-                                <!-- json configuration file-->
-                                <exclude>webapp/.bowerrc</exclude>
-                                <exclude>webapp/.jshintrc</exclude>
-                                <!-- generated dict files -->
-                                <exclude>dictionary/metastore_db/**</exclude>
-
-                                <!-- MIT license -->
-                                <exclude>webapp/app/css/AdminLTE-fonts.css</exclude>
-                                <exclude>webapp/app/css/AdminLTE.css</exclude>
-
-                                <!-- jdbc log -->
-                                <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>
-                        <executions>
-                            <execution>
-                                <phase>verify</phase>
-                                <goals>
-                                    <goal>check</goal>
-                                </goals>
-                            </execution>
-                        </executions>
-                        <dependencies>
-                            <dependency>
-                                <groupId>org.apache.maven.doxia</groupId>
-                                <artifactId>doxia-core</artifactId>
-                                <version>1.6</version>
-                                <exclusions>
-                                    <exclusion>
-                                        <groupId>xerces</groupId>
-                                        <artifactId>xercesImpl</artifactId>
-                                    </exclusion>
-                                </exclusions>
-                            </dependency>
-                        </dependencies>
-                    </plugin>
+                    
                     <plugin>
                         <groupId>net.ju-n.maven.plugins</groupId>
                         <artifactId>checksum-maven-plugin</artifactId>


[22/50] [abbrv] incubator-kylin git commit: KYLIN-957 fix bug in check-env.sh

Posted by li...@apache.org.
KYLIN-957 fix bug in check-env.sh


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

Branch: refs/heads/master
Commit: ff9189cd1137f2fe144d68081d63c0be315a62b5
Parents: 66bd59e
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 13:47:37 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 13:48:06 2015 +0800

----------------------------------------------------------------------
 bin/check-env.sh      | 9 ++++++++-
 conf/kylin.properties | 4 ++--
 2 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ff9189cd/bin/check-env.sh
----------------------------------------------------------------------
diff --git a/bin/check-env.sh b/bin/check-env.sh
index 9efb393..56f2436 100644
--- a/bin/check-env.sh
+++ b/bin/check-env.sh
@@ -47,7 +47,14 @@ fi
 
 WORKING_DIR=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hdfs.working.dir`
 HADOOP_FS=`sh $KYLIN_HOME/bin/get-properties.sh kylin.hadoop.cluster.fs`
-hadoop fs -mkdir -p $HADOOP_FS$WORKING_DIR
+
+if [ "$HADOOP_FS" ]
+then
+  hadoop fs -Dfs.defaultFS=$HADOOP_FS -mkdir -p $WORKING_DIR
+else
+  hadoop fs -mkdir -p $WORKING_DIR
+fi
+
 if [ $? != 0 ]
 then
     echo "failed to create $HADOOP_FS$WORKING_DIR, Please make sure the user has right to access $HADOOP_FS$WORKING_DIR"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ff9189cd/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 3850005..0d1827e 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -29,11 +29,11 @@ kylin.storage.url=hbase
 # Temp folder in hdfs, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
-# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster/ (end with /)
+# Hadoop Cluster FileSystem, which serving hive and mapreduce, format as hdfs://hadoop-cluster:8020
 # leave empty if using default fs configured by local core-site.xml
 kylin.hadoop.cluster.fs=
 
-# HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster/ (end with /)
+# HBase Cluster FileSystem, which serving hbase, format as hdfs://hbase-cluster:8020
 # leave empty if hbase running on same cluster with hive and mapreduce
 kylin.hbase.cluster.fs=
 


[14/50] [abbrv] incubator-kylin git commit: update doc

Posted by li...@apache.org.
update doc


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

Branch: refs/heads/master
Commit: 7f36763428ad842cb0a93cb4344eef7ca38abd1f
Parents: 840c0c0
Author: honma <ho...@ebay.com>
Authored: Sun Sep 6 14:38:45 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Sun Sep 6 14:38:45 2015 +0800

----------------------------------------------------------------------
 website/_data/docs.yml                       | 2 +-
 website/_docs/howto/howto_cleanup_storage.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7f367634/website/_data/docs.yml
----------------------------------------------------------------------
diff --git a/website/_data/docs.yml b/website/_data/docs.yml
index a1ac243..6de173a 100644
--- a/website/_data/docs.yml
+++ b/website/_data/docs.yml
@@ -49,6 +49,6 @@
   - howto/howto_use_restapi
   - howto/howto_optimize_cubes
   - howto/howto_backup_metadata
-  - howto/howto_backup_hbase
+  - howto/howto_cleanup_storage
   - howto/howto_jdbc
   - howto/howto_upgrade

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7f367634/website/_docs/howto/howto_cleanup_storage.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_cleanup_storage.md b/website/_docs/howto/howto_cleanup_storage.md
index f8ebb6f..192bc2a 100644
--- a/website/_docs/howto/howto_cleanup_storage.md
+++ b/website/_docs/howto/howto_cleanup_storage.md
@@ -14,7 +14,7 @@ Steps:
 
 1. Check which resources can be cleanup, this will not remove anything: 
 {% highlight Groff markup %}
-hbase org.apache.hadoop.util.RunJar ${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete true
+hbase org.apache.hadoop.util.RunJar ${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete false
 {% endhighlight %}
 
 Here please replace (version) with the specific Kylin jar version in your installation;


[13/50] [abbrv] incubator-kylin git commit: KYLIN-792, support config mapreduce.queue for hivesql in monitor module

Posted by li...@apache.org.
KYLIN-792, support config mapreduce.queue for hivesql in monitor module


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

Branch: refs/heads/master
Commit: 25cbb2c9dbd7c4d628874b5efe3aebbf5caf3acd
Parents: ebf4629
Author: jiazhong <ji...@ebay.com>
Authored: Fri Aug 28 17:36:03 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 bin/performance-monitor.sh                      |  2 +-
 .../java/org/apache/kylin/monitor/Client.java   |  2 +-
 .../org/apache/kylin/monitor/ConfigUtils.java   |  8 ++-
 .../org/apache/kylin/monitor/DebugClient.java   |  2 +-
 .../org/apache/kylin/monitor/FileUtils.java     |  4 +-
 .../apache/kylin/monitor/HiveJdbcClient.java    | 52 +++++++++++++++++++-
 .../kylin/monitor/MonitorMetaManager.java       |  2 +-
 7 files changed, 63 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/bin/performance-monitor.sh
----------------------------------------------------------------------
diff --git a/bin/performance-monitor.sh b/bin/performance-monitor.sh
index 4d30364..d8421e3 100644
--- a/bin/performance-monitor.sh
+++ b/bin/performance-monitor.sh
@@ -30,7 +30,7 @@
     source ${dir}/find-hive-dependency.sh
     source ${dir}/find-hbase-dependency.sh
 
-    export HBASE_CLASSPATH=$hive_dependency:${HBASE_CLASSPATH}
+    export HBASE_CLASSPATH=$hive_dependency:${HBASE_CLASSPATH}:$KYLIN_HOME/conf
     _monitorjar=`ls ${KYLIN_HOME}/lib |grep kylin-monitor`
 
     hbase \

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/Client.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/Client.java b/monitor/src/main/java/org/apache/kylin/monitor/Client.java
index 699e0de..9c881b2 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/Client.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/Client.java
@@ -24,7 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 
 /**
- * Created by jiazhong on 2015/5/7.
+ * @author jiazhong
  */
 public class Client {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java b/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
index 93cb00f..7cff4d7 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/ConfigUtils.java
@@ -34,7 +34,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 
 /**
- * Created by jiazhong on 2015/4/28.
+ * @author jiazhong
  */
 public class ConfigUtils {
 
@@ -68,6 +68,8 @@ public class ConfigUtils {
     public static final String HIVE_JDBC_CON_USERNAME = "kylin.hive.jdbc.connection.username";
     public static final String HIVE_JDBC_CON_PASSWD = "kylin.hive.jdbc.connection.password";
 
+    public static final String KYLIN_MAP_JOB_QUEUE = "mapred.job.queue.name";
+
     public static final String DEPLOY_ENV = "deploy.env";
 
     public static final String HIVE_JDBC_CON_URL = "kylin.hive.jdbc.connection.url";
@@ -169,6 +171,10 @@ public class ConfigUtils {
         return this.monitorConfig.getProperty(KYLIN_METADATA_URL);
     }
 
+    public String getKylinMapJobQueue(){
+        return this.monitorConfig.getProperty(KYLIN_MAP_JOB_QUEUE);
+    }
+
     public String getMetadataUrlPrefix() {
         String hbaseMetadataUrl = getMetadataUrl();
         String defaultPrefix = "kylin_metadata";

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java b/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
index 0ce4a47..3a683da 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/DebugClient.java
@@ -23,7 +23,7 @@ import java.io.File;
 import org.apache.log4j.Logger;
 
 /**
- * Created by jiazhong on 2015/5/7
+ * @author jiazhong
  */
 public class DebugClient {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/FileUtils.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/FileUtils.java b/monitor/src/main/java/org/apache/kylin/monitor/FileUtils.java
index 625f2c7..f14ecdf 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/FileUtils.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/FileUtils.java
@@ -28,7 +28,7 @@ import org.apache.log4j.Logger;
 import au.com.bytecode.opencsv.CSVWriter;
 
 /**
- * Created by jiazhong on 2015/6/18.
+ * @author jiazhong
  */
 public class FileUtils {
 
@@ -113,7 +113,7 @@ public class FileUtils {
      */
     public static FileSystem getHdfsFileSystem() throws IOException {
         Configuration conf = new Configuration();
-        //        conf.set("dfs.client.block.write.replace-datanode-on-failure.policy", "NEVER");
+        //conf.set("dfs.client.block.write.replace-datanode-on-failure.policy", "NEVER");
         FileSystem fs = null;
         try {
             fs = FileSystem.newInstance(conf);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
index ce34db5..185b717 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
@@ -19,6 +19,7 @@
 package org.apache.kylin.monitor;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -29,9 +30,16 @@ import java.util.Calendar;
 
 import org.apache.log4j.Logger;
 import org.datanucleus.util.StringUtils;
+import org.w3c.dom.Document;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
 
 /**
- * Created by jiazhong on 2015/6/17.
+ * @author jiazhong
  */
 public class HiveJdbcClient {
 
@@ -56,6 +64,10 @@ public class HiveJdbcClient {
 
     final static Logger logger = Logger.getLogger(HiveJdbcClient.class);
 
+    final static String KYLIN_JOB_CONF_XML = "kylin_job_conf.xml";
+
+    static String MAP_QUEUE_NAME;
+
     private static String driverName = "org.apache.hive.jdbc.HiveDriver";
     private static ConfigUtils monitorConfig = ConfigUtils.getInstance();
 
@@ -79,13 +91,22 @@ public class HiveJdbcClient {
      */
     public void start() throws SQLException, IOException {
 
+        MAP_QUEUE_NAME = getQueueName();
+
+        if(org.apache.commons.lang.StringUtils.isEmpty(MAP_QUEUE_NAME)){
+            MAP_QUEUE_NAME = monitorConfig.getKylinMapJobQueue();
+        }
+
+        logger.info("mapred.job.queue.name:"+MAP_QUEUE_NAME);
+
         String CON_URL = monitorConfig.getHiveJdbcConUrl();
 
         String USER_NAME = monitorConfig.getHiveJdbcConUserName();
         String PASSWD = monitorConfig.getHiveJdbcConPasswd();
 
-        Connection con = DriverManager.getConnection(CON_URL, USER_NAME, PASSWD);
+        Connection con = DriverManager.getConnection(CON_URL,USER_NAME,PASSWD);
         Statement stmt = con.createStatement();
+        stmt.execute("set mapred.job.queue.name="+MAP_QUEUE_NAME);
         ResultSet res = null;
 
         SQL_GENERATE_QUERY_LOG_TABLE = generateQueryLogSql();
@@ -205,6 +226,33 @@ public class HiveJdbcClient {
         return monthStasticSqlConvert(SQL_EACH_DAY_PERCENTILE);
     }
 
+    public String getQueueName() throws IOException {
+        String queueName = "";
+        InputStream stream =  this.getClass().getClassLoader().getResourceAsStream(KYLIN_JOB_CONF_XML);
+        if (stream!=null) {
+            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder builder;
+            Document doc;
+            try {
+                builder = factory.newDocumentBuilder();
+                doc = builder.parse(stream);
+                NodeList nl = doc.getElementsByTagName("property");
+                for (int i = 0; i < nl.getLength(); i++) {
+                    String name = doc.getElementsByTagName("name").item(i).getFirstChild().getNodeValue();
+                    String value = doc.getElementsByTagName("value").item(i).getFirstChild().getNodeValue();
+                    if (name.equals("mapreduce.job.queuename")) {
+                        queueName = value;
+                    }
+                }
+
+            } catch (ParserConfigurationException e) {
+                throw new IOException(e);
+            } catch (SAXException e) {
+                throw new IOException(e);
+            }
+        }
+        return queueName;
+    }
     public String monthStasticSqlConvert(String sql) {
 
         SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd");

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/25cbb2c9/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
index 0750436..97200fc 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.log4j.Logger;
 
 /**
- * Created by jiazhong on 2015/5/25.
+ * @author jiazhong
  */
 public class MonitorMetaManager {
 


[47/50] [abbrv] incubator-kylin git commit: KYLIN-978 GarbageCollectionStep dropped Hive Intermediate Table but didn't drop external hdfs path

Posted by li...@apache.org.
KYLIN-978 GarbageCollectionStep dropped Hive Intermediate Table but didn't drop external hdfs path

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/fd172821
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/fd172821
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/fd172821

Branch: refs/heads/master
Commit: fd172821d9070b1bd704291a164569dc9d920045
Parents: 8fd1404
Author: sunyerui <su...@gmail.com>
Authored: Fri Sep 11 00:04:42 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Sep 16 21:02:26 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/cube/CubingJobBuilder.java | 47 ++++++++++++--------
 .../kylin/job/cube/GarbageCollectionStep.java   | 38 ++++++++++++----
 2 files changed, 58 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fd172821/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
index ff79286..de75f7d 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/CubingJobBuilder.java
@@ -25,8 +25,6 @@ import java.util.List;
 import java.util.TimeZone;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
@@ -65,9 +63,11 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         final CubingJob result = initialJob(seg, "BUILD");
         final String jobId = result.getId();
         final String cuboidRootPath = getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/cuboid/";
+        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
+        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
 
         // cubing
-        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(seg, cuboidRootPath, result);
+        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(seg, cuboidRootPath, result, toDeletePathsOnHadoopCluster);
         String intermediateHiveTableStepId = twoSteps.getFirst().getId();
         String baseCuboidStepId = twoSteps.getSecond().getId();
 
@@ -79,7 +79,8 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         final CubeJoinedFlatTableDesc intermediateTableDesc = new CubeJoinedFlatTableDesc(seg.getCubeDesc(), seg);
         final String hiveIntermediateTable = this.getIntermediateHiveTableName(intermediateTableDesc, jobId);
-        result.addTask(createGarbageCollectionStep(seg, null, hiveIntermediateTable, null));
+        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
+        result.addTask(createGarbageCollectionStep(seg, null, hiveIntermediateTable, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
 
         return result;
     }
@@ -92,9 +93,14 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         final String jobId = result.getId();
         final String appendRootPath = getJobWorkingDir(jobId) + "/" + appendSegment.getCubeInstance().getName() + "/append_cuboid/";
         final String mergedRootPath = getJobWorkingDir(jobId) + "/" + appendSegment.getCubeInstance().getName() + "/cuboid/";
+        List<String> mergingSegmentIds = Lists.newArrayList();
+        List<String> mergingCuboidPaths = Lists.newArrayList();
+        List<String> mergingHTables = Lists.newArrayList();
+        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
+        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
 
         // cubing the incremental segment
-        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(appendSegment, appendRootPath, result);
+        Pair<AbstractExecutable, AbstractExecutable> twoSteps = addCubingSteps(appendSegment, appendRootPath, result, toDeletePathsOnHadoopCluster);
         final String intermediateHiveTableStepId = twoSteps.getFirst().getId();
         final String baseCuboidStepId = twoSteps.getSecond().getId();
 
@@ -103,10 +109,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         List<CubeSegment> mergingSegments = mergeSegment.getCubeInstance().getMergingSegments(mergeSegment);
         Preconditions.checkState(mergingSegments.size() > 1, "there should be more than 2 segments to merge");
-        List<String> mergingSegmentIds = Lists.newArrayList();
-        List<String> mergingCuboidPaths = Lists.newArrayList();
-        List<String> mergingHTables = Lists.newArrayList();
-        List<String> toDeletePaths = Lists.newArrayList();
+
         for (CubeSegment merging : mergingSegments) {
             mergingSegmentIds.add(merging.getUuid());
             mergingHTables.add(merging.getStorageLocationIdentifier());
@@ -115,7 +118,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
             } else {
                 mergingCuboidPaths.add(getPathToMerge(merging));
             }
-            toDeletePaths.add(getJobWorkingDir(merging.getLastBuildJobID()));
+            toDeletePathsOnHadoopCluster.add(getJobWorkingDir(merging.getLastBuildJobID()));
         }
 
         // merge cuboid
@@ -126,7 +129,8 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         // update cube info
         result.addTask(createUpdateCubeInfoAfterMergeStep(mergeSegment, mergingSegmentIds, convertCuboidToHfileStep.getId(), jobId));
-        result.addTask(createGarbageCollectionStep(mergeSegment, mergingHTables, null, toDeletePaths));
+        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
+        result.addTask(createGarbageCollectionStep(mergeSegment, mergingHTables, null, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
 
         return result;
     }
@@ -143,12 +147,14 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         List<String> mergingSegmentIds = Lists.newArrayList();
         List<String> mergingCuboidPaths = Lists.newArrayList();
         List<String> mergingHTables = Lists.newArrayList();
-        List<String> toDeletePaths = Lists.newArrayList();
+        final List<String> toDeletePathsOnHadoopCluster = Lists.newArrayList();
+        final List<String> toDeletePathsOnHBaseCluster = Lists.newArrayList();
+
         for (CubeSegment merging : mergingSegments) {
             mergingSegmentIds.add(merging.getUuid());
             mergingCuboidPaths.add(getPathToMerge(merging));
             mergingHTables.add(merging.getStorageLocationIdentifier());
-            toDeletePaths.add(getJobWorkingDir(merging.getLastBuildJobID()));
+            toDeletePathsOnHadoopCluster.add(getJobWorkingDir(merging.getLastBuildJobID()));
         }
 
         // merge cuboid
@@ -159,7 +165,8 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
 
         // update cube info
         result.addTask(createUpdateCubeInfoAfterMergeStep(seg, mergingSegmentIds, convertCuboidToHfileStep.getId(), jobId));
-        result.addTask(createGarbageCollectionStep(seg, mergingHTables, null, toDeletePaths));
+        toDeletePathsOnHBaseCluster.add(getJobWorkingDir(jobId));
+        result.addTask(createGarbageCollectionStep(seg, mergingHTables, null, toDeletePathsOnHadoopCluster, toDeletePathsOnHBaseCluster));
         return result;
     }
 
@@ -171,7 +178,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         result.addTask(createMergeCuboidDataStep(seg, formattedPath, mergedCuboidPath));
     }
 
-    Pair<AbstractExecutable, AbstractExecutable> addCubingSteps(CubeSegment seg, String cuboidRootPath, CubingJob result) {
+    Pair<AbstractExecutable, AbstractExecutable> addCubingSteps(CubeSegment seg, String cuboidRootPath, CubingJob result, List<String> toDeletePaths) {
         final int groupRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getNCuboidBuildLevels();
         final int totalRowkeyColumnsCount = seg.getCubeDesc().getRowkey().getRowKeyColumns().length;
 
@@ -199,6 +206,9 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
             result.addTask(createNDimensionCuboidStep(seg, cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount));
         }
 
+        toDeletePaths.add(intermediateHiveTableLocation);
+        toDeletePaths.add(factDistinctColumnsPath);
+
         return new Pair<AbstractExecutable, AbstractExecutable>(intermediateHiveTableStep, baseCuboidStep);
     }
 
@@ -266,7 +276,7 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
     }
 
     private String getRowkeyDistributionOutputPath(CubeSegment seg, String jobId) {
-        return getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats";
+        return HadoopUtil.makeQualifiedPathInHBaseCluster(getJobWorkingDir(jobId) + "/" + seg.getCubeInstance().getName() + "/rowkey_stats");
     }
 
     private String getFactDistinctColumnsPath(CubeSegment seg, String jobUuid) {
@@ -460,12 +470,13 @@ public final class CubingJobBuilder extends AbstractJobBuilder {
         return result;
     }
 
-    private GarbageCollectionStep createGarbageCollectionStep(CubeSegment seg, List<String> oldHtables, String hiveIntermediateTable, List<String> oldHdsfPaths) {
+    private GarbageCollectionStep createGarbageCollectionStep(CubeSegment seg, List<String> oldHtables, String hiveIntermediateTable, List<String> oldHdsfPaths, List<String> oldHdfsPathsOnHBaseCluster) {
         GarbageCollectionStep result = new GarbageCollectionStep();
         result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
         result.setOldHTables(oldHtables);
         result.setOldHiveTable(hiveIntermediateTable);
-        result.setOldHdsfPaths(oldHdsfPaths);
+        result.setOldHdfsPaths(oldHdsfPaths);
+        result.setOldHdfsPathsOnHBaseCluster(oldHdfsPathsOnHBaseCluster);
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fd172821/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index 72cad96..641454c 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -55,6 +55,8 @@ public class GarbageCollectionStep extends AbstractExecutable {
 
     private static final String OLD_HDFS_PATHS = "oldHdfsPaths";
 
+    private static final String OLD_HDFS_PATHS_ON_HBASE_CLUSTER = "oldHdfsPathsOnHBaseCluster";
+
     private static final Logger logger = LoggerFactory.getLogger(GarbageCollectionStep.class);
 
     private StringBuffer output;
@@ -69,8 +71,9 @@ public class GarbageCollectionStep extends AbstractExecutable {
 
         try {
             dropHBaseTable(context);
-            dropHdfsPath(context);
             dropHiveTable(context);
+            dropHdfsPath(context);
+            dropHdfsPathOnHBaseCluster(context);
         } catch (IOException e) {
             logger.error("job:" + getId() + " execute finished with exception", e);
             output.append("\n").append(e.getLocalizedMessage());
@@ -131,13 +134,11 @@ public class GarbageCollectionStep extends AbstractExecutable {
             }
         }
     }
-    
-    private void dropHdfsPath(ExecutableContext context) throws IOException {
 
-        List<String> oldHdfsPaths = this.getOldHdsfPaths();
+    private void dropHdfsPathOnCluster(List<String> oldHdfsPaths, FileSystem fileSystem) throws IOException {
         if (oldHdfsPaths != null && oldHdfsPaths.size() > 0) {
-            Configuration hconf = HadoopUtil.getCurrentConfiguration();
-            FileSystem fileSystem = FileSystem.get(hconf);
+            logger.debug("Drop HDFS path on FileSystem: " + fileSystem.getUri());
+            output.append("Drop HDFS path on FileSystem: \"" + fileSystem.getUri() + "\" \n");
             for (String path : oldHdfsPaths) {
                 if (path.endsWith("*"))
                     path = path.substring(0, path.length() - 1);
@@ -152,10 +153,21 @@ public class GarbageCollectionStep extends AbstractExecutable {
                     output.append("HDFS path not exists: \"" + path + "\" \n");
                 }
             }
-
         }
     }
 
+    private void dropHdfsPath(ExecutableContext context) throws IOException {
+        List<String> oldHdfsPaths = this.getOldHdfsPaths();
+        FileSystem fileSystem = FileSystem.get(HadoopUtil.getCurrentConfiguration());
+        dropHdfsPathOnCluster(oldHdfsPaths, fileSystem);
+    }
+
+    private void dropHdfsPathOnHBaseCluster(ExecutableContext context) throws IOException {
+        List<String> oldHdfsPaths = this.getOldHdfsPathsOnHBaseCluster();
+        FileSystem fileSystem = FileSystem.get(HadoopUtil.getCurrentHBaseConfiguration());
+        dropHdfsPathOnCluster(oldHdfsPaths, fileSystem);
+    }
+
     public void setOldHTables(List<String> tables) {
         setArrayParam(OLD_HTABLES, tables);
     }
@@ -164,14 +176,22 @@ public class GarbageCollectionStep extends AbstractExecutable {
         return getArrayParam(OLD_HTABLES);
     }
 
-    public void setOldHdsfPaths(List<String> paths) {
+    public void setOldHdfsPaths(List<String> paths) {
         setArrayParam(OLD_HDFS_PATHS, paths);
     }
 
-    private List<String> getOldHdsfPaths() {
+    private List<String> getOldHdfsPaths() {
         return getArrayParam(OLD_HDFS_PATHS);
     }
 
+    public void setOldHdfsPathsOnHBaseCluster(List<String> paths) {
+        setArrayParam(OLD_HDFS_PATHS_ON_HBASE_CLUSTER, paths);
+    }
+
+    private List<String> getOldHdfsPathsOnHBaseCluster() {
+        return getArrayParam(OLD_HDFS_PATHS_ON_HBASE_CLUSTER);
+    }
+
     private void setArrayParam(String paramKey, List<String> paramValues) {
         setParam(paramKey, StringUtils.join(paramValues, ","));
     }


[11/50] [abbrv] incubator-kylin git commit: KYLIN-632 "kylin.sh stop" doesn't check whether KYLIN_HOME was set

Posted by li...@apache.org.
KYLIN-632 "kylin.sh stop" doesn't check whether KYLIN_HOME was set


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

Branch: refs/heads/master
Commit: f9535cb89ea556279773ab56626698c8b7ab1d6b
Parents: d492d5d
Author: honma <ho...@ebay.com>
Authored: Wed Sep 2 17:07:37 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 bin/kylin.sh | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/f9535cb8/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index dcd835e..fbaff24 100644
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -17,10 +17,13 @@
 # limitations under the License.
 #
 
+dir=$(dirname ${0})
+source ${dir}/check-env.sh
+mkdir -p ${KYLIN_HOME}/logs
+
 if [ $1 == "start" ]
 then
-    dir=$(dirname ${0})
-    source ${dir}/check-env.sh
+
     tomcat_root=${dir}/../tomcat
     export tomcat_root
 


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

Posted by li...@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/50e0875d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/50e0875d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/50e0875d

Branch: refs/heads/master
Commit: 50e0875d500cda2a5d5f54246b21b179f6a4f6c2
Parents: 8c8765c
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 6 22:07:55 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 6 22:07:55 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    | 13 +++++++++
 conf/kylin.properties                           |  8 ++++++
 .../test_case_data/sandbox/kylin.properties     |  8 ++++++
 .../kylin/job/constant/BatchConstants.java      |  1 +
 .../hadoop/cube/RangeKeyDistributionJob.java    |  6 ++--
 .../cube/RangeKeyDistributionReducer.java       | 29 +++++++-------------
 6 files changed, 44 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 76031c2..59337d8 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -127,6 +127,10 @@ public class KylinConfig {
     public static final String VERSION = "${project.version}";
 
     public static final String HTABLE_DEFAULT_COMPRESSION_CODEC = "kylin.hbase.default.compression.codec";
+    
+    public static final String HBASE_REGION_SIZE = "kylin.hbase.region.size";
+    
+    public static final String HBASE_REGION_MAX_COUNT = "kylin.hbase.region.max.count";
 
     // static cached instances
     private static KylinConfig ENV_INSTANCE = null;
@@ -651,6 +655,15 @@ public class KylinConfig {
     public void setRemoteHadoopCliPassword(String v) {
         kylinConfig.setProperty(KYLIN_JOB_REMOTE_CLI_PASSWORD, v);
     }
+    
+    public int getHBaseRegionSizeGB(String capacity) {
+        String key = HBASE_REGION_SIZE + "." + capacity.toLowerCase();
+        return Integer.valueOf(getOptional(key, "10"));
+    }
+    
+    public int getHBaseMaxRegionCount() {
+        return Integer.valueOf(getOptional(HBASE_REGION_MAX_COUNT, "500"));
+    }
 
     public String getProperty(String key, String defaultValue) {
         return kylinConfig.getString(key, defaultValue);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index af860bd..98eeafc 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -71,6 +71,14 @@ kylin.job.hive.database.for.intermediatetable=default
 #default compression codec for htable,snappy,lzo,gzip,lz4
 kylin.hbase.default.compression.codec=snappy
 
+# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
+kylin.hbase.region.size.small=10
+kylin.hbase.region.size.medium=20
+kylin.hbase.region.size.large=100
+
+# Max region count when create HTable
+kylin.hbase.region.max.count=500
+
 ## Config for Restful APP ##
 # database connection settings:
 ldap.server=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/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 359174d..985f65e 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -59,6 +59,14 @@ kylin.job.hive.database.for.intermediatetable=default
 #default compression codec for htable,snappy,lzo,gzip,lz4
 kylin.hbase.default.compression.codec=gzip
 
+# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
+kylin.hbase.region.size.small=2
+kylin.hbase.region.size.medium=5
+kylin.hbase.region.size.large=10
+
+# Max region count when create HTable
+kylin.hbase.region.max.count=10
+
 ## Config for Restful APP ##
 # database connection settings:
 ldap.server=

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
index 7776988..03eb2ef 100644
--- a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
+++ b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
@@ -43,6 +43,7 @@ public interface BatchConstants {
 
     String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
     String REGION_NUMBER = "region.number";
+    String REGION_SPLIT_SIZE = "region.split.size";
     String CUBE_CAPACITY = "cube.capacity";
 
     String CFG_KYLIN_LOCAL_TEMP_DIR = "/tmp/kylin/";

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
index 190bf0c..2ca75a5 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
@@ -91,8 +91,10 @@ public class RangeKeyDistributionJob extends AbstractHadoopJob {
             String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
             CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
             CubeInstance cube = cubeMgr.getCube(cubeName);
-            RealizationCapacity realizationCapacity = cube.getDescriptor().getModel().getCapacity();
-            job.getConfiguration().set(BatchConstants.CUBE_CAPACITY, realizationCapacity.toString());
+            int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionSizeGB(cube.getDescriptor().getModel().getCapacity().toString());
+            int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseMaxRegionCount();
+            job.getConfiguration().set(BatchConstants.REGION_SPLIT_SIZE, String.valueOf(regionSplitSize));
+            job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(maxRegionCount));
 
             return waitForCompletion(job);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/50e0875d/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
index 018d952..c97ce24 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
@@ -38,17 +38,13 @@ import org.slf4j.LoggerFactory;
 public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable, Text, LongWritable> {
 
     public static final long ONE_GIGA_BYTES = 1024L * 1024L * 1024L;
-    public static final int SMALL_CUT = 10; //  10 GB per region
-    public static final int MEDIUM_CUT = 20; //  20 GB per region
-    public static final int LARGE_CUT = 100; // 100 GB per region
-
-    public static final int MAX_REGION = 500;
 
     private static final Logger logger = LoggerFactory.getLogger(RangeKeyDistributionReducer.class);
 
     private LongWritable outputValue = new LongWritable(0);
 
-    private int cut;
+    private int maxRegionCount = 500;
+    private int cut = 10;
     private long bytesRead = 0;
     private List<Text> gbPoints = new ArrayList<Text>();
 
@@ -56,20 +52,15 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
     protected void setup(Context context) throws IOException {
         super.publishConfiguration(context.getConfiguration());
 
-        CubeCapacity cubeCapacity = CubeCapacity.valueOf(context.getConfiguration().get(BatchConstants.CUBE_CAPACITY));
-        switch (cubeCapacity) {
-        case SMALL:
-            cut = SMALL_CUT;
-            break;
-        case MEDIUM:
-            cut = MEDIUM_CUT;
-            break;
-        case LARGE:
-            cut = LARGE_CUT;
-            break;
+        if (context.getConfiguration().get(BatchConstants.REGION_SPLIT_SIZE) != null) {
+            cut = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_SPLIT_SIZE));
+        }
+
+        if (context.getConfiguration().get(BatchConstants.REGION_NUMBER) != null) {
+            maxRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER));
         }
 
-        logger.info("Chosen cut for htable is " + cut);
+        logger.info("Chosen cut for htable is " + cut + ", max region count is " + maxRegionCount);
     }
 
     @Override
@@ -88,7 +79,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
     protected void cleanup(Context context) throws IOException, InterruptedException {
         int nRegion = Math.round((float) gbPoints.size() / (float) cut);
         nRegion = Math.max(1, nRegion);
-        nRegion = Math.min(MAX_REGION, nRegion);
+        nRegion = Math.min(maxRegionCount, nRegion);
 
         int gbPerRegion = gbPoints.size() / nRegion;
         gbPerRegion = Math.max(1, gbPerRegion);


[08/50] [abbrv] incubator-kylin git commit: minor, fix bad wrapping of in pom.xml

Posted by li...@apache.org.
minor, fix bad wrapping of <pluginManagement> in pom.xml


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

Branch: refs/heads/master
Commit: e839bc940142b84a6a105f7d66f95144385712e9
Parents: f9535cb
Author: Yang Li <li...@apache.org>
Authored: Thu Sep 3 12:53:27 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 pom.xml | 71 +++++++++++++++++++++++++++---------------------------------
 1 file changed, 32 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e839bc94/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 16786d4..cdce89e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -429,8 +429,6 @@
                     </configuration>
                 </plugin>
 
-
-
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-site-plugin</artifactId>
@@ -546,40 +544,36 @@
                     <version>2.15</version>
                 </plugin>
 
-            </plugins>
-        </pluginManagement>
-
-        <plugins>
-            <!--
-            <plugin>
-                <groupId>com.mycila</groupId>
-                <artifactId>license-maven-plugin</artifactId>
-                <version>${maven-license.plugin.version}</version>
-                <configuration>
-                    <header>License-Header.txt</header>
-                    <licenseName>apache_v2</licenseName>
-                    <addJavaLicenseAfterPackage>false</addJavaLicenseAfterPackage>
-                    <canUpdateCopyright>true</canUpdateCopyright>
-                    <canUpdateDescription>true</canUpdateDescription>
-                    <excludes>
-                        <exclude>**/*.diff</exclude>
-                        <exclude>*.patch</exclude>
-                        <exclude>**/*.log</exclude>
-                        <exclude>**/*.data</exclude>
-                        <exclude>**/README.*</exclude>
-                    </excludes>
-                </configuration>
-            </plugin>
-            -->
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-gpg-plugin</artifactId>
-                <configuration>
-                    <skip>true</skip>
-                </configuration>
-            </plugin>
-            <!-- Checkstyle Plugin -->
-            <plugin>
+                <!--
+                <plugin>
+                    <groupId>com.mycila</groupId>
+                    <artifactId>license-maven-plugin</artifactId>
+                    <version>${maven-license.plugin.version}</version>
+                    <configuration>
+                        <header>License-Header.txt</header>
+                        <licenseName>apache_v2</licenseName>
+                        <addJavaLicenseAfterPackage>false</addJavaLicenseAfterPackage>
+                        <canUpdateCopyright>true</canUpdateCopyright>
+                        <canUpdateDescription>true</canUpdateDescription>
+                        <excludes>
+                            <exclude>**/*.diff</exclude>
+                            <exclude>*.patch</exclude>
+                            <exclude>**/*.log</exclude>
+                            <exclude>**/*.data</exclude>
+                            <exclude>**/README.*</exclude>
+                        </excludes>
+                    </configuration>
+                </plugin>
+                -->
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-gpg-plugin</artifactId>
+                    <configuration>
+                        <skip>true</skip>
+                    </configuration>
+                </plugin>
+                <!-- Checkstyle Plugin -->
+                <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-checkstyle-plugin</artifactId>
                     <executions>
@@ -600,9 +594,8 @@
                         </execution>
                     </executions>
                 </plugin>
-
-        </plugins>
-
+            </plugins>
+        </pluginManagement>
     </build>
 
     <modules>


[28/50] [abbrv] incubator-kylin git commit: Merge branch '0.7-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 0.7-staging

Posted by li...@apache.org.
Merge branch '0.7-staging' of https://git-wip-us.apache.org/repos/asf/incubator-kylin into 0.7-staging


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

Branch: refs/heads/master
Commit: b60823b23a97c9a02b1da449032a5f9460eaaca6
Parents: 4c15ee6 fa925f7
Author: Luke Han <lu...@apache.org>
Authored: Mon Sep 7 16:23:55 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Mon Sep 7 16:23:55 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    | 52 ++++++++++++--------
 conf/kylin.properties                           | 16 +++---
 .../test_case_data/sandbox/kylin.properties     | 16 +++---
 .../kylin/job/constant/BatchConstants.java      |  2 +
 .../hadoop/cube/RangeKeyDistributionJob.java    | 10 ++--
 .../cube/RangeKeyDistributionReducer.java       | 13 +++--
 6 files changed, 67 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



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

Posted by li...@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/fa925f70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/fa925f70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/fa925f70

Branch: refs/heads/master
Commit: fa925f70cb2ff8ecdcd3287784a637d60e672b3c
Parents: 60a95f3
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 7 16:09:46 2015 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 7 16:11:39 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    | 52 ++++++++++++--------
 conf/kylin.properties                           | 16 +++---
 .../test_case_data/sandbox/kylin.properties     | 16 +++---
 .../kylin/job/constant/BatchConstants.java      |  2 +
 .../hadoop/cube/RangeKeyDistributionJob.java    | 10 ++--
 .../cube/RangeKeyDistributionReducer.java       | 13 +++--
 6 files changed, 67 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 5566fa2..f1a8e92 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -127,11 +127,14 @@ public class KylinConfig {
     public static final String VERSION = "${project.version}";
 
     public static final String HTABLE_DEFAULT_COMPRESSION_CODEC = "kylin.hbase.default.compression.codec";
-    
-    public static final String HBASE_REGION_SIZE = "kylin.hbase.region.size";
-    
-    public static final String HBASE_REGION_MAX_COUNT = "kylin.hbase.region.max.count";
 
+    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 HBASE_REGION_COUNT_MIN = "kylin.hbase.region.count.min";
+    public static final String HBASE_REGION_COUNT_MAX = "kylin.hbase.region.count.max";
+    
     // static cached instances
     private static KylinConfig ENV_INSTANCE = null;
 
@@ -655,27 +658,34 @@ public class KylinConfig {
     public void setRemoteHadoopCliPassword(String v) {
         kylinConfig.setProperty(KYLIN_JOB_REMOTE_CLI_PASSWORD, v);
     }
+
+    public int getHBaseRegionCountMin() {
+        return Integer.parseInt(getOptional(HBASE_REGION_COUNT_MIN, "1"));
+    }
+
+    public int getHBaseRegionCountMax() {
+        return Integer.parseInt(getOptional(HBASE_REGION_COUNT_MAX, "500"));
+    }
     
-    public int getHBaseRegionSizeGB(String capacity) {
-        String key = HBASE_REGION_SIZE + "." + capacity.toLowerCase();
-
-        int cut = 20;
-        if (kylinConfig.containsKey(key)) {
-            cut = kylinConfig.getInt(key);
-        } else if ("small".equalsIgnoreCase(capacity)) {
-            cut = 10;
-        } else if ("medium".equalsIgnoreCase(capacity)) {
-            cut = 20;
-        } else if ("large".equalsIgnoreCase(capacity)) {
-            cut = 100;
+    public int getHBaseRegionCut(String capacity) {
+        String cut;
+        switch (capacity) {
+            case "SMALL":
+                cut = getProperty(HBASE_REGION_CUT_SMALL, "10");
+                break;
+            case "MEDIUM":
+                cut = getProperty(HBASE_REGION_CUT_MEDIUM, "20");
+                break;
+            case "LARGE":
+                cut = getProperty(HBASE_REGION_CUT_LARGE, "100");
+                break;
+            default:
+                throw new IllegalArgumentException("Capacity not recognized: " + capacity);
         }
-        return cut;
+
+        return Integer.valueOf(cut);
     }
     
-    public int getHBaseMaxRegionCount() {
-        return Integer.valueOf(getOptional(HBASE_REGION_MAX_COUNT, "500"));
-    }
-
     public String getProperty(String key, String defaultValue) {
         return kylinConfig.getString(key, defaultValue);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index 0d1827e..0aa2898 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -71,13 +71,15 @@ kylin.job.hive.database.for.intermediatetable=default
 #default compression codec for htable,snappy,lzo,gzip,lz4
 kylin.hbase.default.compression.codec=snappy
 
-# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
-kylin.hbase.region.size.small=10
-kylin.hbase.region.size.medium=20
-kylin.hbase.region.size.large=100
-
-# Max region count when create HTable
-kylin.hbase.region.max.count=500
+# The cut size for hbase region, in GB.
+# 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
+
+# HBase min and max region count
+kylin.hbase.region.count.min=1
+kylin.hbase.region.count.min=500
 
 ## Config for Restful APP ##
 # database connection settings:

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/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 985f65e..b3ec4d0 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -59,13 +59,15 @@ kylin.job.hive.database.for.intermediatetable=default
 #default compression codec for htable,snappy,lzo,gzip,lz4
 kylin.hbase.default.compression.codec=gzip
 
-# Region cut size for SMALL, MEDIUM and LARGE cubes, in GB
-kylin.hbase.region.size.small=2
-kylin.hbase.region.size.medium=5
-kylin.hbase.region.size.large=10
-
-# Max region count when create HTable
-kylin.hbase.region.max.count=10
+# The cut size for hbase region, in GB.
+# 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
+
+# HBase min and max region count
+kylin.hbase.region.count.min=1
+kylin.hbase.region.count.min=500
 
 ## Config for Restful APP ##
 # database connection settings:

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
index 03eb2ef..ca8b2c9 100644
--- a/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
+++ b/job/src/main/java/org/apache/kylin/job/constant/BatchConstants.java
@@ -43,6 +43,8 @@ public interface BatchConstants {
 
     String MAPPER_SAMPLE_NUMBER = "mapper.sample.number";
     String REGION_NUMBER = "region.number";
+    String REGION_NUMBER_MIN = "region.number.min";
+    String REGION_NUMBER_MAX = "region.number.max";
     String REGION_SPLIT_SIZE = "region.split.size";
     String CUBE_CAPACITY = "cube.capacity";
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
index 2ca75a5..9c50122 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJob.java
@@ -32,6 +32,7 @@ import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.job.constant.BatchConstants;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
+import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -91,10 +92,13 @@ public class RangeKeyDistributionJob extends AbstractHadoopJob {
             String cubeName = getOptionValue(OPTION_CUBE_NAME).toUpperCase();
             CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
             CubeInstance cube = cubeMgr.getCube(cubeName);
-            int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionSizeGB(cube.getDescriptor().getModel().getCapacity().toString());
-            int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseMaxRegionCount();
+            DataModelDesc.RealizationCapacity cubeCapacity = cube.getDescriptor().getModel().getCapacity();
+            int regionSplitSize = KylinConfig.getInstanceFromEnv().getHBaseRegionCut(cubeCapacity.toString());
+            int maxRegionCount = KylinConfig.getInstanceFromEnv().getHBaseRegionCountMax();
+            int minRegionCount = KylinConfig.getInstanceFromEnv().getHBaseRegionCountMin();
             job.getConfiguration().set(BatchConstants.REGION_SPLIT_SIZE, String.valueOf(regionSplitSize));
-            job.getConfiguration().set(BatchConstants.REGION_NUMBER, String.valueOf(maxRegionCount));
+            job.getConfiguration().set(BatchConstants.REGION_NUMBER_MAX, String.valueOf(maxRegionCount));
+            job.getConfiguration().set(BatchConstants.REGION_NUMBER_MIN, String.valueOf(minRegionCount));
 
             return waitForCompletion(job);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fa925f70/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
index c97ce24..b3ab4db 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionReducer.java
@@ -43,6 +43,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
 
     private LongWritable outputValue = new LongWritable(0);
 
+    private int minRegionCount = 1;
     private int maxRegionCount = 500;
     private int cut = 10;
     private long bytesRead = 0;
@@ -56,11 +57,15 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
             cut = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_SPLIT_SIZE));
         }
 
-        if (context.getConfiguration().get(BatchConstants.REGION_NUMBER) != null) {
-            maxRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER));
+        if (context.getConfiguration().get(BatchConstants.REGION_NUMBER_MIN) != null) {
+            minRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER_MIN));
+        }
+        
+        if (context.getConfiguration().get(BatchConstants.REGION_NUMBER_MAX) != null) {
+            maxRegionCount = Integer.valueOf(context.getConfiguration().get(BatchConstants.REGION_NUMBER_MAX));
         }
 
-        logger.info("Chosen cut for htable is " + cut + ", max region count is " + maxRegionCount);
+        logger.info("Chosen cut for htable is " + cut + ", max region count=" + maxRegionCount + ", min region count =" + minRegionCount);
     }
 
     @Override
@@ -78,7 +83,7 @@ public class RangeKeyDistributionReducer extends KylinReducer<Text, LongWritable
     @Override
     protected void cleanup(Context context) throws IOException, InterruptedException {
         int nRegion = Math.round((float) gbPoints.size() / (float) cut);
-        nRegion = Math.max(1, nRegion);
+        nRegion = Math.max(minRegionCount, nRegion);
         nRegion = Math.min(maxRegionCount, nRegion);
 
         int gbPerRegion = gbPoints.size() / nRegion;


[36/50] [abbrv] incubator-kylin git commit: blog for "fast cubing on spark"

Posted by li...@apache.org.
blog for "fast cubing on spark"


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

Branch: refs/heads/master
Commit: fbe3395f29b2d872d062d4a0f3b8e9e9f011f1bc
Parents: 419798d
Author: qianhao.zhou <qi...@ebay.com>
Authored: Wed Sep 9 15:45:39 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Wed Sep 9 15:45:39 2015 +0800

----------------------------------------------------------------------
 .../blog/2015-09-09-fast-cubing-on-spark.md     | 112 +++++++++++++++++++
 1 file changed, 112 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/fbe3395f/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md b/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
new file mode 100644
index 0000000..fd9e56d
--- /dev/null
+++ b/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
@@ -0,0 +1,112 @@
+# Fast Cubing on Spark in Apache Kylin
+
+
+## Preparation
+
+In order to make POC phase as simple as possible, a standalone spark cluster is the best choice.
+So the environment setup is as below:
+
+1. hadoop sandbox (hortonworks hdp 2.2.0)
+	
+	(8 cores, 16G) * 1
+
+2. spark (1.4.1)
+
+	master:(4 cores, 8G)
+	
+	worker:(4 cores, 8G) * 2
+	
+The hadoop conf should also be in the SPARK_HOME/conf
+
+
+## Fast Cubing Implementation on Spark
+
+Spark as a computation framework has provided much richer operators than map-reduce. And some of them are quite suitable for the cubing algorithm, for instance **aggregate**.
+
+As the [Fast cubing algorithm](http://kylin.incubator.apache.org/blog/2015/08/15/fast-cubing/ "Fast Cubing Algorithm in Apache Kylin"), it contains several steps:
+
+1. build dictionary
+2. calculate region split for hbase
+3. build & output cuboid data
+
+----
+
+**build dictionary**
+
+In order to build dictionary, distinct values of the column are needed, which new API ***DataFrame*** has already provided(since spark 1.3.0).
+
+So after got the data from the hive through SparkSQL, it is quite natural to directly use the api to build dictionary.
+
+----
+
+**calculate region split**
+
+In order to calculate the distribution of all cuboids, Kylin use a HyperLogLog implementation. And each record will have a counter, whose size is by default 16KB each. So if the counter shuffles across the cluster, that will be very expensive.
+
+Spark has provided an operator ***aggregate*** to reduce shuffle size. It first does a map-reduce phase locally, and then another round of reduce to merge the data from each node.
+
+----
+
+**build & output cuboid data**
+
+In order to build cube, Kylin requires a small batch which can fit into memory in the same time.
+
+Previously in map-reduce implementation, Kylin leverage the life-cycle callback **cleanup** to gather all the input together as a batch. This cannot be directly applied in the map reduce operator in spark which we don't have such life-cycle callback.
+
+However spark has provided an operator ***glom*** which coalescing all elements within each partition into an array which is exactly Kylin want to build a small batch.
+
+Once the batch data is ready, we can just apply the Fast Cubing algorithm. 
+
+Then spark api ***saveAsNewAPIHadoopFile*** allow us to write hfile to hdfs and bulk load to HBase.
+
+
+## Statistics
+
+We use the sample data Kylin provided to build cube, total record count is 10000.
+
+Below are results(system environments are mentioned above)
+<table>
+    <tr>
+        <td></td>
+        <td>Spark</td>
+        <td>MR</td>
+    </tr>
+    <tr>
+        <td>Duration</td>
+        <td>5.5 min</td>
+        <td>10+ min</td>
+    </tr>
+</table>
+
+## Issues
+
+Since hdp 2.2+ requires Hive 0.14.0 while spark 1.3.0 only supports Hive 0.13.0. There are several compatibility problems in hive-site.xml we need to fix.
+
+1. some time-related settings
+
+    There are several settings, whose default value in hive 0.14.0 cannot be parsed in 0.13.0. Such as **hive.metastore.client.connect.retry.delay**, its default value is **5s**. And in hive 0.13.0, this value can only be in the format of Long value. So you have to manually change to from **5s** to **5**.
+
+2. hive.security.authorization.manager
+
+    If you have enabled this configuration, its default value is **org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdConfOnlyAuthorizerFactory** which is newly introduced in hive 0.14.0, it means you have to use the another implementation, such as **org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider**
+
+3. hive.execution.engine
+
+    In hive 0.14.0, the default value of **hive.execution.engine** is **tez**, change it to **mr** in the Spark classpath, otherwise there will be NoClassDefFoundError.
+
+NOTE: Spark 1.4.0 has a [bug](https://issues.apache.org/jira/browse/SPARK-8368) which will lead to ClassNotFoundException. And it has been fixed in Spark 1.4.1. So if you are planning to run on Spark 1.4.0, you may need to upgrade to 1.4.1
+
+Last but not least, when you trying to run Spark application on YARN, make sure that you have hive-site.xml and hbase-site.xml in the  HADDOP_CONF_DIR or YARN_CONF_DIR. Since by default HDP lays these conf in separate directories.
+
+
+
+
+## Next move
+
+Clearly above is not a fair competition. The environment is not the same, test data size is too small, etc.
+
+However it showed that it is practical to migrate from MR to Spark, while some useful operators in Spark will save us quite a few codes.
+
+So the next move for us is to setup a cluster, do the benchmark on real data set for both MR and Spark.
+
+We will update the benchmark once we finished, please stay tuned.
\ No newline at end of file


[12/50] [abbrv] incubator-kylin git commit: KYLIN-792, correct lowercase syntax for 'tblproperties' when create external hive table

Posted by li...@apache.org.
KYLIN-792, correct lowercase syntax for 'tblproperties' when create external hive table


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

Branch: refs/heads/master
Commit: d492d5d8cd6979a10d4bc8efe67410b604efd771
Parents: 25cbb2c
Author: jiazhong <ji...@ebay.com>
Authored: Wed Sep 2 16:16:46 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d492d5d8/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
index 185b717..2eda2e8 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/HiveJdbcClient.java
@@ -43,7 +43,7 @@ import javax.xml.parsers.ParserConfigurationException;
  */
 public class HiveJdbcClient {
 
-    static String SQL_GENERATE_QUERY_LOG_TABLE = "CREATE EXTERNAL TABLE IF NOT EXISTS [QUERY_LOG_TABLE_NAME] (REQUEST_TIME STRING,REQUEST_DATE DATE, QUERY_SQL STRING,QUERY_USER STRING,IS_SUCCESS STRING,QUERY_LATENCY DECIMAL(19,4),QUERY_PROJECT STRING,REALIZATION_NAMES STRING,CUBOID_IDS STRING,TOTAL_SCAN_COUNT INT,RESULT_ROW_COUNT INT,ACCEPT_PARTIAL STRING,IS_PARTIAL_RESULT STRING,HIT_CACHE STRING,MESSAGE STRING,DEPLOY_ENV STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' LINES TERMINATED BY '\\n' LOCATION '[QUERY_LOG_PARSE_RESULT_DIR]' TBLPROPERTIES (\"SKIP.HEADER.LINE.COUNT\"=\"1\")";
+    static String SQL_GENERATE_QUERY_LOG_TABLE = "CREATE EXTERNAL TABLE IF NOT EXISTS [QUERY_LOG_TABLE_NAME] (REQUEST_TIME STRING,REQUEST_DATE DATE, QUERY_SQL STRING,QUERY_USER STRING,IS_SUCCESS STRING,QUERY_LATENCY DECIMAL(19,4),QUERY_PROJECT STRING,REALIZATION_NAMES STRING,CUBOID_IDS STRING,TOTAL_SCAN_COUNT INT,RESULT_ROW_COUNT INT,ACCEPT_PARTIAL STRING,IS_PARTIAL_RESULT STRING,HIT_CACHE STRING,MESSAGE STRING,DEPLOY_ENV STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' LINES TERMINATED BY '\\n' LOCATION '[QUERY_LOG_PARSE_RESULT_DIR]' tblproperties (\"skip.header.line.count\"=\"1\")";
 
     static String SQL_TOTAL_QUERY_USER = "SELECT  COUNT(DISTINCT QUERY_USER) FROM [QUERY_LOG_TABLE_NAME]";
 


[10/50] [abbrv] incubator-kylin git commit: KYLIN-958 disallow changing data model in the backend

Posted by li...@apache.org.
KYLIN-958 disallow changing data model in the backend

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/4a4c719d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/4a4c719d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/4a4c719d

Branch: refs/heads/master
Commit: 4a4c719d01fb2c5c38538c4f636ffc3282b4c78f
Parents: 0bc870d
Author: gaodayue <ga...@meituan.com>
Authored: Thu Aug 27 14:00:25 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

----------------------------------------------------------------------
 .../common/persistence/HBaseResourceStore.java  |   2 +-
 .../org/apache/kylin/cube/CubeDescManager.java  |  14 +-
 .../kylin/metadata/model/DataModelDesc.java     |  41 +++++
 .../kylin/rest/controller/CubeController.java   |  76 +++++----
 .../apache/kylin/rest/service/CubeService.java  |   9 +-
 .../rest/controller/CubeControllerTest.java     | 171 ++++++++++++++-----
 6 files changed, 228 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index 37b8f8d..1c4a7ba 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -237,7 +237,7 @@ public class HBaseResourceStore extends ResourceStore {
             boolean ok = table.checkAndPut(row, B_FAMILY, B_COLUMN_TS, bOldTS, put);
             if (!ok) {
                 long real = getResourceTimestamp(resPath);
-                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
+                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
             }
 
             table.flushCommits();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java b/cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
index cd270cf..dfb1b88 100644
--- a/cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
+++ b/cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -253,6 +253,14 @@ public class CubeDescManager {
 
         desc.setSignature(desc.calculateSignature());
 
+        // drop cube segments if signature changes
+        CubeInstance cube = getCubeManager().getCube(desc.getName());
+        if (cube != null && !StringUtils.equals(desc.getSignature(), cube.getDescriptor().getSignature())) {
+            logger.info("Detect signature change of [" + desc.getName() + "], drop all existing segments");
+            cube.getSegments().clear();
+            getCubeManager().updateCube(cube);
+        }
+
         // Save Source
         String path = desc.getResourcePath();
         getStore().putResource(path, desc, CUBE_DESC_SERIALIZER);
@@ -269,6 +277,10 @@ public class CubeDescManager {
         return MetadataManager.getInstance(config);
     }
 
+    private CubeManager getCubeManager() {
+        return CubeManager.getInstance(config);
+    }
+
     private ResourceStore getStore() {
         return ResourceStore.getStore(this.config);
     }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java b/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
index a37d4c6..cb1e784 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/model/DataModelDesc.java
@@ -18,14 +18,17 @@
 
 package org.apache.kylin.metadata.model;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
+import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 
@@ -33,9 +36,12 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class DataModelDesc extends RootPersistentEntity {
+    private static final Logger logger = LoggerFactory.getLogger(DataModelDesc.class);
 
     public static enum RealizationCapacity {
         SMALL, MEDIUM, LARGE
@@ -213,6 +219,41 @@ public class DataModelDesc extends RootPersistentEntity {
         }
     }
 
+    /**
+     * Check whether two data model are compatible or not. Compatible means
+     * having the same structure. Tow models could be compatible even they
+     * have different UUID or last modified time.
+     * @param that model to compare with
+     * @return true if compatible, false otherwise.
+     */
+    public boolean compatibleWith(DataModelDesc that) {
+        if (this == that)
+            return true;
+
+        if (that == null)
+            return false;
+
+        try {
+            String thisRepr = excludeHeaderInfo(this);
+            String thatRepr = excludeHeaderInfo(that);
+            return StringUtils.equals(thisRepr, thatRepr);
+
+        } catch (IOException e) {
+            logger.error("Failed to serialize DataModelDesc to string", e);
+            return false;
+        }
+    }
+
+    private String excludeHeaderInfo(DataModelDesc modelDesc) throws IOException {
+        // make a copy
+        String repr = JsonUtil.writeValueAsString(modelDesc);
+        DataModelDesc copy = JsonUtil.readValue(repr, DataModelDesc.class);
+
+        copy.setUuid(null);
+        copy.setLastModified(0);
+        return JsonUtil.writeValueAsString(copy);
+    }
+
     @Override
     public String toString() {
         return "DataModelDesc [name=" + name + "]";

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/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 9f29753..734ef32 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
@@ -330,56 +330,57 @@ public class CubeController extends BasicController {
     }
 
     /**
-     * Get available table list of the input database
+     * Update cube description. If cube signature has changed, all existing cube segments are dropped.
      *
      * @return Table metadata array
      * @throws JsonProcessingException
-     * @throws IOException
      */
     @RequestMapping(value = "", method = { RequestMethod.PUT })
     @ResponseBody
     public CubeRequest updateCubeDesc(@RequestBody CubeRequest cubeRequest) throws JsonProcessingException {
-
-        //Update Model 
-        MetadataManager metaManager = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        DataModelDesc modelDesc = deserializeDataModelDesc(cubeRequest);
-        if (modelDesc == null) {
+        CubeDesc desc = deserializeCubeDesc(cubeRequest);
+        if (desc == null) {
             return cubeRequest;
         }
-        try {
 
-            DataModelDesc existingModel = metaManager.getDataModelDesc(modelDesc.getName());
-            if (existingModel == null) {
-                metaManager.createDataModelDesc(modelDesc);
-            } else {
+        final String cubeName = desc.getName();
+        if (StringUtils.isEmpty(cubeName)) {
+            return errorRequest(cubeRequest, "Missing cubeName");
+        }
 
-                //ignore overwriting conflict checking before splict MODEL & CUBE
-                modelDesc.setLastModified(existingModel.getLastModified());
-                metaManager.updateDataModelDesc(modelDesc);
+        MetadataManager metadataManager = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+        // KYLIN-958: disallow data model change
+        if (StringUtils.isNotEmpty(cubeRequest.getModelDescData())) {
+            DataModelDesc modelDesc = deserializeDataModelDesc(cubeRequest);
+            if (modelDesc == null) {
+                return cubeRequest;
             }
 
-        } catch (IOException e) {
-            // TODO Auto-generated catch block
-            logger.error("Failed to deal with the request:" + e.getLocalizedMessage(), e);
-            throw new InternalErrorException("Failed to deal with the request: " + e.getLocalizedMessage());
-        }
+            final String modeName = modelDesc.getName();
 
-        //update cube
-        CubeDesc desc = deserializeCubeDesc(cubeRequest);
+            if (!StringUtils.equals(desc.getModelName(), modeName)) {
+                return errorRequest(cubeRequest, "CubeDesc.model_name " + desc.getModelName() + " not consistent with model " + modeName);
+            }
+
+            DataModelDesc oldModelDesc = metadataManager.getDataModelDesc(modeName);
+            if (oldModelDesc == null) {
+                return errorRequest(cubeRequest, "Data model " + modeName + " not found");
+            }
+
+            if (!modelDesc.compatibleWith(oldModelDesc)) {
+                return errorRequest(cubeRequest, "Update data model is not allowed! Please create a new cube if needed");
+            }
 
-        if (desc == null) {
-            return cubeRequest;
         }
 
         // Check if the cube is editable
         if (!cubeService.isCubeDescEditable(desc)) {
             String error = "Cube desc " + desc.getName().toUpperCase() + " is not editable.";
-            updateRequest(cubeRequest, false, error);
-            return cubeRequest;
+            return errorRequest(cubeRequest, error);
         }
 
         try {
-            CubeInstance cube = cubeService.getCubeManager().getCube(cubeRequest.getCubeName());
+            CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
             cube.setRetentionRange(desc.getRetentionRange());
             String projectName = (null == cubeRequest.getProject()) ? ProjectInstance.DEFAULT_PROJECT_NAME : cubeRequest.getProject();
             desc = cubeService.updateCubeAndDesc(cube, desc, projectName);
@@ -395,7 +396,7 @@ public class CubeController extends BasicController {
             cubeRequest.setSuccessful(true);
         } else {
             logger.warn("Cube " + desc.getName() + " fail to create because " + desc.getError());
-            updateRequest(cubeRequest, false, omitMessage(desc.getError()));
+            errorRequest(cubeRequest, omitMessage(desc.getError()));
         }
         String descData = JsonUtil.writeValueAsIndentString(desc);
         cubeRequest.setCubeDescData(descData);
@@ -449,15 +450,15 @@ public class CubeController extends BasicController {
     private CubeDesc deserializeCubeDesc(CubeRequest cubeRequest) {
         CubeDesc desc = null;
         try {
-            logger.debug("Saving cube " + cubeRequest.getCubeDescData());
+            logger.debug("Deserialize cube desc " + cubeRequest.getCubeDescData());
             desc = JsonUtil.readValue(cubeRequest.getCubeDescData(), CubeDesc.class);
             //            desc.setRetentionRange(cubeRequest.getRetentionRange());
         } catch (JsonParseException e) {
             logger.error("The cube definition is not valid.", e);
-            updateRequest(cubeRequest, false, e.getMessage());
+            errorRequest(cubeRequest, e.getMessage());
         } catch (JsonMappingException e) {
             logger.error("The cube definition is not valid.", e);
-            updateRequest(cubeRequest, false, e.getMessage());
+            errorRequest(cubeRequest, e.getMessage());
         } catch (IOException e) {
             logger.error("Failed to deal with the request.", e);
             throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
@@ -468,14 +469,14 @@ public class CubeController extends BasicController {
     private DataModelDesc deserializeDataModelDesc(CubeRequest cubeRequest) {
         DataModelDesc desc = null;
         try {
-            logger.debug("Saving MODEL " + cubeRequest.getModelDescData());
+            logger.debug("Deserialize data model " + cubeRequest.getModelDescData());
             desc = JsonUtil.readValue(cubeRequest.getModelDescData(), DataModelDesc.class);
         } catch (JsonParseException e) {
             logger.error("The data model definition is not valid.", e);
-            updateRequest(cubeRequest, false, e.getMessage());
+            errorRequest(cubeRequest, e.getMessage());
         } catch (JsonMappingException e) {
             logger.error("The data model definition is not valid.", e);
-            updateRequest(cubeRequest, false, e.getMessage());
+            errorRequest(cubeRequest, e.getMessage());
         } catch (IOException e) {
             logger.error("Failed to deal with the request.", e);
             throw new InternalErrorException("Failed to deal with the request:" + e.getMessage(), e);
@@ -496,10 +497,11 @@ public class CubeController extends BasicController {
         return buffer.toString();
     }
 
-    private void updateRequest(CubeRequest request, boolean success, String message) {
+    private CubeRequest errorRequest(CubeRequest request, String errmsg) {
         request.setCubeDescData("");
-        request.setSuccessful(success);
-        request.setMessage(message);
+        request.setSuccessful(false);
+        request.setMessage(errmsg);
+        return request;
     }
 
     public void setCubeService(CubeService cubeService) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/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 21c6ec7..be356af 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
@@ -236,11 +236,11 @@ public class CubeService extends BasicService {
         }
 
         try {
-            if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
-                this.releaseAllSegments(cube);
+            CubeDesc updatedCubeDesc = getCubeDescManager().updateCubeDesc(desc);
+            if (!updatedCubeDesc.getError().isEmpty()) {
+                return updatedCubeDesc;
             }
 
-            CubeDesc updatedCubeDesc = getCubeDescManager().updateCubeDesc(desc);
             cube = getCubeManager().updateCube(cube);
 
             int cuboidCount = CuboidCLI.simulateCuboidGeneration(updatedCubeDesc);
@@ -388,9 +388,6 @@ public class CubeService extends BasicService {
         if (!cubingJobs.isEmpty()) {
             throw new JobException("Enable is not allowed with a running job.");
         }
-        if (!cube.getDescriptor().calculateSignature().equals(cube.getDescriptor().getSignature())) {
-            this.releaseAllSegments(cube);
-        }
 
         cube.setStatus(RealizationStatusEnum.READY);
         try {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4a4c719d/server/src/test/java/org/apache/kylin/rest/controller/CubeControllerTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/controller/CubeControllerTest.java b/server/src/test/java/org/apache/kylin/rest/controller/CubeControllerTest.java
index cf1a718..2f12851 100644
--- a/server/src/test/java/org/apache/kylin/rest/controller/CubeControllerTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/controller/CubeControllerTest.java
@@ -19,32 +19,34 @@
 package org.apache.kylin.rest.controller;
 
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.List;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.kylin.common.util.JsonUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyColDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.rest.request.CubeRequest;
 import org.apache.kylin.rest.service.CubeService;
 import org.apache.kylin.rest.service.JobService;
 import org.apache.kylin.rest.service.ServiceTestBase;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;
 import org.springframework.beans.factory.annotation.Autowired;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.Lists;
 
 /**
  * @author xduo
  */
 public class CubeControllerTest extends ServiceTestBase {
+    private static final String SRC_CUBE_NAME = "test_kylin_cube_with_slr_ready";
+    private static final String TEST_CUBE_NAME = SRC_CUBE_NAME + "_test_save";
 
     private CubeController cubeController;
     private CubeDescController cubeDescController;
     private ModelController modelController;
+    private CubeDesc srcCubeDesc;
 
     @Autowired
     CubeService cubeService;
@@ -52,29 +54,39 @@ public class CubeControllerTest extends ServiceTestBase {
     JobService jobService;
 
     @Before
-    public void setup() throws Exception {
+    public void setUp() throws Exception {
         super.setUp();
-
         cubeController = new CubeController();
         cubeController.setCubeService(cubeService);
         cubeController.setJobService(jobService);
+
         cubeDescController = new CubeDescController();
         cubeDescController.setCubeService(cubeService);
 
         modelController = new ModelController();
         modelController.setCubeService(cubeService);
+
+        srcCubeDesc = getCubeDescByName(SRC_CUBE_NAME);
+
+        saveTestCube(TEST_CUBE_NAME);
     }
 
-    @Test
-    public void testBasics() throws IOException {
-        CubeDesc[] cubes = (CubeDesc[]) cubeDescController.getCube("test_kylin_cube_with_slr_ready");
-        Assert.assertNotNull(cubes);
-        Assert.assertNotNull(cubeController.getSql("test_kylin_cube_with_slr_ready", "20130331080000_20131212080000"));
-        Assert.assertNotNull(cubeController.getCubes(null, null, 0, 5));
+    @After
+    public void tearDown() throws Exception {
+        cubeController.deleteCube(TEST_CUBE_NAME);
+        super.after();
+    }
+
+    private CubeDesc getCubeDescByName(String cubeDescName) {
+        CubeDesc[] cubes = cubeDescController.getCube(cubeDescName);
+        if (cubes == null || cubes.length < 1) {
+            throw new IllegalStateException("cube desc " + cubeDescName + " not existed");
+        }
+        return cubes[0];
+    }
 
-        CubeDesc cube = cubes[0];
+    private void saveTestCube(final String newCubeName) throws IOException {
         CubeDesc newCube = new CubeDesc();
-        String newCubeName = cube.getName() + "_test_save";
 
         try {
             cubeController.deleteCube(newCubeName);
@@ -83,45 +95,124 @@ public class CubeControllerTest extends ServiceTestBase {
         }
 
         newCube.setName(newCubeName);
-        newCube.setModelName(cube.getModelName());
-        newCube.setModel(cube.getModel());
-        newCube.setDimensions(cube.getDimensions());
-        newCube.setHBaseMapping(cube.getHBaseMapping());
-        newCube.setMeasures(cube.getMeasures());
-        newCube.setConfig(cube.getConfig());
-        newCube.setRowkey(cube.getRowkey());
-
-        String newModelName = newCubeName + "_model_desc";
-        newCube.getModel().setName(newModelName);//generate a random model
+        newCube.setModelName(newCubeName);
+        newCube.setModel(srcCubeDesc.getModel());
+        newCube.setDimensions(srcCubeDesc.getDimensions());
+        newCube.setHBaseMapping(srcCubeDesc.getHBaseMapping());
+        newCube.setMeasures(srcCubeDesc.getMeasures());
+        newCube.setConfig(srcCubeDesc.getConfig());
+        newCube.setRowkey(srcCubeDesc.getRowkey());
+
+        newCube.getModel().setName(newCubeName);
         newCube.getModel().setLastModified(0);
 
-        ObjectMapper cubeDescMapper = new ObjectMapper();
-        StringWriter cubeDescWriter = new StringWriter();
-        cubeDescMapper.writeValue(cubeDescWriter, newCube);
+        CubeRequest cubeRequest = new CubeRequest();
+        cubeRequest.setCubeDescData(JsonUtil.writeValueAsIndentString(newCube));
+        cubeRequest.setModelDescData(JsonUtil.writeValueAsIndentString(newCube.getModel()));
 
-        ObjectMapper modelDescMapper = new ObjectMapper();
-        StringWriter modelDescWriter = new StringWriter();
-        modelDescMapper.writeValue(modelDescWriter, newCube.getModel());
+        CubeRequest result = cubeController.saveCubeDesc(cubeRequest);
+        Assert.assertTrue(result.getSuccessful());
+    }
 
-        CubeRequest cubeRequest = new CubeRequest();
-        cubeRequest.setCubeDescData(cubeDescWriter.toString());
-        cubeRequest.setModelDescData(modelDescWriter.toString());
-        cubeRequest = cubeController.saveCubeDesc(cubeRequest);
+    @Test
+    public void testBasics() throws IOException {
+
+        Assert.assertNotNull(cubeController.getSql(SRC_CUBE_NAME, "20130331080000_20131212080000"));
+        Assert.assertNotNull(cubeController.getCubes(null, null, 0, 5));
 
-        DataModelDesc model = modelController.getModel(newModelName);
+        DataModelDesc model = modelController.getModel(TEST_CUBE_NAME);
         Assert.assertNotNull(model);
 
         List<String> notifyList = Lists.newArrayList();
         notifyList.add("john@example.com");
-        cubeController.updateNotifyList(newCubeName, notifyList);
-        cubeController.updateCubeCost(newCubeName, 80);
+        cubeController.updateNotifyList(TEST_CUBE_NAME, notifyList);
+        cubeController.updateCubeCost(TEST_CUBE_NAME, 80);
 
-        List<CubeInstance> cubeInstances = cubeController.getCubes(newCubeName, "default", 1, 0);
+        List<CubeInstance> cubeInstances = cubeController.getCubes(TEST_CUBE_NAME, "default", 1, 0);
 
         CubeInstance cubeInstance = cubeInstances.get(0);
         Assert.assertTrue(cubeInstance.getDescriptor().getNotifyList().contains("john@example.com"));
         Assert.assertTrue(cubeInstance.getCost() == 80);
-        cubeController.deleteCube(newCubeName);
     }
 
+    @Test
+    public void testUpdateCubeDesc() throws IOException {
+        CubeDesc newCubeDesc = getCubeDescByName(TEST_CUBE_NAME);
+
+        // -------------------------------------------------------
+        // negative case
+        // -------------------------------------------------------
+
+        // invalid cube desc
+        CubeRequest req = new CubeRequest();
+        req.setCubeDescData("invalid");
+        assertUpdateFail(req);
+
+        // invalid data model
+        req = new CubeRequest();
+        req.setCubeDescData(JsonUtil.writeValueAsIndentString(newCubeDesc));
+        req.setModelDescData("invalid");
+        assertUpdateFail(req);
+
+        // data model's model_name not consistent with model name
+        req = new CubeRequest();
+        req.setCubeDescData("{\"name\" : \"myCube\", \"model_name\" : \"anotherModelName\"}");
+        req.setModelDescData("{\"name\" : \"myCube\"}");
+        assertUpdateFail(req);
+
+        // non-existed data model
+        req = new CubeRequest();
+        req.setCubeDescData("{\"name\" : \"noSuchCube\", \"model_name\" : \"noSuchModel\"}");
+        req.setModelDescData("{\"name\" : \"noSuchModel\"}");
+        assertUpdateFail(req);
+
+        // modified data model
+        req = new CubeRequest();
+        req.setCubeDescData(JsonUtil.writeValueAsIndentString(newCubeDesc));
+
+        DataModelDesc modifiedModel = new DataModelDesc();
+        modifiedModel.setName(TEST_CUBE_NAME);
+        modifiedModel.setFactTable("anotherFactTable");
+        req.setModelDescData(JsonUtil.writeValueAsIndentString(modifiedModel));
+
+        assertUpdateFail(req);
+
+        // -------------------------------------------------------
+        // positive case
+        // -------------------------------------------------------
+        req = new CubeRequest();
+        req.setModelDescData(JsonUtil.writeValueAsIndentString(newCubeDesc.getModel()));
+
+        // no signature change
+        newCubeDesc.setDescription("hello cube");
+        req.setCubeDescData(JsonUtil.writeValueAsIndentString(newCubeDesc));
+        CubeRequest res = cubeController.updateCubeDesc(req);
+        Assert.assertTrue(res.getSuccessful());
+
+        CubeDesc resultDesc = getCubeDescByName(TEST_CUBE_NAME);
+        Assert.assertEquals("hello cube", resultDesc.getDescription());
+        Assert.assertEquals(newCubeDesc.getSignature(), resultDesc.getSignature());
+
+        // signature change (reverse row key column order)
+        newCubeDesc = getCubeDescByName(TEST_CUBE_NAME);
+        RowKeyColDesc[] rowkeyColumns = newCubeDesc.getRowkey().getRowKeyColumns();
+        for (int i = 0, j = rowkeyColumns.length - 1; i < j; i++, j--) {
+            RowKeyColDesc tmp = rowkeyColumns[i];
+            rowkeyColumns[i] = rowkeyColumns[j];
+            rowkeyColumns[j] = tmp;
+        }
+        req.setCubeDescData(JsonUtil.writeValueAsIndentString(newCubeDesc));
+        res = cubeController.updateCubeDesc(req);
+        Assert.assertTrue(res.getSuccessful());
+
+        resultDesc = getCubeDescByName(TEST_CUBE_NAME);
+        ;
+        Assert.assertNotEquals(newCubeDesc.getSignature(), resultDesc.getSignature());
+        Assert.assertEquals(newCubeDesc.calculateSignature(), resultDesc.getSignature());
+    }
+
+    private void assertUpdateFail(CubeRequest req) throws JsonProcessingException {
+        CubeRequest res = cubeController.updateCubeDesc(req);
+        Assert.assertFalse(res.getSuccessful());
+    }
 }


[42/50] [abbrv] incubator-kylin git commit: KYLIN-910, enable edit 'auto merge time range' on UI

Posted by li...@apache.org.
KYLIN-910, enable edit 'auto merge time range' on UI


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

Branch: refs/heads/master
Commit: e0f5e3983b127649419cbe33c27e647b8e84029f
Parents: 1e9646f
Author: jiazhong <ji...@ebay.com>
Authored: Tue Sep 15 16:46:19 2015 +0800
Committer: jiazhong <ji...@ebay.com>
Committed: Tue Sep 15 16:46:19 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/CubeInstance.java     |  23 +----
 .../java/org/apache/kylin/cube/CubeManager.java |   2 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |  20 ++--
 .../kylin/rest/controller/CubeController.java   |   1 -
 .../rest/controller/CubeDescController.java     |   1 -
 .../apache/kylin/rest/request/CubeRequest.java  |   8 --
 .../apache/kylin/rest/service/CubeService.java  |   5 +-
 webapp/app/js/controllers/cubeAdvanceSetting.js |  57 +++++++++++
 webapp/app/js/filters/filter.js                 |  10 ++
 webapp/app/js/model/cubeDescModel.js            |   2 +-
 .../cubeDesigner/advanced_settings.html         | 102 +++++++++++++++----
 11 files changed, 167 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index e286c2b..81a64e8 100644
--- a/cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -55,7 +55,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
         cubeInstance.setStatus(RealizationStatusEnum.DISABLED);
         cubeInstance.updateRandomUuid();
         cubeInstance.setProjectName(projectName);
-        cubeInstance.setRetentionRange(cubeDesc.getRetentionRange());
 
         return cubeInstance;
     }
@@ -83,12 +82,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;
-
     private String projectName;
 
     public List<CubeSegment> getBuildingSegments() {
@@ -355,13 +348,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
         this.projectName = projectName;
     }
 
-    public long[] getAutoMergeTimeRanges() {
-        return autoMergeTimeRanges;
-    }
-
-    public void setAutoMergeTimeRanges(long[] autoMergeTimeRanges) {
-        this.autoMergeTimeRanges = autoMergeTimeRanges;
-    }
 
     public boolean needAutoMerge() {
         if (!this.getDescriptor().getModel().getPartitionDesc().isPartitioned())
@@ -370,16 +356,9 @@ public class CubeInstance extends RootPersistentEntity implements IRealization {
         if (this.getDescriptor().hasHolisticCountDistinctMeasures())
             return false;
 
-        return autoMergeTimeRanges != null && autoMergeTimeRanges.length > 0;
-    }
-
-    public long getRetentionRange() {
-        return retentionRange;
+        return this.getDescriptor().getAutoMergeTimeRanges() != null && this.getDescriptor().getAutoMergeTimeRanges().length > 0;
     }
 
-    public void setRetentionRange(long retentionRange) {
-        this.retentionRange = retentionRange;
-    }
 
     @Override
     public long getDateRangeStart() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index e4c0e4a..b3eeeef 100644
--- a/cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -676,7 +676,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/e0f5e398/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 72280ea..7f9252c 100644
--- a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -55,7 +55,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,10 +114,10 @@ public class CubeDesc extends RootPersistentEntity {
     private String signature;
     @JsonProperty("notify_list")
     private List<String> notifyList;
-    /*
-     * 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>();
@@ -768,13 +767,20 @@ public class CubeDesc extends RootPersistentEntity {
         this.nullStrings = nullStrings;
     }
 
-    @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;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/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 734ef32..3d0fca3 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
@@ -381,7 +381,6 @@ public class CubeController extends BasicController {
 
         try {
             CubeInstance cube = cubeService.getCubeManager().getCube(cubeName);
-            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/e0f5e398/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 ad22bf1..76da4ea 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
@@ -57,7 +57,6 @@ public class CubeDescController {
             return null;
         }
         CubeDesc cSchema = cubeInstance.getDescriptor();
-        cSchema.setRetentionRange(cubeInstance.getRetentionRange());
         if (cSchema != null) {
             return new CubeDesc[] { cSchema };
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/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 8af963c..e5acbe4 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
@@ -28,7 +28,6 @@ public class CubeRequest {
     private String message;
     private String cubeDescName;
     private String project;
-    private long retentionRange;
 
     public String getUuid() {
         return uuid;
@@ -131,11 +130,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/e0f5e398/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 be356af..fdfd136 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
@@ -574,7 +574,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);
@@ -582,7 +583,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/e0f5e398/webapp/app/js/controllers/cubeAdvanceSetting.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeAdvanceSetting.js b/webapp/app/js/controllers/cubeAdvanceSetting.js
index 7e3d346..742490d 100644
--- a/webapp/app/js/controllers/cubeAdvanceSetting.js
+++ b/webapp/app/js/controllers/cubeAdvanceSetting.js
@@ -30,6 +30,30 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal, cubeConf
   //edit model
   if ($scope.state.mode === "edit") {
     $scope.metaModel = MetaModel;
+    if(!$scope.cubeMetaFrame.auto_merge_time_ranges){
+      $scope.cubeMetaFrame.auto_merge_time_ranges = [604800000,2419200000];
+    }
+    $scope._auto_merge_time_ranges = [];
+    angular.forEach($scope.cubeMetaFrame.auto_merge_time_ranges,function(item){
+      var _day = Math.floor(item/86400000);
+      var _hour = (item%86400000)/3600000;
+
+      var rangeObj = {
+        type:'days',
+        range:0,
+        mills:0
+      }
+      if(_day==0){
+        rangeObj.type = 'hours';
+        rangeObj.range = _hour;
+        rangeObj.mills = rangeObj.range*3600000;
+      }else{
+        rangeObj.type = 'days';
+        rangeObj.range = _day;
+        rangeObj.mills = rangeObj.range*86400000;
+      }
+      $scope._auto_merge_time_ranges.push(rangeObj);
+    })
   }
 
 
@@ -37,6 +61,39 @@ KylinApp.controller('CubeAdvanceSettingCtrl', function ($scope, $modal, cubeConf
     if (rowkey_column.dictionary === "true") {
       rowkey_column.length = 0;
     }
+  }
+
+
+  $scope.addNewMergeTimeRange = function(){
+    $scope._auto_merge_time_ranges.push({
+      type:'days',
+      range:0,
+      mills:0
+    })
+    $scope.updateAutoMergeRange();
+  }
+
+  $scope.removeTimeRange = function(arr,index,item){
+    if (index > -1) {
+      arr.splice(index, 1);
+    }
+    $scope.cubeMetaFrame.auto_merge_time_ranges.splice(index,1);
+  }
+
+
+  $scope.refreshAutoMergeTimeRanges = function(list,index,item){
+    if(item.type=='hours'){
+      item.mills = item.range*3600000;
+    }else{
+      item.mills = item.range*86400000;
+    }
+    $scope.cubeMetaFrame.auto_merge_time_ranges[index] = item.mills;
+  }
 
+  $scope.updateAutoMergeRange = function(){
+    $scope.cubeMetaFrame.auto_merge_time_ranges = [];
+    angular.forEach($scope._auto_merge_time_ranges,function(item){
+      $scope.cubeMetaFrame.auto_merge_time_ranges.push(item.mills);
+    })
   }
 });

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/webapp/app/js/filters/filter.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/filters/filter.js b/webapp/app/js/filters/filter.js
index f5d4750..2cf8114 100755
--- a/webapp/app/js/filters/filter.js
+++ b/webapp/app/js/filters/filter.js
@@ -163,4 +163,14 @@ KylinApp
     return function (item) {
       return item/86400000;
     }
+  }).filter('timeRangeFormat', function ($filter) {
+    return function (item) {
+      var _day = Math.floor(item/86400000);
+      var _hour = (item%86400000)/3600000;
+      if(_day==0){
+        return _hour +" (Hours)"
+      }else{
+        return _day +" (Days)";
+      }
+    }
   });

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/webapp/app/js/model/cubeDescModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeDescModel.js b/webapp/app/js/model/cubeDescModel.js
index 7d20ce7..b10d1e6 100644
--- a/webapp/app/js/model/cubeDescModel.js
+++ b/webapp/app/js/model/cubeDescModel.js
@@ -49,7 +49,7 @@ KylinApp.service('CubeDescModel', function () {
       "hbase_mapping": {
         "column_family": []
       },
-      "retentionRange":"0"
+      "retention_range":"0"
     };
 
     return cubeMeta;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/e0f5e398/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 e5e83f2..02a83fd 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -21,26 +21,86 @@
 
     <div class="form-group">
 
-        <table style="margin-left:42px; width:92%"
-               class="table table-hover list">
-            <tr>
-                <td>
-                    <!--Cube Size-->
-                    <div class="form-group" ng-show="userService.hasRole('ROLE_ADMIN')">
-                        <div class="row">
-                            <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Cube Size</b></label>
-                            <div class="col-xs-12 col-sm-6">
-                                <select class="form-control" ng-if="state.mode=='edit'" style="width: 360px;"
-                                        chosen ng-model="metaModel.model.capacity" required
-                                        ng-options="cc as cc for cc in cubeConfig.cubeCapacities">
-                                    <option value=""></option>
-                                </select>
-                                <span ng-if="state.mode=='view'">{{metaModel.model.capacity}}</span>
-                            </div>
-                        </div>
+        <table style="margin-left:42px; width:92%" class="table table-hover list">
+          <tr>
+              <td>
+                  <!--Cube Size-->
+                  <div class="form-group" ng-show="userService.hasRole('ROLE_ADMIN')">
+                      <div class="row">
+                          <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Cube Size</b></label>
+                          <div class="col-xs-12 col-sm-6">
+                              <select class="form-control" ng-if="state.mode=='edit'" style="width: 360px;"
+                                      chosen ng-model="metaModel.model.capacity" required
+                                      ng-options="cc as cc for cc in cubeConfig.cubeCapacities">
+                                  <option value=""></option>
+                              </select>
+                              <span ng-if="state.mode=='view'">{{metaModel.model.capacity}}</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>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'">
+                      <table class="table">
+                        <tbody>
+                        <tr ng-repeat="timeRange in _auto_merge_time_ranges">
+                          <td class="col-xs-4">
+                            <select  ng-if="timeRange.type=='hours'"
+                                     chosen ng-model="timeRange.range"
+                                     ng-options="rangeTypes as rangeTypes for rangeTypes in [0.5,1,2,4,8] "
+                                     ng-change="refreshAutoMergeTimeRanges(_auto_merge_time_ranges,$index,timeRange)"
+                                     style="width: 100% !important;"
+                                     data-placeholder="hours range"
+                                     class="chosen-select">
+                            </select>
+                            <input type="text" ng-if="timeRange.type=='days'"  ng-change="refreshAutoMergeTimeRanges(_auto_merge_time_ranges,$index,timeRange)" class="form-control input-sm" placeholder="days range" ng-model="timeRange.range">
+                          </td>
+                          <td class="col-xs-4">
+                            <select  chosen ng-model="timeRange.type"
+                                     ng-options="rangeTypes as rangeTypes for rangeTypes in ['hours','days'] "
+                                     ng-change="refreshAutoMergeTimeRanges(_auto_merge_time_ranges,$index,timeRange)"
+                                     style="width: 200px !important;"
+                                     data-placeholder="select a project"
+                                     class="chosen-select">
+                            </select>
+                          </td>
+                          <td>
+                            <button class="btn btn-xs btn-info" ng-show="state.mode=='edit'" ng-click="removeTimeRange(_auto_merge_time_ranges,$index, timeRange)">
+                              <i class="fa fa-minus"></i>
+                            </button>
+                          </td>
+                        </tr>
+                        </tbody>
+                      </table>
+                      <button class="btn btn-xs btn-info" ng-show="state.mode=='edit'" ng-click="addNewMergeTimeRange()">
+                        New Merge Range<i class="fa fa-plus"></i>
+                      </button>
                     </div>
-                </td>
-            </tr>
+                    <div class="row"  ng-if="state.mode=='view'">
+                      <table class="table">
+                        <tbody>
+                        <tr ng-repeat="timeRange in cubeMetaFrame.auto_merge_time_ranges">
+                          <td>
+                            {{timeRange | timeRangeFormat}}
+                          </td>
+                        </tr>
+                        </tbody>
+                      </table>
+                    </div>
+
+                  </div>
+                </div>
+              </div>
+            </td>
+          </tr>
           <tr>
             <td>
               <!--Cube Size-->
@@ -49,9 +109,9 @@
                   <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>


[41/50] [abbrv] incubator-kylin git commit: minor, fix test case BadQueryDetectorTest

Posted by li...@apache.org.
minor, fix test case BadQueryDetectorTest


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

Branch: refs/heads/master
Commit: 1e9646fc956d1acaf1d28bcc473ae1335b544236
Parents: c0ef677
Author: Li, Yang <ya...@ebay.com>
Authored: Mon Sep 14 10:56:23 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Mon Sep 14 10:58:02 2015 +0800

----------------------------------------------------------------------
 .../kylin/rest/service/BadQueryDetectorTest.java      | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1e9646fc/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
index b38ee9d..c849efd 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/BadQueryDetectorTest.java
@@ -35,7 +35,7 @@ public class BadQueryDetectorTest {
         String mockSql = "select * from just_a_test";
         final ArrayList<String[]> alerts = new ArrayList<>();
 
-        BadQueryDetector badQueryDetector = new BadQueryDetector(5000, alertMB, alertRunningSec);
+        BadQueryDetector badQueryDetector = new BadQueryDetector(alertRunningSec * 1000, alertMB, alertRunningSec);
         badQueryDetector.registerNotifier(new BadQueryDetector.Notifier() {
             @Override
             public void badQueryFound(String adj, int runningSec, String sql) {
@@ -45,19 +45,25 @@ public class BadQueryDetectorTest {
         badQueryDetector.start();
 
         {
+            Thread.sleep(1000);
+            
             SQLRequest sqlRequest = new SQLRequest();
             sqlRequest.setSql(mockSql);
             badQueryDetector.queryStart(Thread.currentThread(), sqlRequest);
 
-            Thread.sleep(alertRunningSec * 2 * 1000);
+            // make sure bad query check happens twice
+            Thread.sleep((alertRunningSec * 2 + 1) * 1000);
 
             badQueryDetector.queryEnd(Thread.currentThread());
         }
 
         badQueryDetector.stop();
 
-        assertEquals(2, alerts.size());
+        assertEquals(3, alerts.size());
+        // first check founds Low mem
         assertArrayEquals(new String[] { "Low mem", mockSql }, alerts.get(0));
-        assertArrayEquals(new String[] { "Slow", mockSql }, alerts.get(1));
+        // second check founds Low mem & Slow
+        assertArrayEquals(new String[] { "Low mem", mockSql }, alerts.get(1));
+        assertArrayEquals(new String[] { "Slow", mockSql }, alerts.get(2));
     }
 }


[27/50] [abbrv] incubator-kylin git commit: update website links and content

Posted by li...@apache.org.
update website links and content


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

Branch: refs/heads/master
Commit: 4c15ee6e14a7842a6d6d2218f7e500bdc3bccbcd
Parents: 60a95f3
Author: Luke Han <lu...@apache.org>
Authored: Mon Sep 7 16:23:34 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Mon Sep 7 16:23:34 2015 +0800

----------------------------------------------------------------------
 website/_docs/index.cn.md    | 3 ++-
 website/_docs/index.md       | 4 ++--
 website/download/index.cn.md | 2 +-
 website/download/index.md    | 2 +-
 website/index.cn.md          | 2 +-
 website/index.md             | 2 +-
 6 files changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/_docs/index.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/index.cn.md b/website/_docs/index.cn.md
index 3e682a6..4958cb9 100644
--- a/website/_docs/index.cn.md
+++ b/website/_docs/index.cn.md
@@ -9,7 +9,7 @@ permalink: /cn/docs/index.html
 ------------  
 > Extreme OLAP Engine for Big Data
 
-Apache Kylin 是由eBay开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据。
+Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc.开发并贡献至开源社区。
 
 安装 
 ------------  
@@ -19,3 +19,4 @@ Apache Kylin 是由eBay开源的分布式分析引擎,提供Hadoop之上的SQL
 
 
 
+

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/_docs/index.md
----------------------------------------------------------------------
diff --git a/website/_docs/index.md b/website/_docs/index.md
index 0c62b63..64e1295 100644
--- a/website/_docs/index.md
+++ b/website/_docs/index.md
@@ -9,7 +9,7 @@ Welcome to Apache Kylin
 ------------  
 > Extreme OLAP Engine for Big Data
 
-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.
+Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
 
 Installation & Setup
 ------------  
@@ -51,6 +51,6 @@ Advanced Topics
 
 1.[New meta data model structure](/development/new_metadata.html)
 
----
+
 
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/download/index.cn.md
----------------------------------------------------------------------
diff --git a/website/download/index.cn.md b/website/download/index.cn.md
index e372fdc..5bba309 100644
--- a/website/download/index.cn.md
+++ b/website/download/index.cn.md
@@ -3,7 +3,7 @@ layout: download-cn
 title: 下载
 ---
 
-__最新发布 (源代码)__  
+__最新发布(源代码)__  
 最新发布的Apache Kylin可以从ASF网站下载::
 
 * [Apache Kylin v1.0-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.0-incubating/)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/download/index.md
----------------------------------------------------------------------
diff --git a/website/download/index.md b/website/download/index.md
index e7db862..e757d65 100644
--- a/website/download/index.md
+++ b/website/download/index.md
@@ -4,7 +4,7 @@ title: Download
 permalink: /download/index.html
 ---
 
-__Latest Release (Source Code)__  
+__Latest Release(Source Code)__  
 The latest release of Apache Kylin can be downloaded from the ASF:
 
 * [Apache Kylin v1.0-incubating](http://www.apache.org/dyn/closer.cgi/incubator/kylin/apache-kylin-1.0-incubating/)

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/index.cn.md
----------------------------------------------------------------------
diff --git a/website/index.cn.md b/website/index.cn.md
index e840db9..08e35ca 100644
--- a/website/index.cn.md
+++ b/website/index.cn.md
@@ -13,7 +13,7 @@ title: 首页
           <h4 class="section-title"><span>Apache Kylin 概览</span></h4>
           <div class="row" style="margin-top:-20px;">
             <div class="col-sm-12 col-md-12">
-              <p class="title_text"> Kylin 于2014年11月25日被接受会Apache孵化器项目</p>
+              <p class="title_text"> Apache Kylin 于2014年11月25日被接受会Apache孵化器项目</p>
               <p class="title_text"> Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。</p>
               <img id="diagram" src="{{ "/assets/images/kylin_diagram.png"| prepend: site.baseurl }}"> </div>
           </div>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/4c15ee6e/website/index.md
----------------------------------------------------------------------
diff --git a/website/index.md b/website/index.md
index 8efac9a..f8bdbe3 100644
--- a/website/index.md
+++ b/website/index.md
@@ -13,7 +13,7 @@ title: Home
           <h4 class="section-title"><span>Apache Kylin Overview</span></h4>
           <div class="row" style="margin-top:-20px;">
             <div class="col-sm-12 col-md-12">
-              <p class="title_text"> Kylin has been accepted as Apache Incubator Project on Nov 25, 2014.</p>
+              <p class="title_text"> Apache Kylin has been accepted as Apache Incubator Project on Nov 25, 2014.</p>
               <p class="title_text"> Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.</p>
               <img id="diagram" src="assets/images/kylin_diagram.png"> </div>
           </div>


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

Posted by li...@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/ebf4629e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/ebf4629e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/ebf4629e

Branch: refs/heads/master
Commit: ebf4629ee58426a93c4e1b396ecd6f3df16723ef
Parents: 4a4c719
Author: gaodayue <ga...@meituan.com>
Authored: Tue Sep 1 23:29:32 2015 +0800
Committer: Luke Han <lu...@apache.org>
Committed: Sun Sep 6 14:38:00 2015 +0800

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


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/ebf4629e/script/package.sh
----------------------------------------------------------------------
diff --git a/script/package.sh b/script/package.sh
index 63bfb3b..759c082 100755
--- a/script/package.sh
+++ b/script/package.sh
@@ -47,7 +47,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