This is an automated email from the ASF dual-hosted git repository.

shaofengshi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git


The following commit(s) were added to refs/heads/master by this push:
     new 9f7ec94  KYLIN-3874 "Convert Cuboid Data to HFile" failed when HBase 
and MR use different HDFS clusters
9f7ec94 is described below

commit 9f7ec94c9dee4f0f0946318bff52a8e186c0da87
Author: Guangxu Cheng <gxch...@apache.org>
AuthorDate: Fri Mar 15 19:10:35 2019 +0800

    KYLIN-3874 "Convert Cuboid Data to HFile" failed when HBase and MR use 
different HDFS clusters
---
 .../org/apache/kylin/storage/hbase/steps/CubeHFileJob.java  | 13 +++++++++++++
 1 file changed, 13 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 ec2998b..c0fae42 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
@@ -19,6 +19,7 @@
 package org.apache.kylin.storage.hbase.steps;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
@@ -78,7 +79,10 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
             // use current hbase configuration
             Configuration configuration = new 
Configuration(HBaseConnection.getCurrentHBaseConfiguration());
+            String[] allServices = getAllServices(configuration);
             merge(configuration, getConf());
+            configuration.setStrings(DFSConfigKeys.DFS_NAMESERVICES, 
allServices);
+
             job = Job.getInstance(configuration, 
getOptionValue(OPTION_JOB_NAME));
 
             setJobClasspath(job, cube.getConfig());
@@ -153,6 +157,15 @@ public class CubeHFileJob extends AbstractHadoopJob {
         }
     }
 
+    private String[] getAllServices(Configuration hbaseConf) {
+        Collection<String> hbaseHdfsServices
+            = 
hbaseConf.getTrimmedStringCollection(DFSConfigKeys.DFS_NAMESERVICES);
+        Collection<String> mainNameServices
+            = 
getConf().getTrimmedStringCollection(DFSConfigKeys.DFS_NAMESERVICES);
+        mainNameServices.addAll(hbaseHdfsServices);
+        return mainNameServices.toArray(new String[0]);
+    }
+
     public static void main(String[] args) throws Exception {
         int exitCode = ToolRunner.run(new CubeHFileJob(), args);
         System.exit(exitCode);

Reply via email to