You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by rb...@apache.org on 2016/07/27 22:26:38 UTC

tez git commit: TEZ-3379. Tez analyzer: Move sysout to log4j (rbalamohan)

Repository: tez
Updated Branches:
  refs/heads/master d3011a9bc -> 30eaa1e47


TEZ-3379. Tez analyzer: Move sysout to log4j (rbalamohan)


Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/30eaa1e4
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/30eaa1e4
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/30eaa1e4

Branch: refs/heads/master
Commit: 30eaa1e47f9bde5dc368ddb15e762049091f13bb
Parents: d3011a9
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu Jul 28 03:56:26 2016 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Thu Jul 28 03:56:26 2016 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../tez/history/parser/ATSFileParser.java       |  6 ++--
 .../analyzer/plugins/CriticalPathAnalyzer.java  | 30 +++++++++++---------
 .../tez/analyzer/plugins/TezAnalyzerBase.java   | 20 +++++++------
 .../org/apache/tez/analyzer/utils/SVGUtils.java |  5 +++-
 5 files changed, 36 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/30eaa1e4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 550087a..3d198fd 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3379. Tez analyzer: Move sysout to log4j.
   TEZ-3376. Fix groupId generation to account for dagId starting with 1.
   TEZ-3359. Add granular log levels for HistoryLoggingService.
   TEZ-3374. Change TEZ_HISTORY_LOGGING_TIMELINE_NUM_DAGS_PER_GROUP conf key name.

http://git-wip-us.apache.org/repos/asf/tez/blob/30eaa1e4/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
index 89db976..b4f3df3 100644
--- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
@@ -95,12 +95,12 @@ public class ATSFileParser extends BaseParser implements ATSData {
     //Process vertex information
     Preconditions.checkState(verticesJson != null, "Vertex json can not be null");
     if (verticesJson != null) {
-      LOG.info("Started parsing vertex");
+      LOG.debug("Started parsing vertex");
       for (int i = 0; i < verticesJson.length(); i++) {
         VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i));
         vertexList.add(vertexInfo);
       }
-      LOG.info("Finished parsing vertex");
+      LOG.debug("Finished parsing vertex");
     }
   }
 
@@ -195,7 +195,7 @@ public class ATSFileParser extends BaseParser implements ATSData {
       Enumeration<? extends ZipEntry> zipEntries = atsZipFile.entries();
       while (zipEntries.hasMoreElements()) {
         ZipEntry zipEntry = zipEntries.nextElement();
-        LOG.info("Processing " + zipEntry.getName());
+        LOG.debug("Processing " + zipEntry.getName());
         InputStream inputStream = atsZipFile.getInputStream(zipEntry);
         JSONObject jsonObject = readJson(inputStream);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/30eaa1e4/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
index 88c5134..2edce3e 100644
--- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
@@ -47,9 +47,13 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
 
+  private static final Logger LOG = LoggerFactory.getLogger(CriticalPathAnalyzer.class);
+
   String succeededState = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name());
   String failedState = StringInterner.weakIntern(TaskAttemptState.FAILED.name());
 
@@ -138,7 +142,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
     }
     
     if (lastAttempt == null) {
-      System.out.println("Cannot find last attempt to finish in DAG " + dagInfo.getDagId());
+      LOG.info("Cannot find last attempt to finish in DAG " + dagInfo.getDagId());
       return;
     }
     
@@ -162,7 +166,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
       outputDir = getConf().get(OUTPUT_DIR);
     }
     String outputFileName = outputDir + File.separator + dagInfo.getDagId() + ".svg";
-    System.out.println("Writing output to: " + outputFileName);
+    LOG.info("Writing output to: " + outputFileName);
     svg.saveCriticalPathAsSVG(dagInfo, outputFileName, criticalPath);
   }
   
@@ -259,7 +263,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
         for (TaskAttemptInfo a : t.getTaskAttempts()) {
           if (a.getTerminationCause().equals(
               TaskAttemptTerminationCause.INTERNAL_PREEMPTION.name())) {
-            System.out.println("Found preempted attempt " + a.getTaskAttemptId());
+            LOG.debug("Found preempted attempt " + a.getTaskAttemptId());
             preemptedAttempts.add(a);
           }
         }
@@ -415,8 +419,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
         }
       }
     }
-    System.out
-        .println("DAG time taken: " + dagTime + " TotalAttemptTime: " + totalAttemptCriticalTime
+    LOG.debug("DAG time taken: " + dagTime + " TotalAttemptTime: " + totalAttemptCriticalTime
             + " DAG finish time: " + dag.getFinishTime() + " DAG start time: " + dagStartTime);
   }
   
@@ -451,8 +454,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
       while (true) {
         Preconditions.checkState(currentAttempt != null);
         Preconditions.checkState(currentAttemptStopCriticalPathTime > 0);
-        System.out.println(
-            "Step: " + tempCP.size() + " Attempt: " + currentAttempt.getTaskAttemptId());
+        LOG.debug("Step: " + tempCP.size() + " Attempt: " + currentAttempt.getTaskAttemptId());
         
         currentStep = new CriticalPathStep(currentAttempt, EntityType.ATTEMPT);
         currentStep.stopCriticalPathTime = currentAttemptStopCriticalPathTime;
@@ -501,13 +503,13 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
           // last data event was produced after the attempt was scheduled. use
           // data dependency
           // typically the case when scheduling ahead of time
-          System.out.println("Has data dependency");
+          LOG.debug("Has data dependency");
           if (!Strings.isNullOrEmpty(currentStepLastDataTA)) {
             // there is a valid data causal TA. Use it.
             nextAttemptId = currentStepLastDataTA;
             reason = CriticalPathDependency.DATA_DEPENDENCY;
             startCriticalPathTime = currentStepLastDataEventTime;
-            System.out.println("Using data dependency " + nextAttemptId);
+            LOG.debug("Using data dependency " + nextAttemptId);
           } else {
             // there is no valid data causal TA. This means data event came from the same vertex
             VertexInfo vertex = currentAttempt.getTaskInfo().getVertexInfo();
@@ -516,12 +518,12 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
                     + "TA is null for " + currentAttempt.getTaskAttemptId());
             nextAttemptId = null;
             reason = CriticalPathDependency.INIT_DEPENDENCY;
-            System.out.println("Using init dependency");
+            LOG.debug("Using init dependency");
           }
         } else {
           // attempt was scheduled after last data event. use scheduling dependency
           // typically happens for retries
-          System.out.println("Has scheduling dependency");
+          LOG.debug("Has scheduling dependency");
           if (!Strings.isNullOrEmpty(currentAttempt.getCreationCausalTA())) {
             // there is a scheduling causal TA. Use it.
             nextAttemptId = currentAttempt.getCreationCausalTA();
@@ -553,7 +555,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
                   currentAttempt.getTaskInfo().getTaskId()));
               startCriticalPathTime = nextAttempt.getFinishTime();
             }
-            System.out.println("Using scheduling dependency " + nextAttemptId);
+            LOG.debug("Using scheduling dependency " + nextAttemptId);
           } else {
             // there is no scheduling causal TA.
             if (!Strings.isNullOrEmpty(currentStepLastDataTA)) {
@@ -565,7 +567,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
               long overhead = currentAttempt.getCreationTime() - currentStepLastDataEventTime;
               currentStep.notes
                   .add("Initializer/VertexManager scheduling overhead " + SVGUtils.getTimeStr(overhead));
-              System.out.println("Using data dependency " + nextAttemptId);
+              LOG.debug("Using data dependency " + nextAttemptId);
             } else {
               // there is no scheduling causal TA and no data event casual TA.
               // the vertex has external input that sent the last data events
@@ -573,7 +575,7 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
               // or the vertex has no external inputs or edges
               nextAttemptId = null;
               reason = CriticalPathDependency.INIT_DEPENDENCY;
-              System.out.println("Using init dependency");
+              LOG.debug("Using init dependency");
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/30eaa1e4/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
index 73e731a..38254c8 100644
--- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
@@ -40,9 +40,12 @@ import org.apache.tez.history.parser.SimpleHistoryParser;
 import org.apache.tez.history.parser.datamodel.DagInfo;
 
 import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class TezAnalyzerBase extends Configured implements Tool, Analyzer {
 
+  private static final Logger LOG = LoggerFactory.getLogger(TezAnalyzerBase.class);
   
   private static final String EVENT_FILE_NAME = "eventFileName";
   private static final String OUTPUT_DIR = "outputDir";
@@ -176,7 +179,7 @@ public abstract class TezAnalyzerBase extends Configured implements Tool, Analyz
       String fileName = outputDir + File.separator
           + this.getClass().getName() + "_" + dagInfo.getDagId() + ".csv";
       ((CSVResult) result).dumpToFile(fileName);
-      System.out.println("Saved results in " + fileName);
+      LOG.info("Saved results in " + fileName);
     }
     return 0;
   }
@@ -193,21 +196,22 @@ public abstract class TezAnalyzerBase extends Configured implements Tool, Analyz
       }
       String format = formatBuilder.toString();
 
-      StringBuilder separator = new StringBuilder();
+      StringBuilder sep = new StringBuilder();
       for (int i = 0; i < SEPARATOR_WIDTH; i++) {
-        separator.append("-");
+        sep.append("-");
       }
 
-      System.out.println(separator);
-      System.out.println(String.format(format.toString(), (String[]) headers));
-      System.out.println(separator);
+      String separator = sep.toString();
+      LOG.debug(separator);
+      LOG.debug(String.format(format.toString(), (String[]) headers));
+      LOG.debug(separator);
 
       Iterator<String[]> recordsIterator = ((CSVResult) result).getRecordsIterator();
       while (recordsIterator.hasNext()) {
         String line = String.format(format, (String[]) recordsIterator.next());
-        System.out.println(line);
+        LOG.debug(line);
       }
-      System.out.println(separator);
+      LOG.debug(separator);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/30eaa1e4/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
index 78cb921..90acf3b 100644
--- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
@@ -25,6 +25,8 @@ import org.apache.tez.history.parser.datamodel.DagInfo;
 import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
 
 import com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.BufferedWriter;
 import java.io.IOException;
@@ -35,6 +37,7 @@ import java.util.concurrent.TimeUnit;
 
 public class SVGUtils {
 
+  private static final Logger LOG = LoggerFactory.getLogger(SVGUtils.class);
   private static int MAX_DAG_RUNTIME = 0;
   private static final int SCREEN_WIDTH = 1800;
 
@@ -167,7 +170,7 @@ public class SVGUtils {
       int launchTimeInterval = attempt.getStartTime() > 0 ? 
           (int) (attempt.getStartTime() - dagStartTime) : 0;
       int finishTimeInterval = (int) (attempt.getFinishTime() - dagStartTime);
-      System.out.println(attempt.getTaskAttemptId() + " " + creationTimeInterval + " "
+      LOG.debug(attempt.getTaskAttemptId() + " " + creationTimeInterval + " "
           + allocationTimeInterval + " " + launchTimeInterval + " " + finishTimeInterval);
 
       StringBuilder title = new StringBuilder();