You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by xx...@apache.org on 2020/09/02 01:53:04 UTC

[kylin] branch master updated: KYLIN-4688 Too many tmp files in HDFS tmp dictionary

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

xxyu 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 b519ecd  KYLIN-4688 Too many tmp files in HDFS tmp dictionary
b519ecd is described below

commit b519ecddc0dd104ea07f3ab9cc0cb6a7ff6118ac
Author: ITzhangqiang <IT...@163.com>
AuthorDate: Fri Aug 14 17:51:25 2020 +0800

    KYLIN-4688 Too many tmp files in HDFS tmp dictionary
---
 .../src/main/java/org/apache/kylin/common/util/HadoopUtil.java    | 8 --------
 .../java/org/apache/kylin/storage/hbase/steps/CubeHFileJob.java   | 6 +++++-
 2 files changed, 5 insertions(+), 9 deletions(-)

diff --git a/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
index 73fdf38..0f6da04 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/HadoopUtil.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.kylin.shaded.com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -69,13 +68,6 @@ public class HadoopUtil {
     }
 
     public static Configuration healSickConfig(Configuration conf) {
-        // https://issues.apache.org/jira/browse/KYLIN-953
-        if (StringUtils.isBlank(conf.get("hadoop.tmp.dir"))) {
-            conf.set("hadoop.tmp.dir", "/tmp");
-        }
-        if (StringUtils.isBlank(conf.get("hbase.fs.tmp.dir"))) {
-            conf.set("hbase.fs.tmp.dir", "/tmp");
-        }
         //  https://issues.apache.org/jira/browse/KYLIN-3064
         conf.set("yarn.timeline-service.enabled", "false");
         return conf;
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 3217d75..5ea2392 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
@@ -58,6 +58,7 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
     public int run(String[] args) throws Exception {
         Options options = new Options();
+        Path tmpPartitionFilePath = null;
 
         try {
             options.addOption(OPTION_JOB_NAME);
@@ -101,6 +102,7 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
             // Automatic config !
             HFileOutputFormat3.configureIncrementalLoad(job, htable);
+            tmpPartitionFilePath = new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));
             HFileOutputFormat3.configureHConnection(job, hbaseConf, getJobTempDir());
 
             reconfigurePartitions(configuration, partitionFilePath);
@@ -119,8 +121,10 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
             return waitForCompletion(job);
         } finally {
-            if (job != null)
+            if (job != null) {
                 cleanupTempConfFile(job.getConfiguration());
+                this.deletePath(job.getConfiguration(), tmpPartitionFilePath);
+            }
         }
     }