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/12/23 10:21:59 UTC

kylin git commit: refactor, split part of KylinConfig into AbstractKylinConfig

Repository: kylin
Updated Branches:
  refs/heads/2.x-staging 9c00f308d -> 99ee1b653


refactor, split part of KylinConfig into AbstractKylinConfig


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

Branch: refs/heads/2.x-staging
Commit: 99ee1b653e766195200478a11a5bc48779c3de15
Parents: 9c00f30
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Dec 23 17:19:16 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Wed Dec 23 17:19:54 2015 +0800

----------------------------------------------------------------------
 .../kylin/common/AbstractKylinConfig.java       | 498 ++++++++++++++++
 .../org/apache/kylin/common/KylinConfig.java    | 586 +------------------
 .../kylin/common/KylinConfigOverride.java       |  23 +
 .../apache/kylin/common/util/MailService.java   |   2 +-
 .../kylin/common/util/MailServiceTest.java      |   4 +-
 .../org/apache/kylin/cube/model/CubeDesc.java   |   3 +
 .../validation/rule/AggregationGroupRule.java   |   2 +-
 .../engine/mr/common/MapReduceExecutable.java   |   2 +-
 .../engine/mr/steps/MergeStatisticsStep.java    |   2 +-
 .../kylin/rest/security/CrossDomainFilter.java  |   2 +-
 10 files changed, 558 insertions(+), 566 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-common/src/main/java/org/apache/kylin/common/AbstractKylinConfig.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/AbstractKylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/AbstractKylinConfig.java
new file mode 100644
index 0000000..a16346c
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/AbstractKylinConfig.java
@@ -0,0 +1,498 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Properties;
+import java.util.SortedSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.util.CliCommandExecutor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Sets;
+
+@SuppressWarnings("serial")
+/**
+ * An abstract class to encapsulate access to a set of 'properties'.
+ * Subclass can override methods in this class to extend the content of the 'properties',
+ * with some override values for example.
+ */
+public class AbstractKylinConfig implements Serializable {
+
+    private static final Logger logger = LoggerFactory.getLogger(AbstractKylinConfig.class);
+    
+    /*
+     * DON'T DEFINE CONSTANTS FOR PROPERTY KEYS!
+     * 
+     * For 1), no external need to access property keys, all accesses are by public methods.
+     * For 2), it's cumbersome to maintain constants at top and code at bottom.
+     * For 3), key literals usually appear only once.
+     */
+
+    public static String getKylinHome() {
+        String kylinHome = System.getenv("KYLIN_HOME");
+        if (StringUtils.isEmpty(kylinHome)) {
+            logger.warn("KYLIN_HOME was not set");
+        }
+        return kylinHome;
+    }
+
+    // ============================================================================
+    
+    private volatile Properties properties = new Properties();
+    
+    public String getOptional(String prop) {
+        return getOptional(prop, null);
+    }
+
+    public String getOptional(String prop, String dft) {
+        final String property = System.getProperty(prop);
+        return property != null ? property : properties.getProperty(prop, dft);
+    }
+
+    protected String[] getOptionalStringArray(String prop, String[] dft) {
+        final String property = getOptional(prop);
+        if (!StringUtils.isBlank(property)) {
+            return property.split("\\s*,\\s*");
+        } else {
+            return dft;
+        }
+    }
+
+    public String getRequired(String prop) {
+        String r = getOptional(prop);
+        if (StringUtils.isEmpty(r)) {
+            throw new IllegalArgumentException("missing '" + prop + "' in conf/kylin_instance.properties");
+        }
+        return r;
+    }
+    
+    /**
+     * Use with care, properties should be read-only. This is for testing mostly.
+     */
+    public void setProperty(String key, String value) {
+        logger.info("Kylin Config was updated with " + key + " : " + value);
+        properties.setProperty(key, value);
+    }
+    
+    protected Properties getAllProperties() {
+        return properties;
+    }
+
+    protected void reloadKylinConfig(InputStream is) {
+        Properties newProperties = new Properties();
+        try {
+            newProperties.load(is);
+        } catch (IOException e) {
+            throw new RuntimeException("Cannot load kylin config.", e);
+        } finally {
+            IOUtils.closeQuietly(is);
+        }
+        this.properties = newProperties;
+    }
+    
+    // ============================================================================
+    
+    public String getMetadataUrl() {
+        return getOptional("kylin.metadata.url");
+    }
+
+    public void setMetadataUrl(String metadataUrl) {
+        setProperty("kylin.metadata.url", metadataUrl);
+    }
+
+    public String getMetadataUrlPrefix() {
+        String hbaseMetadataUrl = getMetadataUrl();
+        String defaultPrefix = "kylin_metadata";
+
+        if (org.apache.commons.lang3.StringUtils.containsIgnoreCase(hbaseMetadataUrl, "@hbase")) {
+            int cut = hbaseMetadataUrl.indexOf('@');
+            String tmp = cut < 0 ? defaultPrefix : hbaseMetadataUrl.substring(0, cut);
+            return tmp;
+        } else {
+            return defaultPrefix;
+        }
+    }
+
+    public String getStorageUrl() {
+        return getOptional("kylin.storage.url");
+    }
+
+    public void setStorageUrl(String storageUrl) {
+        setProperty("kylin.storage.url", storageUrl);
+    }
+
+    /** was for route to hive, not used any more */
+    @Deprecated
+    public String getHiveUrl() {
+        return getOptional("hive.url", "");
+    }
+
+    /** was for route to hive, not used any more */
+    @Deprecated
+    public String getHiveUser() {
+        return getOptional("hive.user", "");
+    }
+
+    /** was for route to hive, not used any more */
+    @Deprecated
+    public String getHivePassword() {
+        return getOptional("hive.password", "");
+    }
+
+    public String getHdfsWorkingDirectory() {
+        String root = getRequired("kylin.hdfs.working.dir");
+        if (!root.endsWith("/")) {
+            root += "/";
+        }
+        return root + getMetadataUrlPrefix() + "/";
+    }
+
+    public CliCommandExecutor getCliCommandExecutor() throws IOException {
+        CliCommandExecutor exec = new CliCommandExecutor();
+        if (getRunAsRemoteCommand()) {
+            exec.setRunAtRemote(getRemoteHadoopCliHostname(), getRemoteHadoopCliPort(), getRemoteHadoopCliUsername(), getRemoteHadoopCliPassword());
+        }
+        return exec;
+    }
+
+    public String getHBaseClusterFs() {
+        return getOptional("kylin.hbase.cluster.fs", "");
+    }
+
+    public String getKylinJobLogDir() {
+        return getOptional("kylin.job.log.dir", "/tmp/kylin/logs");
+    }
+
+    public String getKylinJobJarPath() {
+        final String jobJar = getOptional("kylin.job.jar");
+        if (StringUtils.isNotEmpty(jobJar)) {
+            return jobJar;
+        }
+        String kylinHome = getKylinHome();
+        if (StringUtils.isEmpty(kylinHome)) {
+            return "";
+        }
+        return getFileName(kylinHome + File.separator + "lib", JOB_JAR_NAME_PATTERN);
+    }
+
+    public void overrideMRJobJarPath(String path) {
+        logger.info("override " + "kylin.job.jar" + " to " + path);
+        System.setProperty("kylin.job.jar", path);
+    }
+
+    public String getKylinJobMRLibDir() {
+        return getOptional("kylin.job.mr.lib.dir", "");
+    }
+
+    public String getKylinSparkJobJarPath() {
+        final String jobJar = getOptional("kylin.job.jar.spark");
+        if (StringUtils.isNotEmpty(jobJar)) {
+            return jobJar;
+        }
+        String kylinHome = getKylinHome();
+        if (StringUtils.isEmpty(kylinHome)) {
+            return "";
+        }
+        return getFileName(kylinHome + File.separator + "lib", SPARK_JOB_JAR_NAME_PATTERN);
+    }
+
+    public void overrideSparkJobJarPath(String path) {
+        logger.info("override " + "kylin.job.jar.spark" + " to " + path);
+        System.setProperty("kylin.job.jar.spark", path);
+    }
+
+    private static final Pattern COPROCESSOR_JAR_NAME_PATTERN = Pattern.compile("kylin-coprocessor-(.+)\\.jar");
+    private static final Pattern JOB_JAR_NAME_PATTERN = Pattern.compile("kylin-job-(.+)\\.jar");
+    private static final Pattern SPARK_JOB_JAR_NAME_PATTERN = Pattern.compile("kylin-engine-spark-(.+)\\.jar");
+
+    public String getCoprocessorLocalJar() {
+        final String coprocessorJar = getOptional("kylin.coprocessor.local.jar");
+        if (StringUtils.isNotEmpty(coprocessorJar)) {
+            return coprocessorJar;
+        }
+        String kylinHome = getKylinHome();
+        if (StringUtils.isEmpty(kylinHome)) {
+            throw new RuntimeException("getCoprocessorLocalJar needs KYLIN_HOME");
+        }
+        return getFileName(kylinHome + File.separator + "lib", COPROCESSOR_JAR_NAME_PATTERN);
+    }
+
+    public void overrideCoprocessorLocalJar(String path) {
+        logger.info("override " + "kylin.coprocessor.local.jar" + " to " + path);
+        System.setProperty("kylin.coprocessor.local.jar", path);
+    }
+
+    private static String getFileName(String homePath, Pattern pattern) {
+        File home = new File(homePath);
+        SortedSet<String> files = Sets.newTreeSet();
+        if (home.exists() && home.isDirectory()) {
+            for (File file : home.listFiles()) {
+                final Matcher matcher = pattern.matcher(file.getName());
+                if (matcher.matches()) {
+                    files.add(file.getAbsolutePath());
+                }
+            }
+        }
+        if (files.isEmpty()) {
+            throw new RuntimeException("cannot find " + pattern.toString() + " in " + homePath);
+        } else {
+            return files.last();
+        }
+    }
+
+    public double getDefaultHadoopJobReducerInputMB() {
+        return Double.parseDouble(getOptional("kylin.job.mapreduce.default.reduce.input.mb", "500"));
+    }
+
+    public double getDefaultHadoopJobReducerCountRatio() {
+        return Double.parseDouble(getOptional("kylin.job.mapreduce.default.reduce.count.ratio", "1.0"));
+    }
+
+    public int getHadoopJobMaxReducerNumber() {
+        return Integer.parseInt(getOptional("kylin.job.mapreduce.max.reducer.number", "500"));
+    }
+
+    public boolean getRunAsRemoteCommand() {
+        return Boolean.parseBoolean(getOptional("kylin.job.run.as.remote.cmd"));
+    }
+
+    public void setRunAsRemoteCommand(String v) {
+        setProperty("kylin.job.run.as.remote.cmd", v);
+    }
+
+    public int getRemoteHadoopCliPort() {
+        return Integer.parseInt(getOptional("kylin.job.remote.cli.port", "22"));
+    }
+
+    public String getRemoteHadoopCliHostname() {
+        return getOptional("kylin.job.remote.cli.hostname");
+    }
+
+    public void setRemoteHadoopCliHostname(String v) {
+        setProperty("kylin.job.remote.cli.hostname", v);
+    }
+
+    public String getRemoteHadoopCliUsername() {
+        return getOptional("kylin.job.remote.cli.username");
+    }
+
+    public void setRemoteHadoopCliUsername(String v) {
+        setProperty("kylin.job.remote.cli.username", v);
+    }
+
+    public String getRemoteHadoopCliPassword() {
+        return getOptional("kylin.job.remote.cli.password");
+    }
+
+    public void setRemoteHadoopCliPassword(String v) {
+        setProperty("kylin.job.remote.cli.password", v);
+    }
+
+    public String getCliWorkingDir() {
+        return getOptional("kylin.job.remote.cli.working.dir");
+    }
+
+    public String getMapReduceCmdExtraArgs() {
+        return getOptional("kylin.job.cmd.extra.args");
+    }
+
+    public String getOverrideHiveTableLocation(String table) {
+        return getOptional("hive.table.location." + table.toUpperCase());
+    }
+
+    public String getYarnStatusCheckUrl() {
+        return getOptional("kylin.job.yarn.app.rest.check.status.url", null);
+    }
+
+    public int getYarnStatusCheckIntervalSeconds() {
+        return Integer.parseInt(getOptional("kylin.job.yarn.app.rest.check.interval.seconds", "60"));
+    }
+
+    public int getMaxConcurrentJobLimit() {
+        return Integer.parseInt(getOptional("kylin.job.concurrent.max.limit", "10"));
+    }
+
+    public String getTimeZone() {
+        return getOptional("kylin.rest.timezone", "PST");
+    }
+
+    public String[] getRestServers() {
+        return getOptionalStringArray("kylin.rest.servers", new String[0]);
+    }
+
+    public String getAdminDls() {
+        return getOptional("kylin.job.admin.dls", null);
+    }
+
+    public long getJobStepTimeout() {
+        return Long.parseLong(getOptional("kylin.job.step.timeout", String.valueOf(2 * 60 * 60)));
+    }
+
+    public String getServerMode() {
+        return this.getOptional("kylin.server.mode", "all");
+    }
+
+    public int getDictionaryMaxCardinality() {
+        return Integer.parseInt(getOptional("kylin.dictionary.max.cardinality", "5000000"));
+    }
+
+    public int getTableSnapshotMaxMB() {
+        return Integer.parseInt(getOptional("kylin.table.snapshot.max_mb", "300"));
+    }
+
+    public int getHBaseRegionCut(String capacity) {
+        String cut;
+        switch (capacity) {
+        case "SMALL":
+            cut = getOptional("kylin.hbase.region.cut.small", "10");
+            break;
+        case "MEDIUM":
+            cut = getOptional("kylin.hbase.region.cut.medium", "20");
+            break;
+        case "LARGE":
+            cut = getOptional("kylin.hbase.region.cut.large", "100");
+            break;
+        default:
+            throw new IllegalArgumentException("Capacity not recognized: " + capacity);
+        }
+
+        return Integer.valueOf(cut);
+    }
+
+    public int getHBaseRegionCountMin() {
+        return Integer.parseInt(getOptional("kylin.hbase.region.count.min", "1"));
+    }
+
+    public int getHBaseRegionCountMax() {
+        return Integer.parseInt(getOptional("kylin.hbase.region.count.max", "500"));
+    }
+
+    public int getScanThreshold() {
+        return Integer.parseInt(getOptional("kylin.query.scan.threshold", "10000000"));
+    }
+
+    public boolean getQueryRunLocalCoprocessor() {
+        return Boolean.parseBoolean(getOptional("kylin.query.run.local.coprocessor", "false"));
+    }
+
+    public Long getQueryDurationCacheThreshold() {
+        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.duration", String.valueOf(2000)));
+    }
+
+    public Long getQueryScanCountCacheThreshold() {
+        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.scancount", String.valueOf(10 * 1024)));
+    }
+
+    public long getQueryMemBudget() {
+        return Long.parseLong(this.getOptional("kylin.query.mem.budget", String.valueOf(3L * 1024 * 1024 * 1024)));
+    }
+
+    public boolean isQuerySecureEnabled() {
+        return Boolean.parseBoolean(this.getOptional("kylin.query.security.enabled", "false"));
+    }
+
+    public boolean isQueryCacheEnabled() {
+        return Boolean.parseBoolean(this.getOptional("kylin.query.cache.enabled", "true"));
+    }
+
+    public boolean isQueryIgnoreUnknownFunction() {
+        return Boolean.parseBoolean(this.getOptional("kylin.query.ignore_unknown_function", "false"));
+    }
+
+    public int getHBaseKeyValueSize() {
+        return Integer.parseInt(this.getOptional("kylin.hbase.client.keyvalue.maxsize", "10485760"));
+    }
+
+    public int getHBaseScanCacheRows() {
+        return Integer.parseInt(this.getOptional("kylin.hbase.scan.cache_rows", "1024"));
+    }
+
+    public int getHBaseScanMaxResultSize() {
+        return Integer.parseInt(this.getOptional("kylin.hbase.scan.max_result_size", "" + (5 * 1024 * 1024))); // 5 MB
+    }
+
+    public int getCubingInMemSamplingPercent() {
+        int percent = Integer.parseInt(this.getOptional("kylin.job.cubing.inMem.sampling.percent", "100"));
+        percent = Math.max(percent, 1);
+        percent = Math.min(percent, 100);
+        return percent;
+    }
+
+    public String getHbaseDefaultCompressionCodec() {
+        return getOptional("kylin.hbase.default.compression.codec", "");
+    }
+
+    public boolean isHiveKeepFlatTable() {
+        return Boolean.parseBoolean(this.getOptional("kylin.hive.keep.flat.table", "false"));
+    }
+
+    public String getHiveDatabaseForIntermediateTable() {
+        return this.getOptional("kylin.job.hive.database.for.intermediatetable", "default");
+    }
+
+    public String getKylinOwner() {
+        return this.getOptional("kylin.owner", "");
+    }
+
+    public String getSparkHome() {
+        return getRequired("kylin.spark.home");
+    }
+
+    public String getSparkMaster() {
+        return getRequired("kylin.spark.master");
+    }
+
+    public boolean isMailEnabled() {
+        return Boolean.parseBoolean(getOptional("mail.enabled", "false"));
+    }
+
+    public void setMailEnabled(boolean enable) {
+        setProperty("mail.enabled", "" + enable);
+    }
+
+    public String getMailHost() {
+        return getOptional("mail.host", "");
+    }
+
+    public String getMailUsername() {
+        return getOptional("mail.username", "");
+    }
+
+    public String getMailPassword() {
+        return getOptional("mail.password", "");
+    }
+
+    public String getMailSender() {
+        return getOptional("mail.sender", "");
+    }
+
+    public String toString() {
+        return getMetadataUrl();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 43acaf2..a89be5f 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -27,129 +27,29 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintWriter;
-import java.io.Serializable;
 import java.io.StringWriter;
 import java.util.Enumeration;
 import java.util.Map;
 import java.util.Properties;
-import java.util.SortedSet;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.restclient.RestClient;
-import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.Log4jConfigurer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Sets;
-
 /**
  */
 @SuppressWarnings("serial")
-public class KylinConfig implements Serializable {
-
-    /*
-     * NOTE: These key constants should be private or even better be removed.
-     *       All external access should go through public methods.
-     */
-    public static final String KYLIN_OWNER = "kylin.owner";
-
-    public static final String KYLIN_STORAGE_URL = "kylin.storage.url";
-
-    public static final String KYLIN_METADATA_URL = "kylin.metadata.url";
-
-    public static final String KYLIN_REST_SERVERS = "kylin.rest.servers";
-
-    public static final String KYLIN_REST_TIMEZONE = "kylin.rest.timezone";
-
-    public static final String KYLIN_JOB_CONCURRENT_MAX_LIMIT = "kylin.job.concurrent.max.limit";
-
-    public static final String KYLIN_JOB_YARN_APP_REST_CHECK_URL = "kylin.job.yarn.app.rest.check.status.url";
+public class KylinConfig extends AbstractKylinConfig {
 
-    public static final String KYLIN_JOB_YARN_APP_REST_CHECK_INTERVAL_SECONDS = "kylin.job.yarn.app.rest.check.interval.seconds";
-
-    public static final String HIVE_TABLE_LOCATION_PREFIX = "hive.table.location.";
-
-    public static final String KYLIN_JOB_REMOTE_CLI_PASSWORD = "kylin.job.remote.cli.password";
-
-    public static final String KYLIN_JOB_REMOTE_CLI_USERNAME = "kylin.job.remote.cli.username";
-
-    public static final String KYLIN_JOB_REMOTE_CLI_HOSTNAME = "kylin.job.remote.cli.hostname";
-
-    public static final String KYLIN_JOB_REMOTE_CLI_PORT = "kylin.job.remote.cli.port";
-
-    public static final String KYLIN_JOB_REMOTE_CLI_WORKING_DIR = "kylin.job.remote.cli.working.dir";
-
-    public static final String KYLIN_JOB_CMD_EXTRA_ARGS = "kylin.job.cmd.extra.args";
-    /**
-     * Toggle to indicate whether to use hive for table flattening. Default
-     * true.
-     */
-    public static final String KYLIN_JOB_RUN_AS_REMOTE_CMD = "kylin.job.run.as.remote.cmd";
-
-    public static final String KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_COUNT_RATIO = "kylin.job.mapreduce.default.reduce.count.ratio";
-
-    public static final String KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_INPUT_MB = "kylin.job.mapreduce.default.reduce.input.mb";
-
-    public static final String KYLIN_JOB_MAPREDUCE_MAX_REDUCER_NUMBER = "kylin.job.mapreduce.max.reducer.number";
-
-    public static final String KYLIN_JOB_JAR = "kylin.job.jar";
-
-    public static final String KYLIN_JOB_JAR_SPARK = "kylin.job.jar.spark";
-
-    public static final String COPROCESSOR_LOCAL_JAR = "kylin.coprocessor.local.jar";
-
-    public static final String KYLIN_JOB_LOG_DIR = "kylin.job.log.dir";
-
-    public static final String KYLIN_JOB_CUBING_IN_MEM = "kylin.job.cubing.inMem";
-
-    public static final String KYLIN_JOB_CUBING_IN_MEM_SAMPLING_PERCENT = "kylin.job.cubing.inMem.sampling.percent";
-
-    public static final String KYLIN_HDFS_WORKING_DIR = "kylin.hdfs.working.dir";
-
-    public static final String KYLIN_HBASE_CLUSTER_FS = "kylin.hbase.cluster.fs";
-
-    public static final String HIVE_DATABASE_FOR_INTERMEDIATE_TABLE = "kylin.job.hive.database.for.intermediatetable";
-
-    public static final String HIVE_PASSWORD = "hive.password";
-
-    public static final String HIVE_USER = "hive.user";
+    private static final Logger logger = LoggerFactory.getLogger(KylinConfig.class);
 
-    public static final String HIVE_URL = "hive.url";
-    /**
-     * Kylin properties file
-     */
+    /** Kylin properties file name */
     public static final String KYLIN_CONF_PROPERTIES_FILE = "kylin.properties";
-
-    public static final String MAIL_ENABLED = "mail.enabled";
-
-    public static final String MAIL_HOST = "mail.host";
-
-    public static final String MAIL_USERNAME = "mail.username";
-
-    public static final String MAIL_PASSWORD = "mail.password";
-
-    public static final String MAIL_SENDER = "mail.sender";
-
-    public static final String KYLIN_HOME = "KYLIN_HOME";
     public static final String KYLIN_CONF = "KYLIN_CONF";
 
-    public static final String HBASE_REGION_CUT_SMALL = "kylin.hbase.region.cut.small";
-    public static final String HBASE_REGION_CUT_MEDIUM = "kylin.hbase.region.cut.medium";
-    public static final String HBASE_REGION_CUT_LARGE = "kylin.hbase.region.cut.large";
-
-    public static final String SPARK_HOME = "kylin.spark.home";
-    public static final String SPARK_MASTER = "kylin.spark.master";
-
-    private static final Logger logger = LoggerFactory.getLogger(KylinConfig.class);
-
-    public static final String VERSION = "${project.version}";
-
-    public static final String HTABLE_DEFAULT_COMPRESSION_CODEC = "kylin.hbase.default.compression.codec";
-
     // static cached instances
     private static KylinConfig ENV_INSTANCE = null;
 
@@ -249,373 +149,6 @@ public class KylinConfig implements Serializable {
         return config;
     }
 
-    // ============================================================================
-
-    /**
-     * Find config from environment. The Search process: 1. Check the
-     * $KYLIN_CONF/kylin.properties 2. Check the $KYLIN_HOME/conf/kylin.properties
-     */
-    private static KylinConfig loadKylinConfig() {
-        Log4jConfigurer.initLogger();
-
-        InputStream is = getKylinPropertiesAsInputSteam();
-        if (is == null) {
-            throw new IllegalArgumentException("Failed to load kylin config");
-        }
-        KylinConfig config = new KylinConfig();
-        config.reloadKylinConfig(is);
-
-        return config;
-    }
-
-    private volatile Properties properties = new Properties();
-
-    public CliCommandExecutor getCliCommandExecutor() throws IOException {
-        CliCommandExecutor exec = new CliCommandExecutor();
-        if (getRunAsRemoteCommand()) {
-            exec.setRunAtRemote(getRemoteHadoopCliHostname(), getRemoteHadoopCliPort(), getRemoteHadoopCliUsername(), getRemoteHadoopCliPassword());
-        }
-        return exec;
-    }
-
-    // ============================================================================
-
-    public String getStorageUrl() {
-        return getOptional(KYLIN_STORAGE_URL);
-    }
-
-    /** was for route to hive, not used any more */
-    @Deprecated
-    public String getHiveUrl() {
-        return getOptional(HIVE_URL, "");
-    }
-
-    /** was for route to hive, not used any more */
-    @Deprecated
-    public String getHiveUser() {
-        return getOptional(HIVE_USER, "");
-    }
-
-    /** was for route to hive, not used any more */
-    @Deprecated
-    public String getHivePassword() {
-        return getOptional(HIVE_PASSWORD, "");
-    }
-
-    public String getHdfsWorkingDirectory() {
-        String root = getRequired(KYLIN_HDFS_WORKING_DIR);
-        if (!root.endsWith("/")) {
-            root += "/";
-        }
-        return root + getMetadataUrlPrefix() + "/";
-    }
-
-    public String getHBaseClusterFs() {
-        return getOptional(KYLIN_HBASE_CLUSTER_FS, "");
-    }
-
-    public String getKylinJobLogDir() {
-        return getOptional(KYLIN_JOB_LOG_DIR, "/tmp/kylin/logs");
-    }
-
-    public String getKylinJobJarPath() {
-        final String jobJar = getOptional(KYLIN_JOB_JAR);
-        if (StringUtils.isNotEmpty(jobJar)) {
-            return jobJar;
-        }
-        String kylinHome = getKylinHome();
-        if (StringUtils.isEmpty(kylinHome)) {
-            return "";
-        }
-        return getFileName(kylinHome + File.separator + "lib", JOB_JAR_NAME_PATTERN);
-    }
-
-    public String getKylinJobMRLibDir() {
-        return getOptional("kylin.job.mr.lib.dir", "");
-    }
-
-    public String getKylinSparkJobJarPath() {
-        final String jobJar = getOptional(KYLIN_JOB_JAR_SPARK);
-        if (StringUtils.isNotEmpty(jobJar)) {
-            return jobJar;
-        }
-        String kylinHome = getKylinHome();
-        if (StringUtils.isEmpty(kylinHome)) {
-            return "";
-        }
-        return getFileName(kylinHome + File.separator + "lib", SPARK_JOB_JAR_NAME_PATTERN);
-    }
-
-    public void overrideMRJobJarPath(String path) {
-        logger.info("override " + KYLIN_JOB_JAR + " to " + path);
-        System.setProperty(KYLIN_JOB_JAR, path);
-    }
-
-    public void overrideSparkJobJarPath(String path) {
-        logger.info("override " + KYLIN_JOB_JAR_SPARK + " to " + path);
-        System.setProperty(KYLIN_JOB_JAR_SPARK, path);
-    }
-
-    private static final Pattern COPROCESSOR_JAR_NAME_PATTERN = Pattern.compile("kylin-coprocessor-(.+)\\.jar");
-    private static final Pattern JOB_JAR_NAME_PATTERN = Pattern.compile("kylin-job-(.+)\\.jar");
-    private static final Pattern SPARK_JOB_JAR_NAME_PATTERN = Pattern.compile("kylin-engine-spark-(.+)\\.jar");
-
-    public String getCoprocessorLocalJar() {
-        final String coprocessorJar = getOptional(COPROCESSOR_LOCAL_JAR);
-        if (StringUtils.isNotEmpty(coprocessorJar)) {
-            return coprocessorJar;
-        }
-        String kylinHome = getKylinHome();
-        if (StringUtils.isEmpty(kylinHome)) {
-            throw new RuntimeException("getCoprocessorLocalJar needs KYLIN_HOME");
-        }
-        return getFileName(kylinHome + File.separator + "lib", COPROCESSOR_JAR_NAME_PATTERN);
-    }
-
-    private static String getFileName(String homePath, Pattern pattern) {
-        File home = new File(homePath);
-        SortedSet<String> files = Sets.newTreeSet();
-        if (home.exists() && home.isDirectory()) {
-            for (File file : home.listFiles()) {
-                final Matcher matcher = pattern.matcher(file.getName());
-                if (matcher.matches()) {
-                    files.add(file.getAbsolutePath());
-                }
-            }
-        }
-        if (files.isEmpty()) {
-            throw new RuntimeException("cannot find " + pattern.toString() + " in " + homePath);
-        } else {
-            return files.last();
-        }
-    }
-
-    public void overrideCoprocessorLocalJar(String path) {
-        logger.info("override " + COPROCESSOR_LOCAL_JAR + " to " + path);
-        System.setProperty(COPROCESSOR_LOCAL_JAR, path);
-    }
-
-    public double getDefaultHadoopJobReducerInputMB() {
-        return Double.parseDouble(getOptional(KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_INPUT_MB, "500"));
-    }
-
-    public double getDefaultHadoopJobReducerCountRatio() {
-        return Double.parseDouble(getOptional(KYLIN_JOB_MAPREDUCE_DEFAULT_REDUCE_COUNT_RATIO, "1.0"));
-    }
-
-    public int getHadoopJobMaxReducerNumber() {
-        return Integer.parseInt(getOptional(KYLIN_JOB_MAPREDUCE_MAX_REDUCER_NUMBER, "500"));
-    }
-
-    public boolean getRunAsRemoteCommand() {
-        return Boolean.parseBoolean(getOptional(KYLIN_JOB_RUN_AS_REMOTE_CMD));
-    }
-
-    public int getRemoteHadoopCliPort() {
-        return Integer.parseInt(getOptional(KYLIN_JOB_REMOTE_CLI_PORT, "22"));
-    }
-
-    public String getRemoteHadoopCliHostname() {
-        return getOptional(KYLIN_JOB_REMOTE_CLI_HOSTNAME);
-    }
-
-    public String getRemoteHadoopCliUsername() {
-        return getOptional(KYLIN_JOB_REMOTE_CLI_USERNAME);
-    }
-
-    public String getRemoteHadoopCliPassword() {
-        return getOptional(KYLIN_JOB_REMOTE_CLI_PASSWORD);
-    }
-
-    public String getCliWorkingDir() {
-        return getOptional(KYLIN_JOB_REMOTE_CLI_WORKING_DIR);
-    }
-
-    public String getMapReduceCmdExtraArgs() {
-        return getOptional(KYLIN_JOB_CMD_EXTRA_ARGS);
-    }
-
-    public String getOverrideHiveTableLocation(String table) {
-        return getOptional(HIVE_TABLE_LOCATION_PREFIX + table.toUpperCase());
-    }
-
-    public String getYarnStatusCheckUrl() {
-        return getOptional(KYLIN_JOB_YARN_APP_REST_CHECK_URL, null);
-    }
-
-    public int getYarnStatusCheckIntervalSeconds() {
-        return Integer.parseInt(getOptional(KYLIN_JOB_YARN_APP_REST_CHECK_INTERVAL_SECONDS, "60"));
-    }
-
-    public int getMaxConcurrentJobLimit() {
-        return Integer.parseInt(getOptional(KYLIN_JOB_CONCURRENT_MAX_LIMIT, "10"));
-    }
-
-    public String getTimeZone() {
-        return getOptional(KYLIN_REST_TIMEZONE, "PST");
-    }
-
-    public String[] getRestServers() {
-        return getOptionalStringArray(KYLIN_REST_SERVERS, new String[0]);
-    }
-
-    public String getAdminDls() {
-        return getOptional("kylin.job.admin.dls", null);
-    }
-
-    public long getJobStepTimeout() {
-        return Long.parseLong(getOptional("kylin.job.step.timeout", String.valueOf(2 * 60 * 60)));
-    }
-
-    public String getServerMode() {
-        return this.getOptional("kylin.server.mode", "all");
-    }
-
-    public int getDictionaryMaxCardinality() {
-        return Integer.parseInt(getOptional("kylin.dictionary.max.cardinality", "5000000"));
-    }
-
-    public int getTableSnapshotMaxMB() {
-        return Integer.parseInt(getOptional("kylin.table.snapshot.max_mb", "300"));
-    }
-
-    public int getHBaseRegionCountMin() {
-        return Integer.parseInt(getOptional("kylin.hbase.region.count.min", "1"));
-    }
-
-    public int getHBaseRegionCountMax() {
-        return Integer.parseInt(getOptional("kylin.hbase.region.count.max", "500"));
-    }
-
-    public int getScanThreshold() {
-        return Integer.parseInt(getOptional("kylin.query.scan.threshold", "10000000"));
-    }
-
-    public boolean getQueryRunLocalCoprocessor() {
-        return Boolean.parseBoolean(getOptional("kylin.query.run.local.coprocessor", "false"));
-    }
-
-    public Long getQueryDurationCacheThreshold() {
-        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.duration", String.valueOf(2000)));
-    }
-
-    public Long getQueryScanCountCacheThreshold() {
-        return Long.parseLong(this.getOptional("kylin.query.cache.threshold.scancount", String.valueOf(10 * 1024)));
-    }
-
-    public long getQueryMemBudget() {
-        return Long.parseLong(this.getOptional("kylin.query.mem.budget", String.valueOf(3L * 1024 * 1024 * 1024)));
-    }
-
-    public boolean isQuerySecureEnabled() {
-        return Boolean.parseBoolean(this.getOptional("kylin.query.security.enabled", "false"));
-    }
-
-    public boolean isQueryCacheEnabled() {
-        return Boolean.parseBoolean(this.getOptional("kylin.query.cache.enabled", "true"));
-    }
-
-    public boolean isQueryIgnoreUnknownFunction() {
-        return Boolean.parseBoolean(this.getOptional("kylin.query.ignore_unknown_function", "false"));
-    }
-
-    public int getHBaseKeyValueSize() {
-        return Integer.parseInt(this.getOptional("kylin.hbase.client.keyvalue.maxsize", "10485760"));
-    }
-
-    public int getHBaseScanCacheRows() {
-        return Integer.parseInt(this.getOptional("kylin.hbase.scan.cache_rows", "1024"));
-    }
-
-    public int getHBaseScanMaxResultSize() {
-        return Integer.parseInt(this.getOptional("kylin.hbase.scan.max_result_size", "" + (5 * 1024 * 1024))); // 5 MB
-    }
-
-    public boolean isCubingInMem() {
-        return Boolean.parseBoolean(this.getOptional(KYLIN_JOB_CUBING_IN_MEM, "false"));
-    }
-
-    public int getCubingInMemSamplingPercent() {
-        int percent = Integer.parseInt(this.getOptional(KYLIN_JOB_CUBING_IN_MEM_SAMPLING_PERCENT, "100"));
-        percent = Math.max(percent, 1);
-        percent = Math.min(percent, 100);
-        return percent;
-    }
-
-    public String getHbaseDefaultCompressionCodec() {
-        return getOptional(HTABLE_DEFAULT_COMPRESSION_CODEC, "");
-    }
-
-    public boolean isHiveKeepFlatTable() {
-        return Boolean.parseBoolean(this.getOptional("kylin.hive.keep.flat.table", "false"));
-    }
-
-    private String getOptional(String prop) {
-        final String property = System.getProperty(prop);
-        return property != null ? property : properties.getProperty(prop);
-    }
-
-    private String[] getOptionalStringArray(String prop, String[] dft) {
-        final String property = getOptional(prop);
-        if (!StringUtils.isBlank(property)) {
-            return property.split("\\s*,\\s*");
-        } else {
-            return dft;
-        }
-    }
-
-    private String getOptional(String prop, String dft) {
-        final String property = System.getProperty(prop);
-        return property != null ? property : properties.getProperty(prop, dft);
-    }
-
-    private String getRequired(String prop) {
-        final String property = System.getProperty(prop);
-        if (property != null) {
-            return property;
-        }
-        String r = properties.getProperty(prop);
-        if (StringUtils.isEmpty(r)) {
-            throw new IllegalArgumentException("missing '" + prop + "' in conf/kylin_instance.properties");
-        }
-        return r;
-    }
-
-    void reloadKylinConfig(InputStream is) {
-        Properties newProperties = new Properties();
-        try {
-            newProperties.load(is);
-        } catch (IOException e) {
-            throw new RuntimeException("Cannot load kylin config.", e);
-        } finally {
-            IOUtils.closeQuietly(is);
-        }
-        this.properties = newProperties;
-    }
-
-    public void writeProperties(File file) throws IOException {
-        FileOutputStream fos = null;
-        try {
-            fos = new FileOutputStream(file);
-            properties.store(fos, file.getAbsolutePath());
-        } finally {
-            IOUtils.closeQuietly(fos);
-        }
-    }
-
-    public static String getKylinHome() {
-        String kylinHome = System.getenv(KYLIN_HOME);
-        if (StringUtils.isEmpty(kylinHome)) {
-            logger.warn("KYLIN_HOME was not set");
-            return kylinHome;
-        }
-        return kylinHome;
-    }
-
-    public void printProperties() throws IOException {
-        properties.list(System.out);
-    }
-
     private static File getKylinProperties() {
         String kylinConfHome = System.getProperty(KYLIN_CONF);
         if (!StringUtils.isEmpty(kylinConfHome)) {
@@ -689,68 +222,33 @@ public class KylinConfig implements Serializable {
         return new File(path, KYLIN_CONF_PROPERTIES_FILE);
     }
 
-    public String getMetadataUrl() {
-        return getOptional(KYLIN_METADATA_URL);
-    }
-
-    public String getMetadataUrlPrefix() {
-        String hbaseMetadataUrl = getMetadataUrl();
-        String defaultPrefix = "kylin_metadata";
+    /**
+     * Find config from environment. The Search process: 1. Check the
+     * $KYLIN_CONF/kylin.properties 2. Check the $KYLIN_HOME/conf/kylin.properties
+     */
+    private static KylinConfig loadKylinConfig() {
+        Log4jConfigurer.initLogger();
 
-        if (org.apache.commons.lang3.StringUtils.containsIgnoreCase(hbaseMetadataUrl, "@hbase")) {
-            int cut = hbaseMetadataUrl.indexOf('@');
-            String tmp = cut < 0 ? defaultPrefix : hbaseMetadataUrl.substring(0, cut);
-            return tmp;
-        } else {
-            return defaultPrefix;
+        InputStream is = getKylinPropertiesAsInputSteam();
+        if (is == null) {
+            throw new IllegalArgumentException("Failed to load kylin config");
         }
-    }
-
-    public void setMetadataUrl(String metadataUrl) {
-        properties.setProperty(KYLIN_METADATA_URL, metadataUrl);
-    }
-
-    public void setStorageUrl(String storageUrl) {
-        properties.setProperty(KYLIN_STORAGE_URL, storageUrl);
-    }
-
-    public String getHiveDatabaseForIntermediateTable() {
-        return this.getOptional(HIVE_DATABASE_FOR_INTERMEDIATE_TABLE, "default");
-    }
-
-    public String getKylinOwner() {
-        return this.getOptional(KYLIN_OWNER, "");
-    }
-
-    public void setRunAsRemoteCommand(String v) {
-        properties.setProperty(KYLIN_JOB_RUN_AS_REMOTE_CMD, v);
-    }
-
-    public void setRemoteHadoopCliHostname(String v) {
-        properties.setProperty(KYLIN_JOB_REMOTE_CLI_HOSTNAME, v);
-    }
-
-    public void setRemoteHadoopCliUsername(String v) {
-        properties.setProperty(KYLIN_JOB_REMOTE_CLI_USERNAME, v);
-    }
+        KylinConfig config = new KylinConfig();
+        config.reloadKylinConfig(is);
 
-    public void setRemoteHadoopCliPassword(String v) {
-        properties.setProperty(KYLIN_JOB_REMOTE_CLI_PASSWORD, v);
+        return config;
     }
 
-    public String getProperty(String key, String defaultValue) {
-        return properties.getProperty(key, defaultValue);
-    }
+    // ============================================================================
 
-    /**
-     * Set a new key:value into the kylin config.
-     *
-     * @param key
-     * @param value
-     */
-    public void setProperty(String key, String value) {
-        logger.info("Kylin Config was updated with " + key + " : " + value);
-        properties.setProperty(key, value);
+    public void writeProperties(File file) throws IOException {
+        FileOutputStream fos = null;
+        try {
+            fos = new FileOutputStream(file);
+            getAllProperties().store(fos, file.getAbsolutePath());
+        } finally {
+            IOUtils.closeQuietly(fos);
+        }
     }
 
     public String getConfigAsString() throws IOException {
@@ -760,42 +258,12 @@ public class KylinConfig implements Serializable {
     }
 
     private void list(PrintWriter out) {
-        for (Enumeration<?> e = properties.keys(); e.hasMoreElements();) {
+        Properties props = getAllProperties();
+        for (Enumeration<?> e = props.keys(); e.hasMoreElements();) {
             String key = (String) e.nextElement();
-            String val = (String) properties.get(key);
+            String val = (String) props.get(key);
             out.println(key + "=" + val);
         }
     }
 
-    public String getSparkHome() {
-        return properties.getProperty(SPARK_HOME);
-    }
-
-    public String getSparkMaster() {
-        return properties.getProperty(SPARK_MASTER);
-    }
-
-    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 Integer.valueOf(cut);
-    }
-
-    public String toString() {
-        return getMetadataUrl();
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-common/src/main/java/org/apache/kylin/common/KylinConfigOverride.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigOverride.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigOverride.java
new file mode 100644
index 0000000..cda89f0
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigOverride.java
@@ -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.
+*/
+
+package org.apache.kylin.common;
+
+public class KylinConfigOverride {
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-common/src/main/java/org/apache/kylin/common/util/MailService.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/MailService.java b/core-common/src/main/java/org/apache/kylin/common/util/MailService.java
index 3b0a4e1..9a4ec64 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/MailService.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/MailService.java
@@ -42,7 +42,7 @@ public class MailService {
     private static final Log logger = LogFactory.getLog(MailService.class);
 
     public MailService(KylinConfig config) {
-        this("true".equalsIgnoreCase(config.getProperty(KylinConfig.MAIL_ENABLED, "false")), config.getProperty(KylinConfig.MAIL_HOST, ""), config.getProperty(KylinConfig.MAIL_USERNAME, ""), config.getProperty(KylinConfig.MAIL_PASSWORD, ""), config.getProperty(KylinConfig.MAIL_SENDER, ""));
+        this(config.isMailEnabled(), config.getMailHost(), config.getMailUsername(), config.getMailPassword(), config.getMailSender());
     }
 
     private MailService(boolean enabled, String host, String username, String password, String sender) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-common/src/test/java/org/apache/kylin/common/util/MailServiceTest.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/MailServiceTest.java b/core-common/src/test/java/org/apache/kylin/common/util/MailServiceTest.java
index bc4d7cf..df06221 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/MailServiceTest.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/MailServiceTest.java
@@ -51,8 +51,8 @@ public class MailServiceTest extends LocalFileMetadataTestCase {
         boolean sent = sendTestEmail(mailservice);
         assert sent;
 
-        // set mail.enabled=false, and run again, this time should be no mail delviered
-        config.setProperty(KylinConfig.MAIL_ENABLED, "false");
+        // set mail.enabled=false, and run again, this time should be no mail delivered
+        config.setMailEnabled(false);
         mailservice = new MailService(config);
         sent = sendTestEmail(mailservice);
         assert !sent;

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index dc5a17b..97fd1aa 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -130,6 +131,8 @@ public class CubeDesc extends RootPersistentEntity {
     private int engineType = IEngineAware.ID_MR_V1;
     @JsonProperty("storage_type")
     private int storageType = IStorageAware.ID_HBASE;
+    @JsonProperty("override_kylin_properties")
+    private LinkedHashMap<String, String> overrideKylinProps = new LinkedHashMap<String, String>();
 
     private Map<String, Map<String, TblColRef>> columnMap = new HashMap<String, Map<String, TblColRef>>();
     private LinkedHashSet<TblColRef> allColumns = new LinkedHashSet<TblColRef>();

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
index 17f5b2d..40555c2 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/AggregationGroupRule.java
@@ -179,7 +179,7 @@ public class AggregationGroupRule implements IValidatorRule<CubeDesc> {
     }
 
     protected int getMaxAgrGroupSize() {
-        String size = KylinConfig.getInstanceFromEnv().getProperty(KEY_MAX_AGR_GROUP_SIZE, String.valueOf(DEFAULT_MAX_AGR_GROUP_SIZE));
+        String size = KylinConfig.getInstanceFromEnv().getOptional(KEY_MAX_AGR_GROUP_SIZE, String.valueOf(DEFAULT_MAX_AGR_GROUP_SIZE));
         return Integer.parseInt(size);
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
index 46557c8..8c1812f 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
@@ -203,7 +203,7 @@ public class MapReduceExecutable extends AbstractExecutable {
         if (yarnStatusCheckUrl != null) {
             return yarnStatusCheckUrl;
         } else {
-            logger.info(KylinConfig.KYLIN_JOB_YARN_APP_REST_CHECK_URL + " is not set, read from job configuration");
+            logger.info("kylin.job.yarn.app.rest.check.status.url" + " is not set, read from job configuration");
         }
         String rmWebHost = HAUtil.getConfValueForRMInstance(YarnConfiguration.RM_WEBAPP_ADDRESS, YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS, job.getConfiguration());
         if(HAUtil.isHAEnabled(job.getConfiguration())) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
index 3502271..05b56aa 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeStatisticsStep.java
@@ -105,7 +105,7 @@ public class MergeStatisticsStep extends AbstractExecutable {
                         if (key.get() == 0l) {
                             // sampling percentage;
                             averageSamplingPercentage += Bytes.toInt(value.getBytes());
-                        } else {
+                        } else if (key.get() > 0) {
                             HyperLogLogPlusCounter hll = new HyperLogLogPlusCounter(14);
                             ByteArray byteArray = new ByteArray(value.getBytes());
                             hll.readRegisters(byteArray.asBuffer());

http://git-wip-us.apache.org/repos/asf/kylin/blob/99ee1b65/server/src/main/java/org/apache/kylin/rest/security/CrossDomainFilter.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/security/CrossDomainFilter.java b/server/src/main/java/org/apache/kylin/rest/security/CrossDomainFilter.java
index 79dfe50..c9167b9 100644
--- a/server/src/main/java/org/apache/kylin/rest/security/CrossDomainFilter.java
+++ b/server/src/main/java/org/apache/kylin/rest/security/CrossDomainFilter.java
@@ -53,7 +53,7 @@ public class CrossDomainFilter implements Filter {
      */
     @Override
     public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
-        if (Boolean.parseBoolean(KylinConfig.getInstanceFromEnv().getProperty("crossdomain.enable", "true"))) {
+        if (Boolean.parseBoolean(KylinConfig.getInstanceFromEnv().getOptional("crossdomain.enable", "true"))) {
             ((HttpServletResponse) response).addHeader("Access-Control-Allow-Origin", "*");
             ((HttpServletResponse) response).addHeader("Access-Control-Allow-Methods", "GET, POST, PUT, DELETE, OPTIONS");
             ((HttpServletResponse) response).addHeader("Access-Control-Allow-Headers", "Origin, No-Cache, X-Requested-With, If-Modified-Since, Pragma, Last-Modified, Cache-Control, Expires, Content-Type, X-E4M-With, Accept, Authorization");