From 1531f0a27e4a896f01599401482c86b1f699f600 Mon Sep 17 00:00:00 2001 From: wyj Date: Sun, 15 Jul 2018 00:40:11 -0700 Subject: [PATCH] KYLIN-3168 CubeHFileJob should use currentHBaseConfiguration but not new create hbase configuration and the introduced bug in KYLIN-3438 mapreduce.job.queuename does not work at 'Convert Cuboid Data to HFile' Step --- .../main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java index ce4ceac..5ffdd48 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java @@ -46,6 +46,8 @@ import org.apache.kylin.storage.hbase.HBaseConnection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hbase.HBaseConfiguration.merge; + /** * @author George Song (ysong1) */ @@ -76,6 +78,7 @@ public class CubeHFileJob extends AbstractHadoopJob { // use current hbase configuration Configuration configuration = HBaseConnection.getCurrentHBaseConfiguration(); + merge(configuration, getConf()); job = Job.getInstance(configuration, getOptionValue(OPTION_JOB_NAME)); setJobClasspath(job, cube.getConfig()); -- 1.9.1