KYLIN-2757 support getting counters for non-hdfs storage

Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/3fac37fa
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/3fac37fa
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/3fac37fa

Branch: refs/heads/2.0.x-hbase0.98
Commit: 3fac37fa8966751ec737f17a15b9b1277335b6a3
Parents: a58cbb7
Author: shaofengshi <shaofeng...@apache.org>
Authored: Thu Jul 27 21:53:12 2017 +0800
Committer: shaofengshi <shaofeng...@apache.org>
Committed: Sat Jul 29 07:36:03 2017 +0800

----------------------------------------------------------------------
 .../kylin/engine/mr/common/HadoopCmdOutput.java    | 17 +++++++++--------
 1 file changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3fac37fa/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
index 5da1947..11d6d2c 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopCmdOutput.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.FileSystemCounter;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import 
org.apache.kylin.engine.mr.steps.FactDistinctColumnsMapper.RawDataCounter;
@@ -94,17 +95,17 @@ public class HadoopCmdOutput {
             }
             this.output.append(counters.toString()).append("\n");
             logger.debug(counters.toString());
-            
-            String bytsWrittenCounterName = "HDFS_BYTES_WRITTEN";
-            String fsScheme = 
FileSystem.get(job.getConfiguration()).getScheme();
-            if (("wasb").equalsIgnoreCase(fsScheme)) {
-                // for Azure blob store
-                bytsWrittenCounterName = "WASB_BYTES_WRITTEN";
-            }
 
             mapInputRecords = 
String.valueOf(counters.findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue());
-            hdfsBytesWritten = 
String.valueOf(counters.findCounter("FileSystemCounters", 
bytsWrittenCounterName).getValue());
             rawInputBytesRead = 
String.valueOf(counters.findCounter(RawDataCounter.BYTES).getValue());
+
+            String fsScheme = 
FileSystem.get(job.getConfiguration()).getScheme();
+            long bytesWritten = counters.findCounter(fsScheme, 
FileSystemCounter.BYTES_WRITTEN).getValue();
+            if (bytesWritten == 0) {
+                bytesWritten = counters.findCounter("FileSystemCounters", 
"HDFS_BYTES_WRITTEN").getValue();
+            }
+            hdfsBytesWritten = String.valueOf(bytesWritten);
+
         } catch (Exception e) {
             logger.error(e.getLocalizedMessage(), e);
             output.append(e.getLocalizedMessage());

Reply via email to