Repository: kylin Updated Branches: refs/heads/master 8530ebd69 -> 89875fae7
KYLIN-2195 re-format KylinConfigBase Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/89875fae Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/89875fae Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/89875fae Branch: refs/heads/master Commit: 89875fae78556d4d5f5099accb70d6572faeb3d4 Parents: 8530ebd Author: Li Yang <liy...@apache.org> Authored: Fri Nov 25 14:49:58 2016 +0800 Committer: Li Yang <liy...@apache.org> Committed: Fri Nov 25 14:49:58 2016 +0800 ---------------------------------------------------------------------- .../apache/kylin/common/KylinConfigBase.java | 778 ++++++++++--------- 1 file changed, 411 insertions(+), 367 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/89875fae/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java ---------------------------------------------------------------------- diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index 6131013..8ea03be 100644 --- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -142,16 +142,47 @@ abstract public class KylinConfigBase implements Serializable { this.properties = BCC.check(properties); } + private Map<Integer, String> convertKeyToInteger(Map<String, String> map) { + Map<Integer, String> result = Maps.newLinkedHashMap(); + for (Entry<String, String> entry : map.entrySet()) { + result.put(Integer.valueOf(entry.getKey()), entry.getValue()); + } + return result; + } + + public String toString() { + return getMetadataUrl(); + } + + // ============================================================================ + // ENV // ============================================================================ public boolean isDevEnv() { return "DEV".equals(getOptional("kylin.env", "DEV")); } + + public String getDeployEnv() { + return getOptional("kylin.env", "DEV"); + } + + public String getHdfsWorkingDirectory() { + String root = getRequired("kylin.env.hdfs-working-dir"); + if (!root.endsWith("/")) { + root += "/"; + } + return new StringBuffer(root).append(StringUtils.replaceChars(getMetadataUrlPrefix(), ':', '-')).append("/").toString(); + } + + // ============================================================================ + // METADATA + // ============================================================================ public String getMetadataUrl() { return getOptional("kylin.metadata.url"); } + // for test only public void setMetadataUrl(String metadataUrl) { setProperty("kylin.metadata.url", metadataUrl); } @@ -169,178 +200,120 @@ abstract public class KylinConfigBase implements Serializable { } } - public String getServerMode() { - return this.getOptional("kylin.server.mode", "all"); - } - - public String getStorageUrl() { - return getOptional("kylin.storage.url"); - } - - public void setStorageUrl(String storageUrl) { - setProperty("kylin.storage.url", storageUrl); + public String[] getRealizationProviders() { + return getOptionalStringArray("kylin.metadata.realization-providers", // + new String[] { "org.apache.kylin.cube.CubeManager", "org.apache.kylin.storage.hybrid.HybridManager" }); } - /** - * was for route to hive, not used any more - */ - @Deprecated - public String getHiveUrl() { - return getOptional("kylin.source.hive.connection-url", ""); + public String[] getCubeDimensionCustomEncodingFactories() { + return getOptionalStringArray("kylin.metadata.custom-dimension-encodings", new String[0]); } - /** - * was for route to hive, not used any more - */ - @Deprecated - public String getHiveUser() { - return getOptional("kylin.source.hive.connection-user", ""); + public Map<String, String> getCubeCustomMeasureTypes() { + return getPropertiesByPrefix("kylin.metadata.custom-measure-types."); } + + // ============================================================================ + // DICTIONARY & SNAPSHOT + // ============================================================================ - /** - * was for route to hive, not used any more - */ - @Deprecated - public String getHivePassword() { - return getOptional("kylin.source.hive.connection-password", ""); + public int getTrieDictionaryForestMaxTrieSizeMB() { + return Integer.parseInt(getOptional("kylin.dictionary.forest-trie-max-mb", "500")); } - public String getHdfsWorkingDirectory() { - String root = getRequired("kylin.env.hdfs-working-dir"); - if (!root.endsWith("/")) { - root += "/"; - } - return new StringBuffer(root).append(StringUtils.replaceChars(getMetadataUrlPrefix(), ':', '-')).append("/").toString(); + public int getCachedDictMaxEntrySize() { + return Integer.parseInt(getOptional("kylin.dictionary.max-cache-entry", "3000")); } - public String[] getRealizationProviders() { - return getOptionalStringArray("kylin.metadata.realization-providers", // - new String[] { "org.apache.kylin.cube.CubeManager", "org.apache.kylin.storage.hybrid.HybridManager" }); + public boolean isGrowingDictEnabled() { + return Boolean.parseBoolean(this.getOptional("kylin.dictionary.growing-enabled", "false")); } - public CliCommandExecutor getCliCommandExecutor() throws IOException { - CliCommandExecutor exec = new CliCommandExecutor(); - if (getRunAsRemoteCommand()) { - exec.setRunAtRemote(getRemoteHadoopCliHostname(), getRemoteHadoopCliPort(), getRemoteHadoopCliUsername(), getRemoteHadoopCliPassword()); - } - return exec; + public int getAppendDictEntrySize() { + return Integer.parseInt(getOptional("kylin.dictionary.append-entry-size", "10000000")); } - public String getHBaseClusterFs() { - return getOptional("kylin.storage.hbase.cluster-fs", ""); + // for test + public void setAppendDictEntrySize(int entrySize) { + setProperty("kylin.dictionary.append-entry-size", String.valueOf(entrySize)); } - public String getHBaseClusterHDFSConfigFile() { - return getOptional("kylin.storage.hbase.cluster-hdfs-config-file", ""); + public int getAppendDictCacheSize() { + return Integer.parseInt(getOptional("kylin.dictionary.append-cache-size", "20")); } - public String getKylinJobLogDir() { - return getOptional("kylin.job.log-dir", "/tmp/kylin/logs"); + // for test + public void setAppendDictCacheSize(int cacheSize) { + setProperty("kylin.dictionary.append-cache-size", String.valueOf(cacheSize)); } - public String getKylinJobJarPath() { - final String jobJar = getOptional("kylin.engine.mr.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 int getCachedSnapshotMaxEntrySize() { + return Integer.parseInt(getOptional("kylin.snapshot.max-cache-entry", "500")); } - public void overrideMRJobJarPath(String path) { - logger.info("override " + "kylin.engine.mr.job-jar" + " to " + path); - System.setProperty("kylin.engine.mr.job-jar", path); + public int getTableSnapshotMaxMB() { + return Integer.parseInt(getOptional("kylin.snapshot.max-mb", "300")); } - public String getKylinJobMRLibDir() { - return getOptional("kylin.engine.mr.lib-dir", ""); - } + // ============================================================================ + // CUBE + // ============================================================================ - public Map<String, String> getMRConfigOverride() { - return getPropertiesByPrefix("kylin.engine.mr.config-override."); + public double getJobCuboidSizeRatio() { + return Double.parseDouble(getOptional("kylin.cube.size-estimate-ratio", "0.25")); } - public Map<String, String> getHiveConfigOverride() { - return getPropertiesByPrefix("kylin.source.hive.config-override."); + public double getJobCuboidSizeMemHungryRatio() { + return Double.parseDouble(getOptional("kylin.cube.size-estimate-memhungry-ratio", "0.05")); } - public String getKylinSparkJobJarPath() { - final String jobJar = getOptional("kylin.engine.mr.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 String getCubeAlgorithm() { + return getOptional("kylin.cube.algorithm", "auto"); } - public void overrideSparkJobJarPath(String path) { - logger.info("override " + "kylin.engine.mr.job-jar.spark" + " to " + path); - System.setProperty("kylin.engine.mr.job-jar.spark", path); + public double getCubeAlgorithmAutoThreshold() { + return Double.parseDouble(getOptional("kylin.cube.algorithm.layer-or-inmem-threshold", "7")); } - 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.storage.hbase.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 int getCubeAlgorithmAutoMapperLimit() { + return Integer.parseInt(getOptional("kylin.cube.algorithm.inmem-split-limit", "500")); } - public void overrideCoprocessorLocalJar(String path) { - logger.info("override " + "kylin.storage.hbase.coprocessor-local-jar" + " to " + path); - System.setProperty("kylin.storage.hbase.coprocessor-local-jar", path); + @Deprecated + public int getCubeAggrGroupMaxSize() { + return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max-size", "12")); } - 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 int getCubeAggrGroupMaxCombination() { + return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max-combination", "4096")); } - public double getDefaultHadoopJobReducerInputMB() { - return Double.parseDouble(getOptional("kylin.engine.mr.reduce-input-mb", "500")); + public boolean getCubeAggrGroupIsMandatoryOnlyValid() { + return Boolean.parseBoolean(getOptional("kylin.cube.aggrgroup.is-mandatory-only-valid", "false")); } - public double getDefaultHadoopJobReducerCountRatio() { - return Double.parseDouble(getOptional("kylin.engine.mr.reduce-count-ratio", "1.0")); + public int getMaxBuildingSegments() { + return Integer.parseInt(getOptional("kylin.cube.max-building-segments", "10")); } - public int getHadoopJobMinReducerNumber() { - return Integer.parseInt(getOptional("kylin.engine.mr.min-reducer-number", "1")); + public void setMaxBuildingSegments(int maxBuildingSegments) { + setProperty("kylin.cube.max-building-segments", String.valueOf(maxBuildingSegments)); } + + // ============================================================================ + // JOB + // ============================================================================ - public int getHadoopJobMaxReducerNumber() { - return Integer.parseInt(getOptional("kylin.engine.mr.max-reducer-number", "500")); + public CliCommandExecutor getCliCommandExecutor() throws IOException { + CliCommandExecutor exec = new CliCommandExecutor(); + if (getRunAsRemoteCommand()) { + exec.setRunAtRemote(getRemoteHadoopCliHostname(), getRemoteHadoopCliPort(), getRemoteHadoopCliUsername(), getRemoteHadoopCliPassword()); + } + return exec; } - public int getHadoopJobMapperInputRows() { - return Integer.parseInt(getOptional("kylin.engine.mr.mapper-input-rows", "1000000")); + public String getKylinJobLogDir() { + return getOptional("kylin.job.log-dir", "/tmp/kylin/logs"); } public boolean getRunAsRemoteCommand() { @@ -387,177 +360,233 @@ abstract public class KylinConfigBase implements Serializable { return Boolean.parseBoolean(getOptional("kylin.job.allow-empty-segment", "true")); } - //UHC: ultra high cardinality columns, contain the ShardByColumns and the GlobalDictionaryColumns - public int getUHCReducerCount() { - return Integer.parseInt(getOptional("kylin.engine.mr.uhc-reducer-count", "3")); + public int getMaxConcurrentJobLimit() { + return Integer.parseInt(getOptional("kylin.job.max-concurrent-jobs", "10")); } - public String getOverrideHiveTableLocation(String table) { - return getOptional("kylin.source.hive.table-location." + table.toUpperCase()); + public String[] getAdminDls() { + return getOptionalStringArray("kylin.job.notification-admin-emails", null); } - public String getYarnStatusCheckUrl() { - return getOptional("kylin.engine.mr.yarn-check-status-url", null); + public int getCubingInMemSamplingPercent() { + int percent = Integer.parseInt(this.getOptional("kylin.job.sampling-percentage", "100")); + percent = Math.max(percent, 1); + percent = Math.min(percent, 100); + return percent; } - public int getYarnStatusCheckIntervalSeconds() { - return Integer.parseInt(getOptional("kylin.engine.mr.yarn-check-interval-seconds", "60")); + public String getHiveDependencyFilterList() { + return this.getOptional("kylin.job.dependency-filter-list", "[^,]*hive-exec[0-9.-]+[^,]*?\\.jar" + "|" + "[^,]*hive-metastore[0-9.-]+[^,]*?\\.jar" + "|" + "[^,]*hive-hcatalog-core[0-9.-]+[^,]*?\\.jar"); } - public int getMaxConcurrentJobLimit() { - return Integer.parseInt(getOptional("kylin.job.max-concurrent-jobs", "10")); + public boolean isMailEnabled() { + return Boolean.parseBoolean(getOptional("kylin.job.notification-enabled", "false")); } - public String getTimeZone() { - return getOptional("kylin.web.timezone", "PST"); + public void setMailEnabled(boolean enable) { + setProperty("kylin.job.notification-enabled", "" + enable); } - public String[] getRestServers() { - return getOptionalStringArray("kylin.server.cluster-servers", new String[0]); + public String getMailHost() { + return getOptional("kylin.job.notification-mail-host", ""); } - public String getClusterName() { - return this.getOptional("kylin.server.cluster-name", getMetadataUrlPrefix()); + public String getMailUsername() { + return getOptional("kylin.job.notification-mail-username", ""); } - public int getWorkersPerServer() { - //for sequence sql use - return Integer.parseInt(getOptional("kylin.server.sequence-sql.workers-per-server", "1")); + public String getMailPassword() { + return getOptional("kylin.job.notification-mail-password", ""); } - public String[] getAdminDls() { - return getOptionalStringArray("kylin.job.notification-admin-emails", null); + public String getMailSender() { + return getOptional("kylin.job.notification-mail-sender", ""); } - public double getJobCuboidSizeRatio() { - return Double.parseDouble(getOptional("kylin.cube.size-estimate-ratio", "0.25")); + public int getJobRetry() { + return Integer.parseInt(this.getOptional("kylin.job.retry", "0")); } - public double getJobCuboidSizeMemHungryRatio() { - return Double.parseDouble(getOptional("kylin.cube.size-estimate-memhungry-ratio", "0.05")); + public int getCubeStatsHLLPrecision() { + return Integer.parseInt(getOptional("kylin.job.sampling-hll-precision", "14")); } - public String getCubeAlgorithm() { - return getOptional("kylin.cube.algorithm", "auto"); + public String getJobControllerLock() { + return getOptional("kylin.job.lock", "org.apache.kylin.storage.hbase.util.ZookeeperJobLock"); } - public double getCubeAlgorithmAutoThreshold() { - return Double.parseDouble(getOptional("kylin.cube.algorithm.layer-or-inmem-threshold", "7")); + public Map<Integer, String> getSchedulers() { + Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.job.scheduler.provider.")); + r.put(0, "org.apache.kylin.job.impl.threadpool.DefaultScheduler"); + r.put(2, "org.apache.kylin.job.impl.threadpool.DistributedScheduler"); + return r; } - public int getCubeAlgorithmAutoMapperLimit() { - return Integer.parseInt(getOptional("kylin.cube.algorithm.inmem-split-limit", "500")); + public Integer getSchedulerType() { + return Integer.parseInt(getOptional("kylin.job.scheduler.default", "0")); } - @Deprecated - public int getCubeAggrGroupMaxSize() { - return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max-size", "12")); - } + // ============================================================================ + // SOURCE.HIVE + // ============================================================================ - public int getCubeAggrGroupMaxCombination() { - return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max-combination", "4096")); + public Map<Integer, String> getSourceEngines() { + Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.source.provider.")); + // ref constants in ISourceAware + r.put(0, "org.apache.kylin.source.hive.HiveSource"); + r.put(1, "org.apache.kylin.source.kafka.KafkaSource"); + return r; } - public boolean getCubeAggrGroupIsMandatoryOnlyValid() { - return Boolean.parseBoolean(getOptional("kylin.cube.aggrgroup.is-mandatory-only-valid", "false")); + /** + * was for route to hive, not used any more + */ + @Deprecated + public String getHiveUrl() { + return getOptional("kylin.source.hive.connection-url", ""); } - public String[] getCubeDimensionCustomEncodingFactories() { - return getOptionalStringArray("kylin.metadata.custom-dimension-encodings", new String[0]); + /** + * was for route to hive, not used any more + */ + @Deprecated + public String getHiveUser() { + return getOptional("kylin.source.hive.connection-user", ""); } - public Map<String, String> getCubeCustomMeasureTypes() { - return getPropertiesByPrefix("kylin.metadata.custom-measure-types."); + /** + * was for route to hive, not used any more + */ + @Deprecated + public String getHivePassword() { + return getOptional("kylin.source.hive.connection-password", ""); } - public int getTableSnapshotMaxMB() { - return Integer.parseInt(getOptional("kylin.snapshot.max-mb", "300")); + public Map<String, String> getHiveConfigOverride() { + return getPropertiesByPrefix("kylin.source.hive.config-override."); } - public int getTrieDictionaryForestMaxTrieSizeMB() { - return Integer.parseInt(getOptional("kylin.dictionary.forest-trie-max-mb", "500")); + public String getOverrideHiveTableLocation(String table) { + return getOptional("kylin.source.hive.table-location." + table.toUpperCase()); } - public int getHBaseRegionCountMin() { - return Integer.parseInt(getOptional("kylin.storage.hbase.min-region-count", "1")); + public boolean isHiveKeepFlatTable() { + return Boolean.parseBoolean(this.getOptional("kylin.source.hive.keep-flat-table", "false")); } - public int getHBaseRegionCountMax() { - return Integer.parseInt(getOptional("kylin.storage.hbase.max-region-count", "500")); + public String getHiveDatabaseForIntermediateTable() { + return this.getOptional("kylin.source.hive.database-for-flat-table", "default"); } - // for test only - public void setHBaseHFileSizeGB(float size) { - setProperty("kylin.storage.hbase.hfile-size-gb", String.valueOf(size)); + public boolean isHiveRedistributeEnabled() { + return Boolean.parseBoolean(this.getOptional("kylin.source.hive.redistribute-flat-table", "true")); } - public float getHBaseHFileSizeGB() { - return Float.parseFloat(getOptional("kylin.storage.hbase.hfile-size-gb", "2.0")); + public String getHiveClientMode() { + return getOptional("kylin.source.hive.client", "cli"); } - //check KYLIN-1684, in most cases keep the default value - public boolean isSkippingEmptySegments() { - return Boolean.valueOf(getOptional("kylin.query.skip-empty-segments", "true")); + public String getHiveBeelineParams() { + return getOptional("kylin.source.hive.beeline-params", ""); } - public int getStoragePushDownLimitMax() { - return Integer.parseInt(getOptional("kylin.query.max-limit-pushdown", "10000")); + @Deprecated + public String getCreateFlatHiveTableMethod() { + return getOptional("kylin.source.hive.create-flat-table-method", "1"); } - public int getScanThreshold() { - return Integer.parseInt(getOptional("kylin.query.scan-threshold", "10000000")); + // ============================================================================ + // STORAGE.HBASE + // ============================================================================ + + public Map<Integer, String> getStorageEngines() { + Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider.")); + // ref constants in IStorageAware + r.put(0, "org.apache.kylin.storage.hbase.HBaseStorage"); + r.put(1, "org.apache.kylin.storage.hybrid.HybridStorage"); + r.put(2, "org.apache.kylin.storage.hbase.HBaseStorage"); + return r; } - public int getDerivedInThreshold() { - return Integer.parseInt(getOptional("kylin.query.derived-filter-translation-threshold", "20")); + public int getDefaultStorageEngine() { + return Integer.parseInt(getOptional("kylin.storage.default", "2")); } - public int getBadQueryStackTraceDepth() { - return Integer.parseInt(getOptional("kylin.query.badquery-stacktrace-depth", "10")); + public String getStorageUrl() { + return getOptional("kylin.storage.url"); } - public int getBadQueryHistoryNum() { - return Integer.parseInt(getOptional("kylin.query.badquery-history-number", "10")); + // for test only + public void setStorageUrl(String storageUrl) { + setProperty("kylin.storage.url", storageUrl); } - public int getBadQueryDefaultAlertingSeconds() { - return Integer.parseInt(getOptional("kylin.query.badquery-alerting-seconds", "90")); + public String getHBaseClusterFs() { + return getOptional("kylin.storage.hbase.cluster-fs", ""); } - public int getBadQueryDefaultDetectIntervalSeconds() { - return Integer.parseInt(getOptional("kylin.query.badquery-detect-interval", "60")); + public String getHBaseClusterHDFSConfigFile() { + return getOptional("kylin.storage.hbase.cluster-hdfs-config-file", ""); } - public boolean getBadQueryPersistentEnabled() { - return Boolean.parseBoolean(getOptional("kylin.query.badquery-persistent-enabled", "true")); + 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.storage.hbase.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 String[] getQueryTransformers() { - return getOptionalStringArray("kylin.query.transformers", new String[0]); + public void overrideCoprocessorLocalJar(String path) { + logger.info("override " + "kylin.storage.hbase.coprocessor-local-jar" + " to " + path); + System.setProperty("kylin.storage.hbase.coprocessor-local-jar", path); } - public int getCachedDictMaxEntrySize() { - return Integer.parseInt(getOptional("kylin.dictionary.max-cache-entry", "3000")); + 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 int getCachedSnapshotMaxEntrySize() { - return Integer.parseInt(getOptional("kylin.snapshot.max-cache-entry", "500")); + public int getHBaseRegionCountMin() { + return Integer.parseInt(getOptional("kylin.storage.hbase.min-region-count", "1")); } - public boolean getQueryRunLocalCoprocessor() { - return Boolean.parseBoolean(getOptional("kylin.storage.hbase.run-local-coprocessor", "false")); + public int getHBaseRegionCountMax() { + return Integer.parseInt(getOptional("kylin.storage.hbase.max-region-count", "500")); } - public long getQueryDurationCacheThreshold() { - return Long.parseLong(this.getOptional("kylin.query.cache-threshold-duration", String.valueOf(2000))); + // for test only + public void setHBaseHFileSizeGB(float size) { + setProperty("kylin.storage.hbase.hfile-size-gb", String.valueOf(size)); } - public long getQueryScanCountCacheThreshold() { - return Long.parseLong(this.getOptional("kylin.query.cache-threshold-scan-count", String.valueOf(10 * 1024))); + public float getHBaseHFileSizeGB() { + return Float.parseFloat(getOptional("kylin.storage.hbase.hfile-size-gb", "2.0")); } - public long getQueryMemBudget() { - return Long.parseLong(this.getOptional("kylin.query.memory-budget-bytes", String.valueOf(3L * 1024 * 1024 * 1024))); + public boolean getQueryRunLocalCoprocessor() { + return Boolean.parseBoolean(getOptional("kylin.storage.hbase.run-local-coprocessor", "false")); } public double getQueryCoprocessorMemGB() { @@ -568,18 +597,6 @@ abstract public class KylinConfigBase implements Serializable { return Integer.parseInt(this.getOptional("kylin.storage.hbase.coprocessor-timeout-seconds", "0")); } - public boolean isQuerySecureEnabled() { - return Boolean.parseBoolean(this.getOptional("kylin.query.security-enabled", "true")); - } - - 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 getQueryScanFuzzyKeyMax() { return Integer.parseInt(this.getOptional("kylin.storage.hbase.max-fuzzykey-scan", "200")); } @@ -588,23 +605,6 @@ abstract public class KylinConfigBase implements Serializable { return Integer.valueOf(this.getOptional("kylin.storage.hbase.max-visit-scanrange", "1000000")); } - public String getQueryAccessController() { - return getOptional("kylin.query.access-controller", null); - } - - public long getSequenceExpireTime() { - return Long.valueOf(this.getOptional("kylin.server.sequence-sql.expire-time", "86400000"));//default a day - } - - public boolean getQueryMetricsEnabled() { - return Boolean.parseBoolean(getOptional("kylin.server.query-metrics-enabled", "false")); - } - - public int[] getQueryMetricsPercentilesIntervals() { - String[] dft = { "60", "300", "3600" }; - return getOptionalIntArray("kylin.server.query-metrics-percentiles-intervals", dft); - } - public String getDefaultIGTStorage() { return getOptional("kylin.storage.hbase.gtstorage", "org.apache.kylin.storage.hbase.cube.v2.CubeHBaseEndpointRPC"); } @@ -613,15 +613,6 @@ abstract public class KylinConfigBase implements Serializable { return Integer.parseInt(this.getOptional("kylin.storage.hbase.scan-cache-rows", "1024")); } - public boolean isGrowingDictEnabled() { - return Boolean.parseBoolean(this.getOptional("kylin.dictionary.growing-enabled", "false")); - } - - /** - * HBase region cut size, in GB - * - * @return - */ public float getKylinHBaseRegionCut() { return Float.valueOf(getOptional("kylin.storage.hbase.region-cut-gb", "5.0")); } @@ -630,13 +621,6 @@ abstract public class KylinConfigBase implements Serializable { return Integer.parseInt(this.getOptional("kylin.storage.hbase.max-scan-result-bytes", "" + (5 * 1024 * 1024))); // 5 MB } - public int getCubingInMemSamplingPercent() { - int percent = Integer.parseInt(this.getOptional("kylin.job.sampling-percentage", "100")); - percent = Math.max(percent, 1); - percent = Math.min(percent, 100); - return percent; - } - public String getHbaseDefaultCompressionCodec() { return getOptional("kylin.storage.hbase.compression-codec", "none"); } @@ -653,152 +637,204 @@ abstract public class KylinConfigBase implements Serializable { return Integer.valueOf(getOptional("kylin.storage.hbase.small-family-block-size-bytes", "65536")); } - public boolean isHiveKeepFlatTable() { - return Boolean.parseBoolean(this.getOptional("kylin.source.hive.keep-flat-table", "false")); + public String getKylinOwner() { + return this.getOptional("kylin.storage.hbase.owner-tag", ""); } - public String getHiveDatabaseForIntermediateTable() { - return this.getOptional("kylin.source.hive.database-for-flat-table", "default"); + public boolean getCompressionResult() { + return Boolean.parseBoolean(getOptional("kylin.storage.hbase.endpoint-compress-result", "true")); } - public boolean isHiveRedistributeEnabled() { - return Boolean.parseBoolean(this.getOptional("kylin.source.hive.redistribute-flat-table", "true")); + public int getHBaseMaxConnectionThreads() { + return Integer.parseInt(getOptional("kylin.storage.hbase.max-hconnection-threads", "2048")); } - public String getHiveDependencyFilterList() { - return this.getOptional("kylin.job.dependency-filter-list", "[^,]*hive-exec[0-9.-]+[^,]*?\\.jar" + "|" + "[^,]*hive-metastore[0-9.-]+[^,]*?\\.jar" + "|" + "[^,]*hive-hcatalog-core[0-9.-]+[^,]*?\\.jar"); + public int getHBaseCoreConnectionThreads() { + return Integer.parseInt(getOptional("kylin.storage.hbase.core-hconnection-threads", "2048")); } - public String getKylinOwner() { - return this.getOptional("kylin.storage.hbase.owner-tag", ""); + public long getHBaseConnectionThreadPoolAliveSeconds() { + return Long.parseLong(getOptional("kylin.storage.hbase.hconnection-threads-alive-seconds", "60")); } - public String getSparkHome() { - return getRequired("kylin.engine.spark.spark-home"); + // ============================================================================ + // ENGINE.MR + // ============================================================================ + + public Map<Integer, String> getJobEngines() { + Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.engine.provider.")); + // ref constants in IEngineAware + r.put(0, "org.apache.kylin.engine.mr.MRBatchCubingEngine"); + r.put(2, "org.apache.kylin.engine.mr.MRBatchCubingEngine2"); + return r; } - public String getSparkMaster() { - return getRequired("kylin.engine.spark.spark-master"); + public int getDefaultCubeEngine() { + return Integer.parseInt(getOptional("kylin.engine.default", "2")); } - public boolean isMailEnabled() { - return Boolean.parseBoolean(getOptional("kylin.job.notification-enabled", "false")); + public String getKylinJobJarPath() { + final String jobJar = getOptional("kylin.engine.mr.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 setMailEnabled(boolean enable) { - setProperty("kylin.job.notification-enabled", "" + enable); + public void overrideMRJobJarPath(String path) { + logger.info("override " + "kylin.engine.mr.job-jar" + " to " + path); + System.setProperty("kylin.engine.mr.job-jar", path); } - public String getMailHost() { - return getOptional("kylin.job.notification-mail-host", ""); + public String getKylinJobMRLibDir() { + return getOptional("kylin.engine.mr.lib-dir", ""); } - public String getMailUsername() { - return getOptional("kylin.job.notification-mail-username", ""); + public Map<String, String> getMRConfigOverride() { + return getPropertiesByPrefix("kylin.engine.mr.config-override."); } - public String getMailPassword() { - return getOptional("kylin.job.notification-mail-password", ""); + public double getDefaultHadoopJobReducerInputMB() { + return Double.parseDouble(getOptional("kylin.engine.mr.reduce-input-mb", "500")); } - public String getMailSender() { - return getOptional("kylin.job.notification-mail-sender", ""); + public double getDefaultHadoopJobReducerCountRatio() { + return Double.parseDouble(getOptional("kylin.engine.mr.reduce-count-ratio", "1.0")); } - public boolean isWebCrossDomainEnabled() { - return Boolean.parseBoolean(getOptional("kylin.web.cross-domain-enabled", "true")); + public int getHadoopJobMinReducerNumber() { + return Integer.parseInt(getOptional("kylin.engine.mr.min-reducer-number", "1")); } - public int getJobRetry() { - return Integer.parseInt(this.getOptional("kylin.job.retry", "0")); + public int getHadoopJobMaxReducerNumber() { + return Integer.parseInt(getOptional("kylin.engine.mr.max-reducer-number", "500")); } - public String toString() { - return getMetadataUrl(); + public int getHadoopJobMapperInputRows() { + return Integer.parseInt(getOptional("kylin.engine.mr.mapper-input-rows", "1000000")); } - public String getHiveClientMode() { - return getOptional("kylin.source.hive.client", "cli"); + //UHC: ultra high cardinality columns, contain the ShardByColumns and the GlobalDictionaryColumns + public int getUHCReducerCount() { + return Integer.parseInt(getOptional("kylin.engine.mr.uhc-reducer-count", "3")); } - public String getHiveBeelineParams() { - return getOptional("kylin.source.hive.beeline-params", ""); + public String getYarnStatusCheckUrl() { + return getOptional("kylin.engine.mr.yarn-check-status-url", null); } - public String getDeployEnv() { - return getOptional("kylin.env", "DEV"); + public int getYarnStatusCheckIntervalSeconds() { + return Integer.parseInt(getOptional("kylin.engine.mr.yarn-check-interval-seconds", "60")); } - public String getInitTasks() { - return getOptional("kylin.server.init-tasks"); + // ============================================================================ + // ENGINE.SPARK + // ============================================================================ + + public String getKylinSparkJobJarPath() { + final String jobJar = getOptional("kylin.engine.spark.job-jar"); + 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 int getDimCountDistinctMaxCardinality() { - return Integer.parseInt(getOptional("kylin.query.max-dimension-count-distinct", "5000000")); + public void overrideSparkJobJarPath(String path) { + logger.info("override " + "kylin.engine.spark.job-jar" + " to " + path); + System.setProperty("kylin.engine.spark.job-jar", path); } - public int getCubeStatsHLLPrecision() { - return Integer.parseInt(getOptional("kylin.job.sampling-hll-precision", "14")); + public String getSparkHome() { + return getRequired("kylin.engine.spark.spark-home"); } - public String getJobControllerLock() { - return getOptional("kylin.job.lock", "org.apache.kylin.storage.hbase.util.ZookeeperJobLock"); + public String getSparkMaster() { + return getRequired("kylin.engine.spark.spark-master"); } - public Map<Integer, String> getJobEngines() { - Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.engine.provider.")); - // ref constants in IEngineAware - r.put(0, "org.apache.kylin.engine.mr.MRBatchCubingEngine"); - r.put(2, "org.apache.kylin.engine.mr.MRBatchCubingEngine2"); - return r; + // ============================================================================ + // QUERY + // ============================================================================ + + //check KYLIN-1684, in most cases keep the default value + public boolean isSkippingEmptySegments() { + return Boolean.valueOf(getOptional("kylin.query.skip-empty-segments", "true")); } - public Map<Integer, String> getSourceEngines() { - Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.source.provider.")); - // ref constants in ISourceAware - r.put(0, "org.apache.kylin.source.hive.HiveSource"); - r.put(1, "org.apache.kylin.source.kafka.KafkaSource"); - return r; + public int getStoragePushDownLimitMax() { + return Integer.parseInt(getOptional("kylin.query.max-limit-pushdown", "10000")); } - public Map<Integer, String> getStorageEngines() { - Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.storage.provider.")); - // ref constants in IStorageAware - r.put(0, "org.apache.kylin.storage.hbase.HBaseStorage"); - r.put(1, "org.apache.kylin.storage.hybrid.HybridStorage"); - r.put(2, "org.apache.kylin.storage.hbase.HBaseStorage"); - return r; + public int getScanThreshold() { + return Integer.parseInt(getOptional("kylin.query.scan-threshold", "10000000")); } - public int getDefaultStorageEngine() { - return Integer.parseInt(getOptional("kylin.storage.default", "2")); + public int getDerivedInThreshold() { + return Integer.parseInt(getOptional("kylin.query.derived-filter-translation-threshold", "20")); } - public int getDefaultCubeEngine() { - return Integer.parseInt(getOptional("kylin.engine.default", "2")); + public int getBadQueryStackTraceDepth() { + return Integer.parseInt(getOptional("kylin.query.badquery-stacktrace-depth", "10")); } - public Map<Integer, String> getSchedulers() { - Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.job.scheduler.provider.")); - r.put(0, "org.apache.kylin.job.impl.threadpool.DefaultScheduler"); - r.put(2, "org.apache.kylin.job.impl.threadpool.DistributedScheduler"); - return r; + public int getBadQueryHistoryNum() { + return Integer.parseInt(getOptional("kylin.query.badquery-history-number", "10")); } - public Integer getSchedulerType() { - return Integer.parseInt(getOptional("kylin.job.scheduler.default", "0")); + public int getBadQueryDefaultAlertingSeconds() { + return Integer.parseInt(getOptional("kylin.query.badquery-alerting-seconds", "90")); } - private Map<Integer, String> convertKeyToInteger(Map<String, String> map) { - Map<Integer, String> result = Maps.newLinkedHashMap(); - for (Entry<String, String> entry : map.entrySet()) { - result.put(Integer.valueOf(entry.getKey()), entry.getValue()); - } - return result; + public int getBadQueryDefaultDetectIntervalSeconds() { + return Integer.parseInt(getOptional("kylin.query.badquery-detect-interval", "60")); } - public boolean getCompressionResult() { - return Boolean.parseBoolean(getOptional("kylin.storage.hbase.endpoint-compress-result", "true")); + public boolean getBadQueryPersistentEnabled() { + return Boolean.parseBoolean(getOptional("kylin.query.badquery-persistent-enabled", "true")); + } + + public String[] getQueryTransformers() { + return getOptionalStringArray("kylin.query.transformers", new String[0]); + } + + 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-scan-count", String.valueOf(10 * 1024))); + } + + public long getQueryMemBudget() { + return Long.parseLong(this.getOptional("kylin.query.memory-budget-bytes", String.valueOf(3L * 1024 * 1024 * 1024))); + } + + public boolean isQuerySecureEnabled() { + return Boolean.parseBoolean(this.getOptional("kylin.query.security-enabled", "true")); + } + + 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 String getQueryAccessController() { + return getOptional("kylin.query.access-controller", null); + } + + public int getDimCountDistinctMaxCardinality() { + return Integer.parseInt(getOptional("kylin.query.max-dimension-count-distinct", "5000000")); } public Map<String, String> getUDFs() { @@ -806,46 +842,54 @@ abstract public class KylinConfigBase implements Serializable { return udfMap; } - public int getHBaseMaxConnectionThreads() { - return Integer.parseInt(getOptional("kylin.storage.hbase.max-hconnection-threads", "2048")); + // ============================================================================ + // SERVER + // ============================================================================ + + public String getServerMode() { + return this.getOptional("kylin.server.mode", "all"); } - public int getHBaseCoreConnectionThreads() { - return Integer.parseInt(getOptional("kylin.storage.hbase.core-hconnection-threads", "2048")); + public String[] getRestServers() { + return getOptionalStringArray("kylin.server.cluster-servers", new String[0]); } - public long getHBaseConnectionThreadPoolAliveSeconds() { - return Long.parseLong(getOptional("kylin.storage.hbase.hconnection-threads-alive-seconds", "60")); + public String getClusterName() { + return this.getOptional("kylin.server.cluster-name", getMetadataUrlPrefix()); } - public int getAppendDictEntrySize() { - return Integer.parseInt(getOptional("kylin.dictionary.append-entry-size", "10000000")); + public String getInitTasks() { + return getOptional("kylin.server.init-tasks"); } - // for test - public void setAppendDictEntrySize(int entrySize) { - setProperty("kylin.dictionary.append-entry-size", String.valueOf(entrySize)); + public int getWorkersPerServer() { + //for sequence sql use + return Integer.parseInt(getOptional("kylin.server.sequence-sql.workers-per-server", "1")); } - public int getAppendDictCacheSize() { - return Integer.parseInt(getOptional("kylin.dictionary.append-cache-size", "20")); + public long getSequenceExpireTime() { + return Long.valueOf(this.getOptional("kylin.server.sequence-sql.expire-time", "86400000"));//default a day } - // for test - public void setAppendDictCacheSize(int cacheSize) { - setProperty("kylin.dictionary.append-cache-size", String.valueOf(cacheSize)); + public boolean getQueryMetricsEnabled() { + return Boolean.parseBoolean(getOptional("kylin.server.query-metrics-enabled", "false")); } - @Deprecated - public String getCreateFlatHiveTableMethod() { - return getOptional("kylin.source.hive.create-flat-table-method", "1"); + public int[] getQueryMetricsPercentilesIntervals() { + String[] dft = { "60", "300", "3600" }; + return getOptionalIntArray("kylin.server.query-metrics-percentiles-intervals", dft); } - public int getMaxBuildingSegments() { - return Integer.parseInt(getOptional("kylin.cube.max-building-segments", "10")); + // ============================================================================ + // WEB + // ============================================================================ + + public String getTimeZone() { + return getOptional("kylin.web.timezone", "PST"); } - public void setMaxBuildingSegments(int maxBuildingSegments) { - setProperty("kylin.cube.max-building-segments", String.valueOf(maxBuildingSegments)); + public boolean isWebCrossDomainEnabled() { + return Boolean.parseBoolean(getOptional("kylin.web.cross-domain-enabled", "true")); } + }