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 f9c3adb..c7dcda2 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 @@ -224,6 +224,10 @@ abstract public class KylinConfigBase implements Serializable { return cachedHdfsWorkingDirectory; } + public String getZkBasePath() { + return getOptional("kylin.zk.base.path", "/kylin"); + } + /** * A comma separated list of host:port pairs, each corresponding to a ZooKeeper server */ diff --git a/core-common/src/main/resources/kylin-defaults.properties b/core-common/src/main/resources/kylin-defaults.properties index cb511e7..7a411f3 100644 --- a/core-common/src/main/resources/kylin-defaults.properties +++ b/core-common/src/main/resources/kylin-defaults.properties @@ -26,6 +26,8 @@ kylin.env.hdfs-working-dir=/kylin # DEV|QA|PROD. DEV will turn on some dev features, QA and PROD has no difference in terms of functions. kylin.env=QA +# kylin zk base path +kylin.zk.base.path=/kylin ### SERVER | WEB ### diff --git a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java index 9db9dbb..cf0c509 100644 --- a/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java +++ b/core-job/src/main/java/org/apache/kylin/job/impl/threadpool/DistributedScheduler.java @@ -83,7 +83,7 @@ public class DistributedScheduler implements Scheduler, Conn private final static String SEGMENT_ID = "segmentId"; - public static final String ZOOKEEPER_LOCK_PATH = "/job_engine/lock"; // note ZookeeperDistributedLock will ensure zk path prefix: /kylin/metadata + public static final String ZOOKEEPER_LOCK_PATH = "/job_engine/lock"; // note ZookeeperDistributedLock will ensure zk path prefix: /${kylin.zk.base.path}/metadata //only for it test public static DistributedScheduler getInstance(KylinConfig config) { diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java index 63ffda0..de15ebd 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ZookeeperDistributedLock.java @@ -33,6 +33,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.KylinConfigBase; import org.apache.kylin.common.lock.DistributedLock; import org.apache.kylin.common.lock.DistributedLockFactory; import org.apache.kylin.job.lock.JobLock; @@ -102,7 +103,7 @@ public class ZookeeperDistributedLock implements DistributedLock, JobLock { public Factory(KylinConfig config) { this.curator = getZKClient(config); - this.zkPathBase = fixSlash("/kylin/" + KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix()); + this.zkPathBase = fixSlash(config.getZkBasePath() + KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix()); } @Override