You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by zh...@apache.org on 2017/03/28 19:47:42 UTC

[01/50] [abbrv] tez git commit: TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels. Contributed by Harish Jaiprakash. [Forced Update!]

Repository: tez
Updated Branches:
  refs/heads/TEZ-1190 cc1c5cc73 -> d1b08e37e (forced update)


TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at
specific levels. Contributed by Harish Jaiprakash.


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

Branch: refs/heads/TEZ-1190
Commit: c0270cb30a582ab2b5cbc8442054ce0c2a766c15
Parents: a77d22d
Author: Siddharth Seth <ss...@apache.org>
Authored: Sun Feb 5 18:24:15 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Sun Feb 5 18:24:15 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../org/apache/tez/dag/api/HistoryLogLevel.java |   1 +
 .../apache/tez/dag/api/TezConfiguration.java    |  12 +-
 .../org/apache/tez/common/TezUtilsInternal.java |  19 ++++
 .../tez/dag/history/HistoryEventHandler.java    | 114 ++++++++++++++++---
 .../tez/dag/history/HistoryEventType.java       |   4 +-
 .../dag/history/TestHistoryEventHandler.java    |  79 ++++++++++---
 7 files changed, 199 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bf9c9bc..37438d9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3600. Fix flaky test: TestTokenCache
   TEZ-3589. add a unit test for amKeepAlive not being shutdown if an app takes a long time to launch.
   TEZ-3417. Reduce sleep time on AM shutdown to reduce test runtimes
@@ -194,6 +195,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3462. Task attempt failure during container shutdown loses useful container diagnostics
   TEZ-3574. Container reuse won't pickup extra dag level local resource.
   TEZ-3566. Avoid caching fs isntances in TokenCache after a point.

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-api/src/main/java/org/apache/tez/dag/api/HistoryLogLevel.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/HistoryLogLevel.java b/tez-api/src/main/java/org/apache/tez/dag/api/HistoryLogLevel.java
index 5eb4785..96d74f9 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/HistoryLogLevel.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/HistoryLogLevel.java
@@ -34,6 +34,7 @@ public enum HistoryLogLevel {
   DAG,
   VERTEX,
   TASK,
+  TASK_ATTEMPT,
   ALL;
 
   public static final HistoryLogLevel DEFAULT = ALL;

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index df0605c..fd71b35 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -20,7 +20,6 @@ package org.apache.tez.dag.api;
 
 import java.lang.reflect.Field;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -1263,6 +1262,17 @@ public class TezConfiguration extends Configuration {
       TEZ_PREFIX + "history.logging.log.level";
 
   /**
+   * List of comma separated enum values. Specifies the list of task attempt termination causes,
+   * which have to be suppressed from being logged to ATS. The valid filters are defined in the
+   * enum TaskAttemptTerminationCause. The filters are applied only if tez.history.logging.log.level
+   * is set to TASK_ATTEMPT.
+   */
+  @ConfigurationScope(Scope.DAG)
+  @ConfigurationProperty
+  public static final String TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS =
+      TEZ_PREFIX + "history.logging.taskattempt-filters";
+
+  /**
    * Comma separated list of Integers. These are the values that were set for the config value
    * for {@value #TEZ_HISTORY_LOGGING_TIMELINE_NUM_DAGS_PER_GROUP}. The older values are required so
    * that the groupIds generated previously will continue to be generated by the plugin. If an older

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index 7b19293..5ba2972 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -25,8 +25,10 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.charset.Charset;
 import java.util.BitSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -41,6 +43,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.log4j.Appender;
 import org.apache.tez.common.io.NonSyncByteArrayOutputStream;
@@ -338,6 +341,22 @@ public class TezUtilsInternal {
     }
   }
 
+  public static <T extends Enum<T>> Set<T> getEnums(Configuration conf, String confName,
+      Class<T> enumType, String defaultValues) {
+    String[] names = conf.getStrings(confName);
+    if (names == null) {
+      names = StringUtils.getStrings(defaultValues);
+    }
+    if (names == null) {
+      return null;
+    }
+    Set<T> enums = new HashSet<>();
+    for (String name : names) {
+      enums.add(Enum.valueOf(enumType, name));
+    }
+    return enums;
+  }
+
   @Private
   public static void setHadoopCallerContext(HadoopShim hadoopShim, TezTaskAttemptID attemptID) {
     hadoopShim.setHadoopCallerContext("tez_ta:" + attemptID.toString());

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
index 042d022..79d1fc3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.history;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.slf4j.Logger;
@@ -27,13 +28,18 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.dag.api.HistoryLogLevel;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.history.events.DAGSubmittedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.logging.HistoryLoggingService;
 import org.apache.tez.dag.history.recovery.RecoveryService;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
 
 public class HistoryEventHandler extends CompositeService {
 
@@ -45,8 +51,13 @@ public class HistoryEventHandler extends CompositeService {
   private HistoryLoggingService historyLoggingService;
 
   private HistoryLogLevel amHistoryLogLevel;
-  private Map<TezDAGID, HistoryLogLevel> dagIdToLogLevel =
-      new ConcurrentHashMap<TezDAGID, HistoryLogLevel>();
+  private final Map<TezDAGID, HistoryLogLevel> dagIdToLogLevel = new ConcurrentHashMap<>();
+  private Set<TaskAttemptTerminationCause> amTaskAttemptFilters;
+  private final Map<TezDAGID, Set<TaskAttemptTerminationCause>> dagIdToTaskAttemptFilters =
+      new ConcurrentHashMap<>();
+
+  private final ConcurrentHashMap<TezTaskAttemptID, DAGHistoryEvent> suppressedEvents =
+      new ConcurrentHashMap<>();
 
   public HistoryEventHandler(AppContext context) {
     super(HistoryEventHandler.class.getName());
@@ -80,6 +91,11 @@ public class HistoryEventHandler extends CompositeService {
     }
 
     amHistoryLogLevel = HistoryLogLevel.getLogLevel(context.getAMConf(), HistoryLogLevel.DEFAULT);
+    amTaskAttemptFilters = TezUtilsInternal.getEnums(
+        context.getAMConf(),
+        TezConfiguration.TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS,
+        TaskAttemptTerminationCause.class,
+        null);
 
     super.serviceInit(conf);
   }
@@ -108,15 +124,20 @@ public class HistoryEventHandler extends CompositeService {
     if(dagId != null) {
       dagIdStr = dagId.toString();
     }
+    HistoryEvent historyEvent = event.getHistoryEvent();
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Handling history event"
-          + ", eventType=" + event.getHistoryEvent().getEventType());
+          + ", eventType=" + historyEvent.getEventType());
     }
-    if (recoveryEnabled && event.getHistoryEvent().isRecoveryEvent()) {
+    if (recoveryEnabled && historyEvent.isRecoveryEvent()) {
       recoveryService.handle(event);
     }
-    if (event.getHistoryEvent().isHistoryEvent() && shouldLogEvent(event)) {
+    if (historyEvent.isHistoryEvent() && shouldLogEvent(event)) {
+      DAGHistoryEvent suppressedEvent = getSupressedEvent(historyEvent);
+      if (suppressedEvent != null) {
+        historyLoggingService.handle(suppressedEvent);
+      }
       historyLoggingService.handle(event);
     }
 
@@ -140,23 +161,86 @@ public class HistoryEventHandler extends CompositeService {
     }
 
     HistoryEvent historyEvent = event.getHistoryEvent();
-    if (historyEvent.getEventType() == HistoryEventType.DAG_SUBMITTED) {
-      dagLogLevel = HistoryLogLevel.getLogLevel(((DAGSubmittedEvent)historyEvent).getConf(),
-          amHistoryLogLevel);
+    HistoryEventType eventType = historyEvent.getEventType();
+    if (eventType == HistoryEventType.DAG_SUBMITTED) {
+      Configuration dagConf = ((DAGSubmittedEvent)historyEvent).getConf();
+      dagLogLevel = HistoryLogLevel.getLogLevel(dagConf, amHistoryLogLevel);
       dagIdToLogLevel.put(dagId, dagLogLevel);
-    } else if (historyEvent.getEventType() == HistoryEventType.DAG_RECOVERED) {
+      maybeUpdateDagTaskAttemptFilters(dagId, dagLogLevel, dagConf);
+    } else if (eventType == HistoryEventType.DAG_RECOVERED) {
       if (context.getCurrentDAG() != null) {
-        dagLogLevel = HistoryLogLevel.getLogLevel(context.getCurrentDAG().getConf(),
-            amHistoryLogLevel);
+        Configuration dagConf = context.getCurrentDAG().getConf();
+        dagLogLevel = HistoryLogLevel.getLogLevel(dagConf, amHistoryLogLevel);
         dagIdToLogLevel.put(dagId, dagLogLevel);
+        maybeUpdateDagTaskAttemptFilters(dagId, dagLogLevel, dagConf);
+      }
+    } else if (eventType == HistoryEventType.DAG_FINISHED) {
+      dagIdToLogLevel.remove(dagId);
+      dagIdToTaskAttemptFilters.remove(dagId);
+      suppressedEvents.clear();
+    }
+
+    if (dagLogLevel.shouldLog(historyEvent.getEventType().getHistoryLogLevel())) {
+      return shouldLogTaskAttemptEvents(event, dagLogLevel);
+    }
+    return false;
+  }
+
+  // If the log level is set to TASK_ATTEMPT and filters are configured, then we should suppress
+  // the start event and publish it only when TaskAttemptFinishedEvent is received after
+  // matching against the filter.
+  // Note: if the AM is killed before we get the TaskAttemptFinishedEvent, we'll lose this event.
+  private boolean shouldLogTaskAttemptEvents(DAGHistoryEvent event, HistoryLogLevel dagLogLevel) {
+    HistoryEvent historyEvent = event.getHistoryEvent();
+    HistoryEventType eventType = historyEvent.getEventType();
+    if (dagLogLevel == HistoryLogLevel.TASK_ATTEMPT &&
+        (eventType == HistoryEventType.TASK_ATTEMPT_STARTED ||
+         eventType == HistoryEventType.TASK_ATTEMPT_FINISHED)) {
+      TezDAGID dagId = event.getDagID();
+      Set<TaskAttemptTerminationCause> filters = null;
+      if (dagId != null) {
+        filters = dagIdToTaskAttemptFilters.get(dagId);
+      }
+      if (filters == null) {
+        filters = amTaskAttemptFilters;
+      }
+      if (filters == null) {
+        return true;
       }
-    } else if (historyEvent.getEventType() == HistoryEventType.DAG_FINISHED) {
-      if (dagIdToLogLevel.containsKey(dagId)) {
-        dagIdToLogLevel.remove(dagId);
+      if (eventType == HistoryEventType.TASK_ATTEMPT_STARTED) {
+        suppressedEvents.put(((TaskAttemptStartedEvent)historyEvent).getTaskAttemptID(), event);
+        return false;
+      } else { // TaskAttemptFinishedEvent
+        TaskAttemptFinishedEvent finishedEvent = (TaskAttemptFinishedEvent)historyEvent;
+        if (filters.contains(finishedEvent.getTaskAttemptError())) {
+          suppressedEvents.remove(finishedEvent.getTaskAttemptID());
+          return false;
+        }
       }
     }
+    return true;
+  }
 
-    return dagLogLevel.shouldLog(historyEvent.getEventType().getHistoryLogLevel());
+  private void maybeUpdateDagTaskAttemptFilters(TezDAGID dagId, HistoryLogLevel dagLogLevel,
+      Configuration dagConf) {
+    if (dagLogLevel == HistoryLogLevel.TASK_ATTEMPT) {
+      Set<TaskAttemptTerminationCause> filters = TezUtilsInternal.getEnums(
+          dagConf,
+          TezConfiguration.TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS,
+          TaskAttemptTerminationCause.class,
+          null);
+      if (filters != null) {
+        dagIdToTaskAttemptFilters.put(dagId, filters);
+      }
+    }
+  }
+
+  private DAGHistoryEvent getSupressedEvent(HistoryEvent historyEvent) {
+    if (historyEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) {
+      TaskAttemptFinishedEvent finishedEvent = (TaskAttemptFinishedEvent)historyEvent;
+      return suppressedEvents.remove(finishedEvent.getTaskAttemptID());
+    }
+    return null;
   }
 
   public void handle(DAGHistoryEvent event) {

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
index a41d0e6..a536fdf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
@@ -36,8 +36,8 @@ public enum HistoryEventType {
   VERTEX_FINISHED(HistoryLogLevel.VERTEX),
   TASK_STARTED(HistoryLogLevel.TASK),
   TASK_FINISHED(HistoryLogLevel.TASK),
-  TASK_ATTEMPT_STARTED(HistoryLogLevel.ALL),
-  TASK_ATTEMPT_FINISHED(HistoryLogLevel.ALL),
+  TASK_ATTEMPT_STARTED(HistoryLogLevel.TASK_ATTEMPT),
+  TASK_ATTEMPT_FINISHED(HistoryLogLevel.TASK_ATTEMPT),
   CONTAINER_LAUNCHED(HistoryLogLevel.ALL),
   CONTAINER_STOPPED(HistoryLogLevel.ALL),
   DAG_COMMIT_STARTED(HistoryLogLevel.DAG),

http://git-wip-us.apache.org/repos/asf/tez/blob/c0270cb3/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
index c8a076d..4c0fe3f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
@@ -32,23 +32,29 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.tez.dag.api.HistoryLogLevel;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.DAGState;
 import org.apache.tez.dag.history.events.AMStartedEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+import org.apache.tez.dag.history.events.ContainerStoppedEvent;
 import org.apache.tez.dag.history.events.DAGFinishedEvent;
 import org.apache.tez.dag.history.events.DAGRecoveredEvent;
 import org.apache.tez.dag.history.events.DAGSubmittedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
 import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.logging.HistoryLoggingService;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.hadoop.shim.HadoopShim;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestHistoryEventHandler {
@@ -56,42 +62,69 @@ public class TestHistoryEventHandler {
   private static ApplicationId appId = ApplicationId.newInstance(1000l, 1);
   private static ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
   private static String user = "TEST_USER";
+  private Configuration baseConfig;
+
+  @Before
+  public void setupConfig() {
+    baseConfig = new Configuration(false);
+  }
 
   @Test
   public void testAll() {
-    testLogLevel(null, 6);
+    testLogLevel(null, 11);
     testLogLevel(HistoryLogLevel.NONE, 0);
     testLogLevel(HistoryLogLevel.AM, 1);
     testLogLevel(HistoryLogLevel.DAG, 3);
     testLogLevel(HistoryLogLevel.VERTEX, 4);
     testLogLevel(HistoryLogLevel.TASK, 5);
-    testLogLevel(HistoryLogLevel.ALL, 6);
+    testLogLevel(HistoryLogLevel.TASK_ATTEMPT, 9);
+    testLogLevel(HistoryLogLevel.ALL, 11);
+  }
+
+  @Test
+  public void testTaskAttemptFilters() {
+    baseConfig.set(TezConfiguration.TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS,
+        "EXTERNAL_PREEMPTION,INTERRUPTED_BY_USER");
+    testLogLevel(HistoryLogLevel.TASK_ATTEMPT, 5);
+    testLogLevelWithRecovery(HistoryLogLevel.TASK_ATTEMPT, 5);
+
+    baseConfig.set(TezConfiguration.TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS,
+        "EXTERNAL_PREEMPTION");
+    testLogLevel(HistoryLogLevel.TASK_ATTEMPT, 7);
+    testLogLevelWithRecovery(HistoryLogLevel.TASK_ATTEMPT, 7);
+
+    baseConfig.set(TezConfiguration.TEZ_HISTORY_LOGGING_TASKATTEMPT_FILTERS, "INTERNAL_PREEMPTION");
+    testLogLevel(HistoryLogLevel.TASK_ATTEMPT, 9);
+    testLogLevelWithRecovery(HistoryLogLevel.TASK_ATTEMPT, 9);
   }
 
   @Test
   public void testWithDAGRecovery() {
-    testLogLevelWithRecovery(null, 6);
+    testLogLevelWithRecovery(null, 11);
     testLogLevelWithRecovery(HistoryLogLevel.AM, 1);
     testLogLevelWithRecovery(HistoryLogLevel.DAG, 3);
     testLogLevelWithRecovery(HistoryLogLevel.VERTEX, 4);
     testLogLevelWithRecovery(HistoryLogLevel.TASK, 5);
-    testLogLevelWithRecovery(HistoryLogLevel.ALL, 6);
+    testLogLevelWithRecovery(HistoryLogLevel.TASK_ATTEMPT, 9);
+    testLogLevelWithRecovery(HistoryLogLevel.ALL, 11);
   }
 
   @Test
   public void testMultipleDag() {
-    testLogLevel(null, HistoryLogLevel.NONE, 7);
-    testLogLevel(null, HistoryLogLevel.AM, 7);
-    testLogLevel(null, HistoryLogLevel.DAG, 9);
-    testLogLevel(null, HistoryLogLevel.VERTEX, 10);
-    testLogLevel(null, HistoryLogLevel.TASK, 11);
-    testLogLevel(null, HistoryLogLevel.ALL, 12);
+    testLogLevel(null, HistoryLogLevel.NONE, 14);
+    testLogLevel(null, HistoryLogLevel.AM, 14);
+    testLogLevel(null, HistoryLogLevel.DAG, 16);
+    testLogLevel(null, HistoryLogLevel.VERTEX, 17);
+    testLogLevel(null, HistoryLogLevel.TASK, 18);
+    testLogLevel(null, HistoryLogLevel.TASK_ATTEMPT, 22);
+    testLogLevel(null, HistoryLogLevel.ALL, 22);
     testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.NONE, 5);
     testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.AM, 5);
     testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.DAG, 7);
     testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.VERTEX, 8);
     testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.TASK, 9);
-    testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.ALL, 10);
+    testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.TASK_ATTEMPT, 13);
+    testLogLevel(HistoryLogLevel.VERTEX, HistoryLogLevel.ALL, 13);
     testLogLevel(HistoryLogLevel.NONE, HistoryLogLevel.NONE, 0);
   }
 
@@ -153,7 +186,7 @@ public class TestHistoryEventHandler {
   }
 
   private HistoryEventHandler createHandler(HistoryLogLevel logLevel) {
-    Configuration conf = new Configuration(false);
+    Configuration conf = new Configuration(baseConfig);
     conf.setBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, false);
     conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
         InMemoryHistoryLoggingService.class.getName());
@@ -181,6 +214,7 @@ public class TestHistoryEventHandler {
 
     long time = System.currentTimeMillis();
     Configuration conf = new Configuration(inConf);
+
     historyEvents.add(new DAGHistoryEvent(null,
         new AMStartedEvent(attemptId, time, user)));
     historyEvents.add(new DAGHistoryEvent(dagId,
@@ -189,16 +223,33 @@ public class TestHistoryEventHandler {
     TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
     historyEvents.add(new DAGHistoryEvent(dagId,
         new VertexStartedEvent(vertexID, time, time)));
+    ContainerId containerId = ContainerId.newContainerId(attemptId, dagId.getId());
     TezTaskID tezTaskID = TezTaskID.getInstance(vertexID, 1);
     historyEvents.add(new DAGHistoryEvent(dagId,
         new TaskStartedEvent(tezTaskID, "test", time, time)));
+    historyEvents.add(
+        new DAGHistoryEvent(new ContainerLaunchedEvent(containerId, time, attemptId)));
     historyEvents.add(new DAGHistoryEvent(dagId,
         new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(tezTaskID, 1), "test", time,
-            ContainerId.newContainerId(attemptId, 1), NodeId.newInstance("localhost", 8765), null,
-            null, null)));
+            containerId, NodeId.newInstance("localhost", 8765), null, null, null)));
+    historyEvents.add(new DAGHistoryEvent(dagId,
+        new TaskAttemptFinishedEvent(TezTaskAttemptID.getInstance(tezTaskID, 1), "test", time,
+            time + 1, TaskAttemptState.KILLED, null,
+            TaskAttemptTerminationCause.EXTERNAL_PREEMPTION, "", null, null, null, time, null, time,
+            containerId, NodeId.newInstance("localhost", 8765), null, null, null)));
+    historyEvents.add(new DAGHistoryEvent(dagId,
+        new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(tezTaskID, 2), "test", time,
+            containerId, NodeId.newInstance("localhost", 8765), null, null, null)));
+    historyEvents.add(new DAGHistoryEvent(dagId,
+        new TaskAttemptFinishedEvent(TezTaskAttemptID.getInstance(tezTaskID, 2), "test", time + 2,
+            time + 3, TaskAttemptState.KILLED, null,
+            TaskAttemptTerminationCause.INTERRUPTED_BY_USER, "", null, null, null, time, null,
+            time + 2, containerId, NodeId.newInstance("localhost", 8765), null, null, null)));
     historyEvents.add(new DAGHistoryEvent(dagId,
         new DAGFinishedEvent(dagId, time, time, DAGState.SUCCEEDED, null, null, user, "test", null,
             attemptId, DAGPlan.getDefaultInstance())));
+    historyEvents.add(
+        new DAGHistoryEvent(new ContainerStoppedEvent(containerId, time + 4, 0, attemptId)));
     return historyEvents;
   }
 }


[16/50] [abbrv] tez git commit: TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount. (sseth)

Posted by zh...@apache.org.
TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small
amount. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: 2d8090e9cffa1f3643c93dd13507cea0a7a092a4
Parents: fc0897b
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 22 17:11:21 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Feb 22 17:11:21 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                       |  1 +
 .../library/common/sort/impl/PipelinedSorter.java |  9 +++++++--
 .../common/sort/impl/TestPipelinedSorter.java     | 18 +++++++++---------
 3 files changed, 17 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2d8090e9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5427e12..a1e23f4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.
   TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.
   TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes
   TEZ-3625. Dag.getVertex should obtain a readlock.

http://git-wip-us.apache.org/repos/asf/tez/blob/2d8090e9/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
index 4258fff..3d4bfbe 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -132,7 +132,8 @@ public class PipelinedSorter extends ExternalSorter {
        * When lazy-allocation is enabled, framework takes care of auto
        * allocating memory on need basis. Desirable block size is set to 256MB
        */
-      MIN_BLOCK_SIZE = 256 << 20; //256 MB
+      //256 MB - 64 bytes. See comment for the 32MB allocation.
+      MIN_BLOCK_SIZE = ((256 << 20) - 64);
     } else {
       int minBlockSize = conf.getInt(TezRuntimeConfiguration
               .TEZ_RUNTIME_PIPELINED_SORTER_MIN_BLOCK_SIZE_IN_MB,
@@ -267,7 +268,11 @@ public class PipelinedSorter extends ExternalSorter {
      */
     if (lazyAllocateMem) {
       if (buffers == null || buffers.isEmpty()) {
-        return 32 << 20; //32 MB
+        //32 MB - 64 bytes
+        // These buffers end up occupying 33554456 (32M + 24) bytes.
+        // On large JVMs (64G+), with G1Gc - the region size maxes out at
+        // 32M. Without the -64, this structure would end up using 2 regions.
+        return ((32 << 20) - 64);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/2d8090e9/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
index 80e7b14..d295640 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
@@ -601,13 +601,13 @@ public class TestPipelinedSorter {
         numOutputs, (128l << 20));
     assertTrue("Expected 1 sort buffers. current len=" + sorter.buffers.size(),
         sorter.buffers.size() == 1);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
     writeData(sorter, 100, 1024*1024, false); //100 1 MB KV. Will spill
 
     //Now it should have created 2 buffers, 32 & 96 MB buffers.
     assertTrue(sorter.buffers.size() == 2);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
-    assertTrue(sorter.buffers.get(1).capacity() == 96 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
+    assertTrue(sorter.buffers.get(1).capacity() == 96 * 1024 * 1024 + 64);
     closeSorter(sorter);
     verifyCounters(sorter, outputContext);
 
@@ -619,12 +619,12 @@ public class TestPipelinedSorter {
         .TEZ_RUNTIME_PIPELINED_SORTER_LAZY_ALLOCATE_MEMORY, true);
     sorter = new PipelinedSorter(this.outputContext, conf, numOutputs, (300l << 20));
     assertTrue(sorter.buffers.size() == 1);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
 
     writeData(sorter, 50, 1024*1024, false); //50 1 MB KV to allocate 2nd buf
     assertTrue(sorter.buffers.size() == 2);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
-    assertTrue(sorter.buffers.get(1).capacity() == 268 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
+    assertTrue(sorter.buffers.get(1).capacity() == 268 * 1024 * 1024 + 64);
 
     //48 MB. Do not pre-allocate.
     // Get 32 MB buffer first invariably and proceed with the rest.
@@ -635,13 +635,13 @@ public class TestPipelinedSorter {
         numOutputs, (48l << 20));
     assertTrue("Expected 1 sort buffers. current len=" + sorter.buffers.size(),
         sorter.buffers.size() == 1);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
     writeData(sorter, 20, 1024*1024, false); //100 1 MB KV. Will spill
 
     //Now it should have created 2 buffers, 32 & 96 MB buffers.
     assertTrue(sorter.buffers.size() == 2);
-    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024);
-    assertTrue(sorter.buffers.get(1).capacity() == 16 * 1024 * 1024);
+    assertTrue(sorter.buffers.get(0).capacity() == 32 * 1024 * 1024 - 64);
+    assertTrue(sorter.buffers.get(1).capacity() == 16 * 1024 * 1024 + 64);
     closeSorter(sorter);
   }
 


[12/50] [abbrv] tez git commit: TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes (jeagles)

Posted by zh...@apache.org.
TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes (jeagles)


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

Branch: refs/heads/TEZ-1190
Commit: 10ded7cbbf8d774f209de35461f14c9e526da888
Parents: dcae5e4
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Thu Feb 16 10:52:58 2017 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Thu Feb 16 10:52:58 2017 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../main/webapp/app/controllers/home/index.js   | 22 +++++++++++++++++++-
 tez-ui/src/main/webapp/app/router.js            |  5 ++++-
 3 files changed, 26 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/10ded7cb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c505964..f34252b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes
   TEZ-3625. Dag.getVertex should obtain a readlock.
   TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
   TEZ-3550. Provide access to sessionId/dagId via DagClient.

http://git-wip-us.apache.org/repos/asf/tez/blob/10ded7cb/tez-ui/src/main/webapp/app/controllers/home/index.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js
index bf9dcde..61e7a9f 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/index.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/index.js
@@ -24,13 +24,17 @@ import TableDefinition from 'em-table/utils/table-definition';
 
 export default TableController.extend({
 
-  queryParams: ["dagName", "dagID", "submitter", "status", "appID", "callerID"],
+  queryParams: ["dagName", "dagID", "submitter", "status", "appID", "callerID", "appid", "id", "user", "dag_name"],
   dagName: "",
   dagID: "",
   submitter: "",
   status: "",
   appID: "",
   callerID: "",
+  appid: "",
+  id: "",
+  user: "",
+  dag_name: "",
 
   // Because pageNo is a query param added by table controller, and in the current design
   // we don't want page to be a query param as only the first page will be loaded first.
@@ -52,6 +56,22 @@ export default TableController.extend({
       "appID", "callerID", "pageNum", "moreAvailable", "loadingMore", function () {
 
     var definition = this.get("_definition");
+    if (!this.get("appID")) {
+      this.set("appID", this.get("appid"));
+      this.set("appid", "");
+    }
+    if (!this.get("dagID")) {
+      this.set("dagID", this.get("id"));
+      this.set("id", "");
+    }
+    if (!this.get("submitter")) {
+      this.set("submitter", this.get("user"));
+      this.set("user", "");
+    }
+    if (!this.get("dagName")) {
+      this.set("dagName", this.get("dag_name"));
+      this.set("dag_name", "");
+    }
 
     definition.setProperties({
       dagName: this.get("dagName"),

http://git-wip-us.apache.org/repos/asf/tez/blob/10ded7cb/tez-ui/src/main/webapp/app/router.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/router.js b/tez-ui/src/main/webapp/app/router.js
index 6aa0b2f..7a8901b 100644
--- a/tez-ui/src/main/webapp/app/router.js
+++ b/tez-ui/src/main/webapp/app/router.js
@@ -55,7 +55,10 @@ Router.map(function() {
   });
 
   // Alias for backward compatibility with Tez UI V1
-  this.route('app', {path: '/tez-app/:app_id'}, function () {});
+  this.route('app', {path: '/tez-app/:app_id'}, function () {
+    this.route('dags');
+    this.route('configs');
+  });
   this.route('app', {path: '/app/:app_id'}, function () {
     this.route('dags');
     this.route('configs');


[17/50] [abbrv] tez git commit: TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling (sree)

Posted by zh...@apache.org.
TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling (sree)


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

Branch: refs/heads/TEZ-1190
Commit: 359cd75ca7933443cb5fca01fc3254d2ae8d4adf
Parents: 2d8090e
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Fri Feb 24 00:27:13 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Fri Feb 24 00:27:13 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../main/webapp/app/routes/multi-am-pollster.js |  7 +++-
 .../webapp/app/routes/single-am-pollster.js     |  7 +++-
 .../tests/unit/routes/multi-am-pollster-test.js | 44 ++++++++++++++++++++
 .../unit/routes/single-am-pollster-test.js      | 23 ++++++++--
 5 files changed, 74 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/359cd75c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a1e23f4..b855af0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -203,6 +203,7 @@ ALL CHANGES:
   TEZ-3619. Tez UI: Improve DAG Data download
   TEZ-3629. Tez UI: Enable the UI to display log links from LLAP
   TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly
+  TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/359cd75c/tez-ui/src/main/webapp/app/routes/multi-am-pollster.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/routes/multi-am-pollster.js b/tez-ui/src/main/webapp/app/routes/multi-am-pollster.js
index c3260a6..ca20cd5 100644
--- a/tez-ui/src/main/webapp/app/routes/multi-am-pollster.js
+++ b/tez-ui/src/main/webapp/app/routes/multi-am-pollster.js
@@ -21,8 +21,11 @@ import AmPollsterRoute from './am-pollster';
 
 export default AmPollsterRoute.extend({
 
-  canPoll: Ember.computed("polledRecords.0.app.isComplete", "loadedValue", function () {
-    var isComplete = this.get("polledRecords.0.app.isComplete");
+  canPoll: Ember.computed("polledRecords.0.dag.isComplete", "polledRecords.0.app.isComplete", "loadedValue", function () {
+    var isComplete = this.get("polledRecords.0.dag.isComplete");
+    if(isComplete === undefined) {
+      isComplete = this.get("polledRecords.0.app.isComplete");
+    }
     return isComplete === false && this._super();
   }),
 

http://git-wip-us.apache.org/repos/asf/tez/blob/359cd75c/tez-ui/src/main/webapp/app/routes/single-am-pollster.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/routes/single-am-pollster.js b/tez-ui/src/main/webapp/app/routes/single-am-pollster.js
index 4b72ee7..27bcced 100644
--- a/tez-ui/src/main/webapp/app/routes/single-am-pollster.js
+++ b/tez-ui/src/main/webapp/app/routes/single-am-pollster.js
@@ -21,8 +21,11 @@ import AmPollsterRoute from './am-pollster';
 
 export default AmPollsterRoute.extend({
 
-  canPoll: Ember.computed("polledRecords", "loadedValue.app.isComplete", function () {
-    var isComplete = this.get("loadedValue.app.isComplete");
+  canPoll: Ember.computed("polledRecords", "loadedValue.dag.isComplete", "loadedValue.app.isComplete", function () {
+    var isComplete = this.get("loadedValue.dag.isComplete");
+    if(isComplete === undefined) {
+      isComplete = this.get("loadedValue.app.isComplete");
+    }
     return isComplete === false && this._super();
   }),
 

http://git-wip-us.apache.org/repos/asf/tez/blob/359cd75c/tez-ui/src/main/webapp/tests/unit/routes/multi-am-pollster-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/routes/multi-am-pollster-test.js b/tez-ui/src/main/webapp/tests/unit/routes/multi-am-pollster-test.js
index eb3670b..2903d39 100644
--- a/tez-ui/src/main/webapp/tests/unit/routes/multi-am-pollster-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/routes/multi-am-pollster-test.js
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+import Ember from 'ember';
 import { moduleFor, test } from 'ember-qunit';
 
 moduleFor('route:multi-am-pollster', 'Unit | Route | multi am pollster', {
@@ -30,3 +31,46 @@ test('Basic creation test', function(assert) {
   assert.ok(route.canPoll);
   assert.ok(route.actions.setPollingRecords);
 });
+
+test('canPoll test', function(assert) {
+  let record = Ember.Object.create({
+      }),
+      route = this.subject({
+        polling: {
+          resetPoll: function () {}
+        },
+        _canPollObserver: function () {},
+        polledRecords: Ember.A([record]),
+        loadedValue: {}
+      });
+
+  assert.notOk(route.get("canPoll"));
+
+  record.setProperties({
+    app: Ember.Object.create({
+      isComplete: false
+    }),
+    dag: undefined
+  });
+  assert.ok(route.get("canPoll"), true, "Test 1");
+
+  record.set("app.isComplete", true);
+  assert.notOk(route.get("canPoll"), "Test 2");
+
+  record.set("app.isComplete", undefined);
+  assert.notOk(route.get("canPoll"), "Test 3");
+
+  record.set("dag", Ember.Object.create({
+    isComplete: false
+  }));
+  assert.ok(route.get("canPoll"), "Test 4");
+
+  record.set("dag.isComplete", true);
+  assert.notOk(route.get("canPoll"), "Test 5");
+
+  record.set("dag", undefined);
+  assert.notOk(route.get("canPoll"), "Test 6");
+
+  record.set("app.isComplete", false);
+  assert.ok(route.get("canPoll"), "Test 7");
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/359cd75c/tez-ui/src/main/webapp/tests/unit/routes/single-am-pollster-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/routes/single-am-pollster-test.js b/tez-ui/src/main/webapp/tests/unit/routes/single-am-pollster-test.js
index f9b3385..069c9fc 100644
--- a/tez-ui/src/main/webapp/tests/unit/routes/single-am-pollster-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/routes/single-am-pollster-test.js
@@ -48,16 +48,31 @@ test('canPoll test', function(assert) {
     loadedValue: {
       app: {
         isComplete: false
-      }
+      },
+      dag: undefined
     }
   });
-  assert.ok(route.get("canPoll"), true);
+  assert.ok(route.get("canPoll"), true, "Test 1");
 
   route.set("loadedValue.app.isComplete", true);
-  assert.notOk(route.get("canPoll"));
+  assert.notOk(route.get("canPoll"), "Test 2");
 
   route.set("loadedValue.app.isComplete", undefined);
-  assert.notOk(route.get("canPoll"));
+  assert.notOk(route.get("canPoll"), "Test 3");
+
+  route.set("loadedValue.dag", Ember.Object.create({
+    isComplete: false
+  }));
+  assert.ok(route.get("canPoll"), "Test 4");
+
+  route.set("loadedValue.dag.isComplete", true);
+  assert.notOk(route.get("canPoll"), "Test 5");
+
+  route.set("loadedValue.dag", undefined);
+  assert.notOk(route.get("canPoll"), "Test 6");
+
+  route.set("loadedValue.app.isComplete", false);
+  assert.ok(route.get("canPoll"), "Test 7");
 });
 
 test('_loadedValueObserver test', function(assert) {


[37/50] [abbrv] tez git commit: TEZ-3651 part2. Update website, documentation etc for Tez 0.8.5 release.

Posted by zh...@apache.org.
TEZ-3651 part2. Update website, documentation etc for Tez 0.8.5 release.


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

Branch: refs/heads/TEZ-1190
Commit: 908142047d198569088d7166a7eda128c8f95c54
Parents: b2be0d5
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Mar 13 19:22:19 2017 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Mar 13 19:22:19 2017 -0700

----------------------------------------------------------------------
 Tez_DOAP.rdf                                    |  7 +++++
 .../site/markdown/releases/apache-tez-0-8-5.md  | 30 ++++++++++++++++++++
 docs/src/site/markdown/releases/index.md        |  1 +
 docs/src/site/site.xml                          |  2 +-
 4 files changed, 39 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/90814204/Tez_DOAP.rdf
----------------------------------------------------------------------
diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf
index 01ac0d8..2755f97 100644
--- a/Tez_DOAP.rdf
+++ b/Tez_DOAP.rdf
@@ -36,6 +36,13 @@
     <category rdf:resource="http://projects.apache.org/category/big-data" />
     <release>
       <Version>
+        <name>Version 0.8.5</name>
+        <created>2017-03-13</created>
+        <revision>0.8.5</revision>
+      </Version>
+    </release>
+    <release>
+      <Version>
         <name>Version 0.8.4</name>
         <created>2016-07-08</created>
         <revision>0.8.4</revision>

http://git-wip-us.apache.org/repos/asf/tez/blob/90814204/docs/src/site/markdown/releases/apache-tez-0-8-5.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/releases/apache-tez-0-8-5.md b/docs/src/site/markdown/releases/apache-tez-0-8-5.md
new file mode 100644
index 0000000..9b73cec
--- /dev/null
+++ b/docs/src/site/markdown/releases/apache-tez-0-8-5.md
@@ -0,0 +1,30 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<head><title>Apache Tez&trade; 0.8.5</title></head>
+
+Apache Tez&trade; 0.8.5
+----------------------
+
+- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.8.5/)
+- [Release Notes](0.8.5/release-notes.txt)
+- Documentation
+    - [API Javadocs](0.8.5/tez-api-javadocs/index.html) : Documentation for the Tez APIs
+    - [Runtime Library Javadocs](0.8.5/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs 
+    - [Tez Mapreduce Javadocs](0.8.5/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs 
+    - [Tez Configuration](0.8.5/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml.
+    - [Tez Runtime Configuration](0.8.5/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters.

http://git-wip-us.apache.org/repos/asf/tez/blob/90814204/docs/src/site/markdown/releases/index.md
----------------------------------------------------------------------
diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md
index 1554a41..259a5a8 100644
--- a/docs/src/site/markdown/releases/index.md
+++ b/docs/src/site/markdown/releases/index.md
@@ -19,6 +19,7 @@
 
 Releases
 ------------
+-   [Apache Tez&trade; 0.8.5](./apache-tez-0-8-5.html) (Mar 13, 2017)
 -   [Apache Tez&trade; 0.8.4](./apache-tez-0-8-4.html) (Jul 08, 2016)
 -   [Apache Tez&trade; 0.8.3](./apache-tez-0-8-3.html) (Apr 15, 2016)
 -   [Apache Tez&trade; 0.8.2](./apache-tez-0-8-2.html) (Jan 19, 2016)

http://git-wip-us.apache.org/repos/asf/tez/blob/90814204/docs/src/site/site.xml
----------------------------------------------------------------------
diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml
index 3317bac..91aa8de 100644
--- a/docs/src/site/site.xml
+++ b/docs/src/site/site.xml
@@ -124,7 +124,7 @@
       <item name="0.5.4" href="./releases/apache-tez-0-5-4.html"/>
       <item name="0.6.2" href="./releases/apache-tez-0-6-2.html"/>
       <item name="0.7.1" href="./releases/apache-tez-0-7-1.html"/>
-      <item name="0.8.4" href="./releases/apache-tez-0-8-4.html"/>
+      <item name="0.8.5" href="./releases/apache-tez-0-8-5.html"/>
       <item name="All Releases" href="./releases/index.html"/>
     </menu>
 


[47/50] [abbrv] tez git commit: TEZ-3642. Remove ShuffleClientMetrics (zhiyuany)

Posted by zh...@apache.org.
TEZ-3642. Remove ShuffleClientMetrics (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: f7f60385cf5d8d66a1ee02e64d1e671bf1ad8771
Parents: 2bdf58a
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Mon Mar 27 18:53:45 2017 -0700
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Mon Mar 27 18:53:45 2017 -0700

----------------------------------------------------------------------
 .../orderedgrouped/FetcherOrderedGrouped.java   | 10 ---
 .../orderedgrouped/ShuffleClientMetrics.java    | 92 --------------------
 .../orderedgrouped/ShuffleScheduler.java        |  6 +-
 .../shuffle/orderedgrouped/TestFetcher.java     | 27 ++----
 4 files changed, 10 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/f7f60385/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
index 58ca1e2..5cad6fc 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
@@ -69,7 +69,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
   private final TezCounter wrongReduceErrs;
   private final FetchedInputAllocatorOrderedGrouped allocator;
   private final ShuffleScheduler scheduler;
-  private final ShuffleClientMetrics metrics;
   private final ExceptionReporter exceptionReporter;
   private final int id;
   private final String logIdentifier;
@@ -107,7 +106,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
   public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams,
                                ShuffleScheduler scheduler,
                                FetchedInputAllocatorOrderedGrouped allocator,
-                               ShuffleClientMetrics metrics,
                                ExceptionReporter exceptionReporter, JobTokenSecretManager jobTokenSecretMgr,
                                boolean ifileReadAhead, int ifileReadAheadLength,
                                CompressionCodec codec,
@@ -130,7 +128,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
                                boolean verifyDiskChecksum) {
     this.scheduler = scheduler;
     this.allocator = allocator;
-    this.metrics = metrics;
     this.exceptionReporter = exceptionReporter;
     this.mapHost = mapHost;
     this.currentPartition = this.mapHost.getPartitionId();
@@ -169,8 +166,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
   @VisibleForTesting
   protected void fetchNext() throws InterruptedException, IOException {
     try {
-      metrics.threadBusy();
-
       if (localDiskFetchEnabled && mapHost.getHost().equals(localShuffleHost) && mapHost.getPort() == localShufflePort) {
         setupLocalDiskFetch(mapHost);
       } else {
@@ -180,7 +175,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
     } finally {
       cleanupCurrentConnection(false);
       scheduler.freeHost(mapHost);
-      metrics.threadFree();
     }
   }
 
@@ -524,7 +518,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
                               endTime - startTime, mapOutput, false);
       // Note successful shuffle
       remaining.remove(srcAttemptId.toString());
-      metrics.successFetch();
       return null;
     } catch (IOException ioe) {
       if (stopped) {
@@ -562,7 +555,6 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
 
       // Inform the shuffle-scheduler
       mapOutput.abort();
-      metrics.failedFetch();
       return new InputAttemptIdentifier[] {srcAttemptId};
     }
   }
@@ -685,13 +677,11 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
           scheduler.copySucceeded(srcAttemptId, host, indexRecord.getPartLength(),
               indexRecord.getRawLength(), (endTime - startTime), mapOutput, true);
           iter.remove();
-          metrics.successFetch();
         } catch (IOException e) {
           if (mapOutput != null) {
             mapOutput.abort();
           }
           if (!stopped) {
-            metrics.failedFetch();
             ioErrs.increment(1);
             scheduler.copyFailed(srcAttemptId, host, true, false, true);
             LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " +

http://git-wip-us.apache.org/repos/asf/tez/blob/f7f60385/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleClientMetrics.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleClientMetrics.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleClientMetrics.java
deleted file mode 100644
index f297dad..0000000
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleClientMetrics.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.runtime.library.common.shuffle.orderedgrouped;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.tez.common.TezRuntimeFrameworkConfigs;
-import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
-import org.apache.tez.runtime.library.common.Constants;
-import org.apache.tez.runtime.library.common.TezRuntimeUtils;
-
-class ShuffleClientMetrics implements Updater {
-
-  private MetricsRecord shuffleMetrics = null;
-  private int numFailedFetches = 0;
-  private int numSuccessFetches = 0;
-  private long numBytes = 0;
-  private int numThreadsBusy = 0;
-  private final int numCopiers;
-  
-  ShuffleClientMetrics(String dagName, String vertexName, int taskIndex, Configuration conf, 
-      String user) {
-    this.numCopiers = 
-        conf.getInt(
-            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
-            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES_DEFAULT);
-
-    MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
-    this.shuffleMetrics = 
-      MetricsUtil.createRecord(metricsContext, "shuffleInput");
-    this.shuffleMetrics.setTag("user", user);
-    this.shuffleMetrics.setTag("dagName", dagName);
-    this.shuffleMetrics.setTag("taskId", TezRuntimeUtils.getTaskIdentifier(vertexName, taskIndex));
-    this.shuffleMetrics.setTag("sessionId", 
-        conf.get(
-            TezRuntimeFrameworkConfigs.TEZ_RUNTIME_METRICS_SESSION_ID,
-            TezRuntimeFrameworkConfigs.TEZ_RUNTIME_METRICS_SESSION_ID_DEFAULT));
-    metricsContext.registerUpdater(this);
-  }
-  public synchronized void inputBytes(long numBytes) {
-    this.numBytes += numBytes;
-  }
-  public synchronized void failedFetch() {
-    ++numFailedFetches;
-  }
-  public synchronized void successFetch() {
-    ++numSuccessFetches;
-  }
-  public synchronized void threadBusy() {
-    ++numThreadsBusy;
-  }
-  public synchronized void threadFree() {
-    --numThreadsBusy;
-  }
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
-      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
-                                numFailedFetches);
-      shuffleMetrics.incrMetric("shuffle_success_fetches", 
-                                numSuccessFetches);
-      if (numCopiers != 0) {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
-            100*((float)numThreadsBusy/numCopiers));
-      } else {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
-      }
-      numBytes = 0;
-      numSuccessFetches = 0;
-      numFailedFetches = 0;
-    }
-    shuffleMetrics.update();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/f7f60385/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
index cce486c..953c73e 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
@@ -204,7 +204,6 @@ class ShuffleScheduler {
 
   private final HttpConnectionParams httpConnectionParams;
   private final FetchedInputAllocatorOrderedGrouped allocator;
-  private final ShuffleClientMetrics shuffleMetrics;
   private final ExceptionReporter exceptionReporter;
   private final MergeManager mergeManager;
   private final JobTokenSecretManager jobTokenSecretManager;
@@ -370,9 +369,6 @@ class ShuffleScheduler {
         TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL_DEFAULT);
     this.asyncHttp = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, false);
     this.httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
-    this.shuffleMetrics = new ShuffleClientMetrics(inputContext.getDAGName(),
-        inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
-        this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
     SecretKey jobTokenSecret = ShuffleUtils
         .getJobTokenSecretFromTokenBytes(inputContext
             .getServiceConsumerMetaData(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
@@ -1391,7 +1387,7 @@ class ShuffleScheduler {
   @VisibleForTesting
   FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) {
     return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator,
-        shuffleMetrics, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength,
+        exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength,
         codec, conf, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost,
         ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter,
         connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle,

http://git-wip-us.apache.org/repos/asf/tez/blob/f7f60385/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
index 310f1b2..a9b57a9 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
@@ -110,7 +110,6 @@ public class TestFetcher {
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
 
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
 
     InputContext inputContext = mock(InputContext.class);
@@ -124,7 +123,7 @@ public class TestFetcher {
     doReturn(mapsForHost).when(scheduler).getMapsForHost(mapHost);
 
     FetcherOrderedGrouped fetcher =
-        new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+        new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
             null, conf, false, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
             wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
@@ -142,7 +141,6 @@ public class TestFetcher {
     Configuration conf = new TezConfiguration();
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
 
     InputContext inputContext = mock(InputContext.class);
@@ -153,7 +151,7 @@ public class TestFetcher {
     final boolean DISABLE_LOCAL_FETCH = false;
     MapHost mapHost = new MapHost(HOST, PORT, 0);
     FetcherOrderedGrouped fetcher =
-        new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+        new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
             wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
@@ -171,7 +169,7 @@ public class TestFetcher {
     // if hostname does not match use http
     mapHost = new MapHost(HOST + "_OTHER", PORT, 0);
     fetcher =
-        new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+        new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
             wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
@@ -187,7 +185,7 @@ public class TestFetcher {
     // if port does not match use http
     mapHost = new MapHost(HOST, PORT + 1, 0);
     fetcher =
-        new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+        new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
             wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
@@ -202,7 +200,7 @@ public class TestFetcher {
 
     //if local fetch is not enabled
     mapHost = new MapHost(HOST, PORT, 0);
-    fetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+    fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
         null, conf, DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
         wrongLengthErrsCounter, badIdErrsCounter,
         wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
@@ -221,14 +219,13 @@ public class TestFetcher {
     Configuration conf = new TezConfiguration();
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
     InputContext inputContext = mock(InputContext.class);
     when(inputContext.getCounters()).thenReturn(new TezCounters());
     when(inputContext.getSourceVertexName()).thenReturn("");
 
     MapHost host = new MapHost(HOST, PORT, 1);
-    FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+    FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
         null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter,
         wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
         false, false, true);
@@ -300,9 +297,6 @@ public class TestFetcher {
     verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX), host, true, false, true);
     verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX), host, true, false, true);
 
-    verify(metrics, times(3)).successFetch();
-    verify(metrics, times(2)).failedFetch();
-
     verify(spyFetcher).putBackRemainingMapOutputs(host);
     verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX));
     verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX));
@@ -364,7 +358,6 @@ public class TestFetcher {
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
 
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
     InputContext inputContext = mock(InputContext.class);
     when(inputContext.getCounters()).thenReturn(new TezCounters());
@@ -373,7 +366,7 @@ public class TestFetcher {
 
     HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
     final MapHost host = new MapHost(HOST, PORT, 1);
-    FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+    FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
         null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter,
         wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,
         false, false, true);
@@ -449,7 +442,6 @@ public class TestFetcher {
 
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
 
     TezCounters counters = new TezCounters();
@@ -463,7 +455,7 @@ public class TestFetcher {
     HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
     final MapHost host = new MapHost(HOST, PORT, 1);
     FetcherOrderedGrouped mockFetcher =
-        new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, metrics, shuffle, jobMgr,
+        new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, shuffle, jobMgr,
             false, 0,
             null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
@@ -528,11 +520,10 @@ public class TestFetcher {
     Configuration conf = new TezConfiguration();
     ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
     MergeManager merger = mock(MergeManager.class);
-    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
     Shuffle shuffle = mock(Shuffle.class);
     MapHost mapHost = new MapHost(HOST, PORT, 0);
     FetcherOrderedGrouped fetcher =
-        new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
+        new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0,
             null, conf, false, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
             wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID,


[50/50] [abbrv] tez git commit: TEZ-3511. Allow user to create named edge

Posted by zh...@apache.org.
TEZ-3511. Allow user to create named edge


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

Branch: refs/heads/TEZ-1190
Commit: d1b08e37e3e5bc16c6eea130dfefb52aca84b3a5
Parents: 906c2a8
Author: Zhiyuan Yang <sj...@gmail.com>
Authored: Mon Dec 5 15:34:55 2016 -0800
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Tue Mar 28 11:20:23 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/tez/dag/api/DAG.java   | 114 ++++++---
 .../main/java/org/apache/tez/dag/api/Edge.java  |  81 +++---
 .../org/apache/tez/dag/api/EdgeProperty.java    |   4 +-
 .../java/org/apache/tez/dag/api/TestDAG.java    |  26 ++
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 249 +++++++++++++++++++
 .../java/org/apache/tez/dag/api/TestEdge.java   |  75 ++++++
 6 files changed, 480 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index c136811..9151d48 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -273,6 +273,13 @@ public class DAG {
 
   /**
    * Add an {@link Edge} connecting vertices in the DAG
+   *
+   * All edges within a DAG must be either named (created via
+   * {@link org.apache.tez.dag.api.Edge#create(Vertex, Vertex, EdgeProperty, String)}) or unnamed
+   * (created via {@link org.apache.tez.dag.api.Edge#create(Vertex, Vertex, EdgeProperty)}).
+   * If edges are named, all inbound edges to a vertex should have unique names. Likewise for outbound edges.
+   * A vertex can have an inbound edge that uses the same name as that used by an outbound edge.
+   *
    * @param edge The edge to be added
    * @return {@link DAG}
    */
@@ -571,8 +578,6 @@ public class DAG {
     // check for valid vertices, duplicate vertex names,
     // and prepare for cycle detection
     Map<String, AnnotatedVertex> vertexMap = new HashMap<String, AnnotatedVertex>();
-    Map<Vertex, Set<String>> inboundVertexMap = new HashMap<Vertex, Set<String>>();
-    Map<Vertex, Set<String>> outboundVertexMap = new HashMap<Vertex, Set<String>>();
     for (Vertex v : vertices.values()) {
       if (vertexMap.containsKey(v.getName())) {
         throw new IllegalStateException("DAG contains multiple vertices"
@@ -581,33 +586,45 @@ public class DAG {
       vertexMap.put(v.getName(), new AnnotatedVertex(v));
     }
 
-    Map<Vertex, List<Edge>> edgeMap = new HashMap<Vertex, List<Edge>>();
+    // named edge cannot be mixed with unnamed edge or group edge
+    Edge namedEdge = null, unnamedEdge = null;
+    for (Edge e : edges) {
+      if (e.getName() == null) {
+        unnamedEdge = e;
+      } else {
+        namedEdge = e;
+      }
+
+      if (namedEdge != null && !groupInputEdges.isEmpty()) {
+        throw new IllegalStateException("DAG shouldn't contains both named edge " + namedEdge
+          + " and group edge " + groupInputEdges.iterator().next());
+      }
+      if (namedEdge != null && unnamedEdge != null) {
+        throw new IllegalStateException("DAG shouldn't contains both named edge " + namedEdge
+          + " and unnamed edge " + unnamedEdge);
+      }
+    }
+
+    Map<Vertex, List<Edge>> inEdgeMap = new HashMap<>();
+    Map<Vertex, List<Edge>> outEdgeMap = new HashMap<>();
     for (Edge e : edges) {
       // Construct structure for cycle detection
       Vertex inputVertex = e.getInputVertex();
       Vertex outputVertex = e.getOutputVertex();      
-      List<Edge> edgeList = edgeMap.get(inputVertex);
-      if (edgeList == null) {
-        edgeList = new ArrayList<Edge>();
-        edgeMap.put(inputVertex, edgeList);
-      }
-      edgeList.add(e);
-      
-      // Construct map for Input name verification
-      Set<String> inboundSet = inboundVertexMap.get(outputVertex);
-      if (inboundSet == null) {
-        inboundSet = new HashSet<String>();
-        inboundVertexMap.put(outputVertex, inboundSet);
+
+      List<Edge> outEdgeList = outEdgeMap.get(inputVertex);
+      if (outEdgeList == null) {
+        outEdgeList = new ArrayList<Edge>();
+        outEdgeMap.put(inputVertex, outEdgeList);
       }
-      inboundSet.add(inputVertex.getName());
-      
-      // Construct map for Output name verification
-      Set<String> outboundSet = outboundVertexMap.get(inputVertex);
-      if (outboundSet == null) {
-        outboundSet = new HashSet<String>();
-        outboundVertexMap.put(inputVertex, outboundSet);
+      outEdgeList.add(e);
+
+      List<Edge> inEdgeList = inEdgeMap.get(outputVertex);
+      if (inEdgeList == null) {
+        inEdgeList = new ArrayList<Edge>();
+        inEdgeMap.put(outputVertex, inEdgeList);
       }
-      outboundSet.add(outputVertex.getName());
+      inEdgeList.add(e);
     }
 
     // check input and output names don't collide with vertex names
@@ -633,29 +650,54 @@ public class DAG {
     }
 
     // Check for valid InputNames
-    for (Entry<Vertex, Set<String>> entry : inboundVertexMap.entrySet()) {
+    for (Entry<Vertex, List<Edge>> entry : inEdgeMap.entrySet()) {
       Vertex vertex = entry.getKey();
+      Set<String> inputs = new HashSet<>();
+
+      for (Edge edge : entry.getValue()) {
+        String name = edge.getName();
+        if (name == null) {
+          name = edge.getInputVertex().getName();
+        }
+        if (inputs.contains(name)) {
+          throw new IllegalStateException("Vertex: " + vertex.getName() + " contains multiple " +
+            "incoming edges with name " + name);
+        }
+        inputs.add(name);
+      }
       for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> 
            input : vertex.getInputs()) {
-        if (entry.getValue().contains(input.getName())) {
+        if (inputs.contains(input.getName())) {
           throw new IllegalStateException("Vertex: "
               + vertex.getName()
-              + " contains an incoming vertex and Input with the same name: "
-              + input.getName());
+              + " contains an incoming " + (namedEdge != null ? "edge" : "vertex")
+              + " and Input with the same name: " + input.getName());
         }
       }
     }
 
-    // Check for valid OutputNames
-    for (Entry<Vertex, Set<String>> entry : outboundVertexMap.entrySet()) {
+    for (Entry<Vertex, List<Edge>> entry : outEdgeMap.entrySet()) {
       Vertex vertex = entry.getKey();
-      for (RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor> 
-            output : vertex.getOutputs()) {
-        if (entry.getValue().contains(output.getName())) {
+      Set<String> outputs = new HashSet<>();
+
+      for (Edge edge : entry.getValue()) {
+        String name = edge.getName();
+        if (name == null) {
+          name = edge.getOutputVertex().getName();
+        }
+        if (outputs.contains(name)) {
+          throw new IllegalStateException("Vertex: " + vertex.getName() + " contains multiple " +
+            "outgoing edges with name " + name);
+        }
+        outputs.add(name);
+      }
+      for (RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>
+        output : vertex.getOutputs()) {
+        if (outputs.contains(output.getName())) {
           throw new IllegalStateException("Vertex: "
-              + vertex.getName()
-              + " contains an outgoing vertex and Output with the same name: "
-              + output.getName());
+            + vertex.getName()
+            + " contains an outgoing " + (namedEdge != null ? "edge" : "vertex")
+            + " and Output with the same name: " + output.getName());
         }
       }
     }
@@ -666,7 +708,7 @@ public class DAG {
     // When additional inputs are supported, this can be chceked easily (and early)
     // within the addInput / addOutput call itself.
 
-    Deque<String> topologicalVertexStack = detectCycles(edgeMap, vertexMap);
+    Deque<String> topologicalVertexStack = detectCycles(outEdgeMap, vertexMap);
 
     checkAndInferOneToOneParallelism();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
index db509f7..794d88d 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
  * Edge defines the connection between a producer and consumer vertex in the DAG.
  * @link {@link EdgeProperty} defines the relationship between them. The producer
  * vertex provides input to the edge and the consumer vertex reads output from the 
- * edge.
+ * edge. Edge could be either named or not.
  * 
  */
 @Public
@@ -33,13 +33,16 @@ public class Edge {
   private final Vertex inputVertex;
   private final Vertex outputVertex;
   private final EdgeProperty edgeProperty;
+  private final String name;
 
   private Edge(Vertex inputVertex,
                Vertex outputVertex,
-               EdgeProperty edgeProperty) {
+               EdgeProperty edgeProperty,
+               String name) {
     this.inputVertex = inputVertex;
     this.outputVertex = outputVertex;
     this.edgeProperty = edgeProperty;
+    this.name = name;
   }
 
 
@@ -57,7 +60,25 @@ public class Edge {
   public static Edge create(Vertex inputVertex,
                             Vertex outputVertex,
                             EdgeProperty edgeProperty) {
-    return new Edge(inputVertex, outputVertex, edgeProperty);
+    return new Edge(inputVertex, outputVertex, edgeProperty, null);
+  }
+
+  /**
+   * Creates an edge with specified name between the specified vertices.
+   *
+   * InputVertex(EdgeInput) ----- Edge ----- OutputVertex(EdgeOutput)]
+   *
+   * @param inputVertex the vertex which generates data to the edge.
+   * @param outputVertex the vertex which consumes data from the edge
+   * @param edgeProperty {@link org.apache.tez.dag.api.EdgeProperty} associated with this edge
+   * @param name name of edge
+   * @return the {@link org.apache.tez.dag.api.Edge}
+   */
+  public static Edge create(Vertex inputVertex,
+                            Vertex outputVertex,
+                            EdgeProperty edgeProperty,
+                            String name) {
+    return new Edge(inputVertex, outputVertex, edgeProperty, name);
   }
 
   /**
@@ -83,6 +104,14 @@ public class Edge {
   public EdgeProperty getEdgeProperty() {
     return edgeProperty;
   }
+
+  /**
+   * The name of this edge (or null if edge has no name)
+   * @return edge name or null
+   */
+  public String getName() {
+    return name;
+  }
   
   /*
    * Used to identify the edge in the configuration
@@ -95,39 +124,29 @@ public class Edge {
  
   @Override
   public String toString() {
-    return inputVertex + " -> " + outputVertex + " (" + edgeProperty + ")";
+    return "{" + name + " : " + inputVertex + " -> " + outputVertex + " " + edgeProperty + "}";
   }
 
   @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result
-        + ((inputVertex == null) ? 0 : inputVertex.hashCode());
-    result = prime * result
-        + ((outputVertex == null) ? 0 : outputVertex.hashCode());
-    return result;
-  }
+  public boolean equals(Object other) {
+    if (this == other) return true;
+    if (other == null || getClass() != other.getClass()) return false;
 
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    Edge other = (Edge) obj;
-    if (inputVertex == null) {
-      if (other.inputVertex != null)
-        return false;
-    } else if (!inputVertex.equals(other.inputVertex))
+    Edge edge = (Edge) other;
+
+    if (inputVertex != null ? !inputVertex.equals(edge.inputVertex) : edge.inputVertex != null)
       return false;
-    if (outputVertex == null) {
-      if (other.outputVertex != null)
-        return false;
-    } else if (!outputVertex.equals(other.outputVertex))
+    if (outputVertex != null ? !outputVertex.equals(edge.outputVertex) : edge.outputVertex != null)
       return false;
-    return true;
+    return name != null ? name.equals(edge.name) : edge.name == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = inputVertex != null ? inputVertex.hashCode() : 0;
+    result = 31 * result + (outputVertex != null ? outputVertex.hashCode() : 0);
+    result = 31 * result + (name != null ? name.hashCode() : 0);
+    return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
index 07fb2c1..3723433 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
@@ -243,10 +243,10 @@ public class EdgeProperty {
   
   @Override
   public String toString() {
-    return "{ " + dataMovementType + " : " + inputDescriptor.getClassName()
+    return "(" + dataMovementType + " : " + inputDescriptor.getClassName()
         + " >> " + dataSourceType + " >> " + outputDescriptor.getClassName()
         + " >> " + (edgeManagerDescriptor == null ? "NullEdgeManager" : edgeManagerDescriptor.getClassName())
-        + " }";
+        + ")";
   }
   
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
index 05c4e30..691b4c1 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAG.java
@@ -88,6 +88,32 @@ public class TestDAG {
   }
 
   @Test(timeout = 5000)
+  public void testAddDuplicatedNamedEdge() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+    Edge edge2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+
+    DAG dag = DAG.create("testDAG").addVertex(v1).addVertex(v2).addEdge(edge1);
+
+    try {
+      dag.addEdge(edge2);
+      Assert.fail("should fail it due to duplicate named edges");
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains(edge1 + " already defined"));
+    }
+  }
+
+  @Test(timeout = 5000)
   public void testDuplicatedVertexGroup() {
     Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
         dummyTaskCount, dummyTaskResource);

http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
index 5706542..720820d 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -1191,4 +1191,253 @@ public class TestDAGVerify {
       Assert.assertTrue(e.getMessage().contains("There is conflicting local resource"));
     }
   }
+
+  @Test(timeout = 5000)
+  public void testNamedEdge() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+
+    DAG dag = DAG.create("testDAG").addVertex(v1).addVertex(v2).addEdge(edge1);
+    dag.verify();
+
+    Edge edge2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge2");
+
+    dag.addEdge(edge2).verify();
+  }
+
+  @Test(timeout = 5000)
+  public void testNamedEdgeMixedWithUnnamedEdge() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+    Edge edge2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")));
+
+    DAG dag = DAG.create("testDAG").addVertex(v1).addVertex(v2).addEdge(edge1).addEdge(edge2);
+
+    try {
+      dag.verify();
+      Assert.fail("should fail it because DAG has both named and unnamed edge");
+    } catch (Exception e) {
+      Assert.assertTrue(
+        e.getMessage().contains(
+          "DAG shouldn't contains both named edge " + edge1 + " and unnamed edge " + edge2));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testNamedEdgeWithGroupEdge() {
+    Vertex v1 = Vertex.create("v1",
+      ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2",
+      ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v3 = Vertex.create("v3",
+      ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    DAG dag = DAG.create("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    String groupName1 = "uv12";
+    VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
+
+    GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
+      EdgeProperty.create(DataMovementType.SCATTER_GATHER,
+        DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
+        OutputDescriptor.create("dummy output class"),
+        InputDescriptor.create("dummy input class")),
+      InputDescriptor.create("dummy input class"));
+
+    Edge e2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.CONCURRENT, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "e2");
+
+    dag.addEdge(e1).addEdge(e2);
+    try {
+      dag.verify();
+      Assert.fail("should fail it because DAG has both named edge and group edge");
+    } catch (Exception e) {
+      Assert.assertTrue(
+        e.getMessage().contains("DAG shouldn't contains both named edge " + e2 + " and group edge "
+          + e1));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testInNamedEdgeCollide() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v3 = Vertex.create("v3", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    Edge edge1 = Edge.create(v1, v3, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+    Edge edge2 = Edge.create(v2, v3, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+
+    DAG dag =
+      DAG.create("testDAG").addVertex(v1).addVertex(v2).addVertex(v3).addEdge(edge1).addEdge(edge2);
+
+    try {
+      dag.verify();
+      Assert.fail("should fail it because v3 gets multiple incoming edges with same name");
+    } catch (Exception e) {
+      Assert.assertTrue(
+        e.getMessage().contains("v3 contains multiple incoming edges with name Edge1"));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testOutNamedEdgeCollide() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v3 = Vertex.create("v3", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    Edge edge1 = Edge.create(v1, v3, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+    Edge edge2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+
+    DAG dag =
+      DAG.create("testDAG").addVertex(v1).addVertex(v2).addVertex(v3).addEdge(edge1).addEdge(edge2);
+
+    try {
+      dag.verify();
+      Assert.fail("should fail it because v3 gets multiple outgoing edges with same name");
+    } catch (Exception e) {
+      Assert.assertTrue(
+        e.getMessage().contains("v1 contains multiple outgoing edges with name Edge1"));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testInEdgeOutEdgeWithSameName() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v3 = Vertex.create("v3", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+    Edge edge2 = Edge.create(v2, v3, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "Edge1");
+
+    DAG.create("testDAG")
+      .addVertex(v1).addVertex(v2).addVertex(v3)
+      .addEdge(edge1).addEdge(edge2)
+      .verify();
+  }
+
+  @Test(timeout = 5000)
+  public void testNamedEdgeCollideWithRootInput() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    v2.addDataSource("input",
+      DataSourceDescriptor.create(InputDescriptor.create("input"), null, null));
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "input");
+
+    DAG dag = DAG.create("testDag").addVertex(v1).addVertex(v2).addEdge(edge1);
+
+    try {
+      dag.verify();
+      Assert.fail("should fail it because v2 get incoming edge and input with same name");
+    } catch (Exception e) {
+      Assert.assertTrue(
+        e.getMessage().contains("v2 contains an incoming edge and Input with the same name: input"));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testNamedEdgeCollideWithLeafOutput() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    v1.addDataSink("output",
+      DataSinkDescriptor.create(OutputDescriptor.create("output"), null, null));
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "output");
+
+    DAG dag = DAG.create("testDag").addVertex(v1).addVertex(v2).addEdge(edge1);
+
+    try {
+      dag.verify();
+      Assert.fail("should fail it because v2 get outgoing edge and output with same name");
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains(
+        "v1 contains an outgoing edge and Output with the same name: output"));
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testNamedEdgeUsingVertexName() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+    Vertex v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"),
+      dummyTaskCount, dummyTaskResource);
+
+    Edge edge1 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "v1");
+    Edge edge2 = Edge.create(v1, v2, EdgeProperty.create(
+      DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+      SchedulingType.SEQUENTIAL, OutputDescriptor.create("output"),
+      InputDescriptor.create("input")), "v2");
+
+    DAG dag = DAG.create("testDag").addVertex(v1).addVertex(v2).addEdge(edge1).addEdge(edge2);
+    dag.verify();
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/d1b08e37/tez-api/src/test/java/org/apache/tez/dag/api/TestEdge.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestEdge.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestEdge.java
new file mode 100644
index 0000000..61f4fbc
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestEdge.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.dag.api;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestEdge {
+  Vertex v1, v2;
+  EdgeProperty edgeProperty;
+  Set<Edge> set;
+
+  @Before
+  public void setup() {
+    v1 = Vertex.create("v1", ProcessorDescriptor.create("Processor"));
+    v2 = Vertex.create("v2", ProcessorDescriptor.create("Processor"));
+    edgeProperty = EdgeProperty.create(EdgeProperty.DataMovementType.SCATTER_GATHER,
+      EdgeProperty.DataSourceType.PERSISTED, EdgeProperty.SchedulingType.CONCURRENT,
+      OutputDescriptor.create("output"), InputDescriptor.create("input"));
+    set = new HashSet<>();
+  }
+
+  @Test(timeout = 5000)
+  public void testHashAndEqualsUnnamed() {
+    // edges without name but everything else same are equal and have same hash
+    Edge e1 = Edge.create(v1, v2, edgeProperty);
+    Edge e2 = Edge.create(v1, v2, edgeProperty);
+    assertEquals(e1, e2);
+    set.add(e1);
+    assertTrue(set.contains(e2));
+  }
+
+  @Test(timeout = 5000)
+  public void testHashAndEqualsNamed() {
+    // edges with everything same including name are equal and have same hash
+    Edge e1 = Edge.create(v1, v2, edgeProperty, "e1");
+    Edge e2 = Edge.create(v1, v2, edgeProperty, "e1");
+    assertEquals(e1, e2);
+    set.add(e1);
+    assertTrue(set.contains(e2));
+  }
+
+  @Test(timeout = 5000)
+  public void testHashAndEqualsDifferentName() {
+    // edges with different name but everything else same are not equal and have different hash
+    Edge e1 = Edge.create(v1, v2, edgeProperty, "e1");
+    Edge e2 = Edge.create(v1, v2, edgeProperty, "e2");
+    assertNotEquals(e1, e2);
+    set.add(e1);
+    assertFalse(set.contains(e2));
+  }
+}


[44/50] [abbrv] tez git commit: TEZ-3665. TestATSV15HistoryLoggingService should use mocked TimelineClient (zhiyuany)

Posted by zh...@apache.org.
TEZ-3665. TestATSV15HistoryLoggingService should use mocked TimelineClient (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: cbcb3a786e1af6a9ee61648ec9614686123182c3
Parents: 293c593
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Wed Mar 22 21:14:51 2017 -0700
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Wed Mar 22 21:14:51 2017 -0700

----------------------------------------------------------------------
 .../ats/TestATSV15HistoryLoggingService.java    | 144 ++++++++-----------
 1 file changed, 58 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/cbcb3a78/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
index cbded35..ef5da81 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
+++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
@@ -24,6 +24,9 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.anyVararg;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
@@ -68,13 +71,20 @@ import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.hadoop.shim.HadoopShim;
 import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestATSV15HistoryLoggingService {
   private static ApplicationId appId = ApplicationId.newInstance(1000l, 1);
   private static ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
   private static String user = "TEST_USER";
 
-  private InMemoryTimelineClient timelineClient;
+  private TimelineClient timelineClient;
+  Map<TimelineEntityGroupId, List<TimelineEntity>> entityLog;
+  final TimelineEntityGroupId DEFAULT_GROUP_ID =
+    TimelineEntityGroupId.newInstance(ApplicationId.newInstance(0, -1), "");
+
   private AppContext appContext;
 
   @Test(timeout=2000)
@@ -91,14 +101,14 @@ public class TestATSV15HistoryLoggingService {
       Thread.sleep(100);
     }
 
-    assertEquals(2, timelineClient.entityLog.size());
+    assertEquals(2, entityLog.size());
 
-    List<TimelineEntity> amEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> amEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, appId.toString()));
     assertNotNull(amEvents);
     assertEquals(1, amEvents.size());
 
-    List<TimelineEntity> nonGroupedDagEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> nonGroupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId1.toString()));
     assertNotNull(nonGroupedDagEvents);
     assertEquals(4, nonGroupedDagEvents.size());
@@ -119,14 +129,14 @@ public class TestATSV15HistoryLoggingService {
       Thread.sleep(100);
     }
 
-    assertEquals(2, timelineClient.entityLog.size());
+    assertEquals(2, entityLog.size());
 
-    List<TimelineEntity> amEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> amEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, appId.toString()));
     assertNotNull(amEvents);
     assertEquals(1, amEvents.size());
 
-    List<TimelineEntity> nonGroupedDagEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> nonGroupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId1.toString()));
     assertNotNull(nonGroupedDagEvents);
     assertEquals(4, nonGroupedDagEvents.size());
@@ -161,27 +171,27 @@ public class TestATSV15HistoryLoggingService {
     assertEquals(dagId1.getGroupId(numDagsPerGroup), dagId2.getGroupId(numDagsPerGroup));
     assertNotEquals(dagId2.getGroupId(numDagsPerGroup), dagId3.getGroupId(numDagsPerGroup));
 
-    assertEquals(3, timelineClient.entityLog.size());
+    assertEquals(3, entityLog.size());
 
-    List<TimelineEntity> amEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> amEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, appId.toString()));
     assertNotNull(amEvents);
     assertEquals(3, amEvents.size());
 
-    List<TimelineEntity> nonGroupedDagEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> nonGroupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId1.toString()));
     assertNull(nonGroupedDagEvents);
 
-    List<TimelineEntity> groupedDagEvents = timelineClient.entityLog.get(
+    List<TimelineEntity> groupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId1.getGroupId(numDagsPerGroup)));
     assertNotNull(groupedDagEvents);
     assertEquals(8, groupedDagEvents.size());
 
-    nonGroupedDagEvents = timelineClient.entityLog.get(
+    nonGroupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId3.toString()));
     assertNull(nonGroupedDagEvents);
 
-    groupedDagEvents = timelineClient.entityLog.get(
+    groupedDagEvents = entityLog.get(
         TimelineEntityGroupId.newInstance(appId, dagId3.getGroupId(numDagsPerGroup)));
     assertNotNull(groupedDagEvents);
     assertEquals(4, groupedDagEvents.size());
@@ -219,7 +229,7 @@ public class TestATSV15HistoryLoggingService {
 
     // All calls made with session domain id.
     verify(historyACLPolicyManager, times(5)).updateTimelineEntityDomain(any(), eq("session-id"));
-    assertTrue(timelineClient.entityLog.size() > 0);
+    assertTrue(entityLog.size() > 0);
 
     service.stop();
   }
@@ -252,7 +262,7 @@ public class TestATSV15HistoryLoggingService {
 
     // History logging is disabled.
     verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any());
-    assertEquals(0, timelineClient.entityLog.size());
+    assertEquals(0, entityLog.size());
 
     service.stop();
   }
@@ -285,7 +295,7 @@ public class TestATSV15HistoryLoggingService {
 
     // No domain updates but history logging happened.
     verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any());
-    assertTrue(timelineClient.entityLog.size() > 0);
+    assertTrue(entityLog.size() > 0);
 
     service.stop();
   }
@@ -370,7 +380,7 @@ public class TestATSV15HistoryLoggingService {
 
     // No history logging calls were done
     verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any());
-    assertEquals(0, timelineClient.entityLog.size());
+    assertEquals(0, entityLog.size());
 
     service.stop();
   }
@@ -413,12 +423,12 @@ public class TestATSV15HistoryLoggingService {
     // AM events sent, dag events are not sent.
     verify(historyACLPolicyManager, times(1)).updateTimelineEntityDomain(any(), eq("session-id"));
     verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), eq("dag-id"));
-    assertEquals(1, timelineClient.entityLog.size());
+    assertEquals(1, entityLog.size());
 
     service.stop();
   }
 
-  private ATSV15HistoryLoggingService createService(int numDagsPerGroup) {
+  private ATSV15HistoryLoggingService createService(int numDagsPerGroup) throws IOException, YarnException {
     ATSV15HistoryLoggingService service = new ATSV15HistoryLoggingService();
     appContext = mock(AppContext.class);
     when(appContext.getApplicationID()).thenReturn(appId);
@@ -433,13 +443,40 @@ public class TestATSV15HistoryLoggingService {
     service.init(conf);
 
     // Set timeline service.
-    timelineClient = new InMemoryTimelineClient();
-    timelineClient.init(conf);
+    timelineClient = mock(TimelineClient.class);
+    entityLog = new HashMap<>();
+    //timelineClient.init(conf);
+    when(timelineClient.getDelegationToken(anyString())).thenReturn(null);
+    when(timelineClient.renewDelegationToken(Matchers.<Token<TimelineDelegationTokenIdentifier>>any())).thenReturn(0L);
+    when(timelineClient.putEntities(Matchers.<TimelineEntity>anyVararg())).thenAnswer(new Answer() {
+      @Override
+      public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable {
+        return putEntityHelper(DEFAULT_GROUP_ID, invocation.getArguments(), 0);
+      }
+    });
+    when(timelineClient.putEntities(any(ApplicationAttemptId.class), any(TimelineEntityGroupId.class), Matchers.<TimelineEntity>anyVararg())).thenAnswer(new Answer() {
+      @Override
+      public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable {
+        return putEntityHelper(invocation.getArgumentAt(1, TimelineEntityGroupId.class), invocation.getArguments(), 2);
+      }
+    });
     service.timelineClient = timelineClient;
 
     return service;
   }
 
+  private TimelinePutResponse putEntityHelper(TimelineEntityGroupId groupId, Object[] args, int firstEntityIdx) {
+    List<TimelineEntity> groupEntities = entityLog.get(groupId);
+    if (groupEntities == null) {
+      groupEntities = new ArrayList<>();
+      entityLog.put(groupId, groupEntities);
+    }
+    for (int i = firstEntityIdx; i < args.length; i++) {
+      groupEntities.add((TimelineEntity) args[i]);
+    }
+    return null;
+  }
+
   private List<DAGHistoryEvent> makeHistoryEvents(TezDAGID dagId,
                                                   ATSV15HistoryLoggingService service) {
     List<DAGHistoryEvent> historyEvents = new ArrayList<>();
@@ -463,69 +500,4 @@ public class TestATSV15HistoryLoggingService {
             null, null)));
     return historyEvents;
   }
-
-  private static class InMemoryTimelineClient extends TimelineClient {
-    Map<TimelineEntityGroupId, List<TimelineEntity>> entityLog = new HashMap<>();
-
-    protected InMemoryTimelineClient() {
-      super("InMemoryTimelineClient");
-    }
-
-    @Override
-    public void flush() throws IOException {
-    }
-
-    public static final ApplicationId DEFAULT_APP_ID = ApplicationId.newInstance(0, -1);
-    public static final TimelineEntityGroupId DEFAULT_GROUP_ID =
-        TimelineEntityGroupId.newInstance(DEFAULT_APP_ID, "");
-
-    @Override
-    public synchronized TimelinePutResponse putEntities(TimelineEntity... entities)
-        throws IOException, YarnException {
-      return putEntities(null, DEFAULT_GROUP_ID, entities);
-    }
-
-    @Override
-    public TimelinePutResponse putEntities(ApplicationAttemptId appAttemptId,
-        TimelineEntityGroupId groupId,
-        TimelineEntity... entities) throws IOException, YarnException {
-      List<TimelineEntity> groupEntities = entityLog.get(groupId);
-      if (groupEntities == null) {
-        groupEntities = new ArrayList<>();
-        entityLog.put(groupId, groupEntities);
-      }
-      for (TimelineEntity entity : entities) {
-        groupEntities.add(entity);
-      }
-      return null;
-    }
-
-    @Override
-    public void putDomain(TimelineDomain domain) throws IOException, YarnException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void putDomain(ApplicationAttemptId appAttemptId, TimelineDomain domain)
-        throws IOException, YarnException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Token<TimelineDelegationTokenIdentifier> getDelegationToken(String renewer)
-        throws IOException, YarnException {
-      return null;
-    }
-
-    @Override
-    public long renewDelegationToken(Token<TimelineDelegationTokenIdentifier> timelineDT)
-        throws IOException, YarnException {
-      return 0;
-    }
-
-    @Override
-    public void cancelDelegationToken(Token<TimelineDelegationTokenIdentifier> timelineDT)
-        throws IOException, YarnException {
-    }
-  }
 }


[18/50] [abbrv] tez git commit: Revert "TEZ-3625. Dag.getVertex should obtain a readlock. (sseth)"

Posted by zh...@apache.org.
Revert "TEZ-3625. Dag.getVertex should obtain a readlock. (sseth)"

This reverts commit dcae5e42e961a6b2fd0e82a2392d601ca935d0f1.


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

Branch: refs/heads/TEZ-1190
Commit: 2158b95df521b1e49b0a5ee1bd71560bb12234c7
Parents: 359cd75
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 24 12:52:20 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Feb 24 12:52:20 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                                   | 1 -
 .../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java    | 1 -
 .../main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java    | 7 +------
 3 files changed, 1 insertion(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2158b95d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b855af0..34f8624 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -10,7 +10,6 @@ ALL CHANGES:
   TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.
   TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.
   TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes
-  TEZ-3625. Dag.getVertex should obtain a readlock.
   TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
   TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3267. Publish queue name to ATS as part of dag summary.

http://git-wip-us.apache.org/repos/asf/tez/blob/2158b95d/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 2085789..de02c18 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -2638,7 +2638,6 @@ public class DAGAppMaster extends AbstractService {
 
     // Send out an event to inform components that a new DAG has been submitted.
     // Information about this DAG is available via the context.
-    // This event may be processed after DAG_INIT, but will be processed before DAG_START
     sendEvent(new DAGAppMasterEvent(DAGAppMasterEventType.NEW_DAG_SUBMITTED));
     // create a job event for job initialization
     DAGEvent initDagEvent = new DAGEvent(currentDAG.getID(), DAGEventType.DAG_INIT);

http://git-wip-us.apache.org/repos/asf/tez/blob/2158b95d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 047fea6..481353b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -1810,12 +1810,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   @Override
   public Vertex getVertex(String vertexName) {
-    this.readLock.lock();
-    try {
-      return vertexMap.get(vertexName);
-    } finally {
-      this.readLock.unlock();
-    }
+    return vertexMap.get(vertexName);
   }
 
   private void mayBeConstructFinalFullCounters() {


[21/50] [abbrv] tez git commit: TEZ-3639. Tez UI: Footer pagination is improper in landing page (sree)

Posted by zh...@apache.org.
TEZ-3639. Tez UI: Footer pagination is improper in landing page (sree)


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

Branch: refs/heads/TEZ-1190
Commit: 57119a5119637c1f9f356216bcc155bb238d01ec
Parents: bdc0ee9
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Tue Feb 28 14:47:23 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Tue Feb 28 14:47:23 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                       | 1 +
 tez-ui/src/main/webapp/app/controllers/home/index.js              | 1 +
 tez-ui/src/main/webapp/app/controllers/home/queries.js            | 1 +
 tez-ui/src/main/webapp/app/templates/home/index.hbs               | 1 +
 tez-ui/src/main/webapp/app/templates/home/queries.hbs             | 1 +
 tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js  | 3 ++-
 .../src/main/webapp/tests/unit/controllers/home/queries-test.js   | 1 +
 7 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 88b0b98..8bb3749 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -205,6 +205,7 @@ ALL CHANGES:
   TEZ-3629. Tez UI: Enable the UI to display log links from LLAP
   TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly
   TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling
+  TEZ-3639. Tez UI: Footer pagination is improper in landing page
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/app/controllers/home/index.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js
index 61e7a9f..9194fd6 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/index.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/index.js
@@ -49,6 +49,7 @@ export default TableController.extend({
   loadingMore: false,
 
   headerComponentNames: ['dags-page-search', 'table-controls', 'pagination-ui'],
+  footerComponentNames: ['pagination-ui'],
 
   _definition: TableDefinition.create(),
   // Using computed, as observer won't fire if the property is not used

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/app/controllers/home/queries.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js
index 49da719..57b3964 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/queries.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js
@@ -50,6 +50,7 @@ export default TableController.extend({
   loadingMore: false,
 
   headerComponentNames: ['queries-page-search', 'table-controls', 'pagination-ui'],
+  footerComponentNames: ['pagination-ui'],
 
   _definition: TableDefinition.create(),
   // Using computed, as observer won't fire if the property is not used

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/app/templates/home/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/home/index.hbs b/tez-ui/src/main/webapp/app/templates/home/index.hbs
index bf4319c..fe18266 100644
--- a/tez-ui/src/main/webapp/app/templates/home/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/home/index.hbs
@@ -25,6 +25,7 @@
     classNames="all-dags-table"
 
     headerComponentNames=headerComponentNames
+    footerComponentNames=footerComponentNames
 
     definition=definition
     enableSort=false

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/app/templates/home/queries.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/home/queries.hbs b/tez-ui/src/main/webapp/app/templates/home/queries.hbs
index 22e60e3..db38f96 100644
--- a/tez-ui/src/main/webapp/app/templates/home/queries.hbs
+++ b/tez-ui/src/main/webapp/app/templates/home/queries.hbs
@@ -25,6 +25,7 @@
   classNames="all-queries-table"
 
   headerComponentNames=headerComponentNames
+  footerComponentNames=footerComponentNames
 
   definition=definition
   enableSort=false

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
index 810c8c9..579cc76 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
@@ -26,7 +26,7 @@ moduleFor('controller:home/index', 'Unit | Controller | home/index', {
 });
 
 test('Basic creation test', function(assert) {
-  assert.expect(2 + 12);
+  assert.expect(2 + 4 + 1 + 4 + 2 + 2);
 
   let controller = this.subject({
     initVisibleColumns: Ember.K,
@@ -47,6 +47,7 @@ test('Basic creation test', function(assert) {
   assert.ok(controller.queryParams);
   assert.ok(controller.headerComponentNames);
   assert.equal(controller.headerComponentNames.length, 3);
+  assert.equal(controller.footerComponentNames.length, 1);
 
   assert.ok(controller._definition);
   assert.ok(controller.definition);

http://git-wip-us.apache.org/repos/asf/tez/blob/57119a51/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
index ccec36c..22461d2 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
@@ -38,6 +38,7 @@ test('Basic creation test', function(assert) {
   assert.ok(controller.breadcrumbs);
   assert.ok(controller.headerComponentNames);
   assert.equal(controller.headerComponentNames.length, 3);
+  assert.equal(controller.footerComponentNames.length, 1);
 
   assert.ok(controller.definition);
   assert.ok(controller.columns);


[15/50] [abbrv] tez git commit: TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly (sree)

Posted by zh...@apache.org.
TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly (sree)


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

Branch: refs/heads/TEZ-1190
Commit: fc0897b92802bbbd922d2a90cdc07cf2d3b0a3e1
Parents: 3914342
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Wed Feb 22 17:05:18 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Wed Feb 22 17:05:18 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 tez-ui/src/main/webapp/app/components/date-formatter.js  |  8 ++++++++
 .../webapp/app/templates/components/date-formatter.hbs   |  2 +-
 .../tests/integration/components/date-formatter-test.js  | 11 +++++++++++
 4 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fc0897b9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 54cddc8..5427e12 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -201,6 +201,7 @@ ALL CHANGES:
   TEZ-3615. Tez UI: Table changes
   TEZ-3619. Tez UI: Improve DAG Data download
   TEZ-3629. Tez UI: Enable the UI to display log links from LLAP
+  TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc0897b9/tez-ui/src/main/webapp/app/components/date-formatter.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/components/date-formatter.js b/tez-ui/src/main/webapp/app/components/date-formatter.js
index da21383..e428f51 100644
--- a/tez-ui/src/main/webapp/app/components/date-formatter.js
+++ b/tez-ui/src/main/webapp/app/components/date-formatter.js
@@ -24,6 +24,14 @@ export default Ember.Component.extend({
 
   content: null,
 
+  date: Ember.computed("content", function () {
+    var content = this.get("content");
+    if(content <= 0) {
+      content = undefined;
+    }
+    return content;
+  }),
+
   env: Ember.inject.service('env'),
   timeZone: Ember.computed.oneWay('env.app.timeZone'),
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fc0897b9/tez-ui/src/main/webapp/app/templates/components/date-formatter.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/date-formatter.hbs b/tez-ui/src/main/webapp/app/templates/components/date-formatter.hbs
index 0f740cb..cdf90da 100644
--- a/tez-ui/src/main/webapp/app/templates/components/date-formatter.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/date-formatter.hbs
@@ -16,4 +16,4 @@
  * limitations under the License.
 }}
 
-{{txt content type="date" timeZone=timeZone}}
\ No newline at end of file
+{{txt date type="date" timeZone=timeZone}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/fc0897b9/tez-ui/src/main/webapp/tests/integration/components/date-formatter-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/integration/components/date-formatter-test.js b/tez-ui/src/main/webapp/tests/integration/components/date-formatter-test.js
index dad41f9..9b1ccea 100644
--- a/tez-ui/src/main/webapp/tests/integration/components/date-formatter-test.js
+++ b/tez-ui/src/main/webapp/tests/integration/components/date-formatter-test.js
@@ -38,3 +38,14 @@ test('Basic creation test', function(assert) {
 
   assert.equal(this.$().text().trim(), 'Not Available!');
 });
+
+test('Negative value test', function(assert) {
+  this.render(hbs`{{date-formatter -1}}`);
+  assert.equal(this.$().text().trim(), 'Not Available!');
+
+  this.render(hbs`{{date-formatter -99}}`);
+  assert.equal(this.$().text().trim(), 'Not Available!');
+
+  this.render(hbs`{{date-formatter 0}}`);
+  assert.equal(this.$().text().trim(), 'Not Available!');
+});
\ No newline at end of file


[35/50] [abbrv] tez git commit: TEZ-3656. Tez UI: Status correction is not working as expected (sree)

Posted by zh...@apache.org.
TEZ-3656. Tez UI: Status correction is not working as expected (sree)


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

Branch: refs/heads/TEZ-1190
Commit: 852e80d843c7bffd0c0670047f3b4fd392ea6d5f
Parents: a9af6cf
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Mon Mar 13 18:30:28 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Mon Mar 13 18:30:28 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                      |  1 +
 tez-ui/src/main/webapp/app/models/dag.js         |  2 +-
 .../main/webapp/tests/unit/models/dag-test.js    | 19 +++++++++++++++++++
 3 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/852e80d8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 54b17b1..87dda41 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -218,6 +218,7 @@ ALL CHANGES:
   TEZ-3640. Tez UI: Add associated llap application id to queries page
   TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
   TEZ-3503. Tez UI: Support search by queue name
+  TEZ-3656. Tez UI: Status correction is not working as expected
 
 Release 0.8.6: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/852e80d8/tez-ui/src/main/webapp/app/models/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/dag.js b/tez-ui/src/main/webapp/app/models/dag.js
index 7e57615..2ab4af3 100644
--- a/tez-ui/src/main/webapp/app/models/dag.js
+++ b/tez-ui/src/main/webapp/app/models/dag.js
@@ -44,7 +44,7 @@ export default AMTimelineModel.extend({
       type: ["AhsApp", "appRm"],
       idKey: "appID",
       loadType: function (record) {
-        if(record.get("queueName")) {
+        if(record.get("queueName") && record.get("atsStatus") !== "RUNNING") {
           return "demand";
         }
       },

http://git-wip-us.apache.org/repos/asf/tez/blob/852e80d8/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
index a219cf8..06e0ec1 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
@@ -59,6 +59,25 @@ test('Basic creation test', function(assert) {
   assert.ok(model.amWsVersion);
 });
 
+test('app loadType test', function(assert) {
+  let loadType = this.subject().get("needs.app.loadType"),
+      record = Ember.Object.create();
+
+  assert.equal(loadType(record), undefined);
+
+  record.set("queueName", "Q");
+  assert.equal(loadType(record), "demand");
+
+  record.set("atsStatus", "RUNNING");
+  assert.equal(loadType(record), undefined);
+
+  record.set("atsStatus", "SUCCEEDED");
+  assert.equal(loadType(record), "demand");
+
+  record.set("queueName", undefined);
+  assert.equal(loadType(record), undefined);
+});
+
 test('queue test', function(assert) {
   let model = this.subject(),
       queueName = "queueName",


[27/50] [abbrv] tez git commit: TEZ-3649. AsyncHttpConnection should add StopWatch start (Fei Hui via rbalamohan)

Posted by zh...@apache.org.
TEZ-3649. AsyncHttpConnection should add StopWatch start (Fei Hui via rbalamohan)


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

Branch: refs/heads/TEZ-1190
Commit: 518deb6c2ffba7db9af770ce0b5b248409d56aff
Parents: 1b1eb1d
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Mon Mar 6 14:30:50 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Mon Mar 6 14:30:50 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../java/org/apache/tez/http/async/netty/AsyncHttpConnection.java   | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/518deb6c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 763acd8..f90a29e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3649. AsyncHttpConnection should add StopWatch start.
   TEZ-3647. Add a setting which lets Tez determine Xmx.
   TEZ-3644. Cleanup container list stored in AMNode.
   TEZ-3646. IFile.Writer has an extra output stream flush call

http://git-wip-us.apache.org/repos/asf/tez/blob/518deb6c/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
index 5adfc3c..735bb46 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
@@ -176,6 +176,7 @@ public class AsyncHttpConnection extends BaseHttpConnection {
   }
 
   public void validate() throws IOException {
+    stopWatch.reset().start();
     // get the shuffle version
     if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME
         .equals(response.getHeader(ShuffleHeader.HTTP_HEADER_NAME))


[10/50] [abbrv] tez git commit: TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl. (sseth)

Posted by zh...@apache.org.
TEZ-3624. Split multiple calls on the same line in
TaskCommunicatorContextImpl. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: 2268c720f1cb5b0f36d970ef6da88d940d13f6c7
Parents: daa8d3d
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 15 20:29:38 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Feb 15 20:29:38 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../dag/app/TaskCommunicatorContextImpl.java    | 25 ++++++++++++++------
 2 files changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2268c720/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 31e141c..a3323b9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
   TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3267. Publish queue name to ATS as part of dag summary.
   TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
@@ -203,6 +204,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
   TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent

http://git-wip-us.apache.org/repos/asf/tez/blob/2268c720/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
index 1fbf853..2709787 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorContextImpl.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.runtime.api.TaskFailureType;
 import org.apache.tez.serviceplugins.api.DagInfo;
@@ -144,7 +145,8 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   public void registerForVertexStateUpdates(String vertexName,
                                             @Nullable Set<VertexState> stateSet) {
     Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
-    getDag().getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet,
+    DAG dag = getDag();
+    dag.getStateChangeNotifier().registerForVertexUpdates(vertexName, stateSet,
         this);
   }
 
@@ -162,7 +164,8 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   @Override
   public Iterable<String> getInputVertexNames(String vertexName) {
     Preconditions.checkNotNull(vertexName, "VertexName cannot be null: " + vertexName);
-    Vertex vertex = getDag().getVertex(vertexName);
+    DAG dag = getDag();
+    Vertex vertex = dag.getVertex(vertexName);
     Set<Vertex> sources = vertex.getInputVertices().keySet();
     return Iterables.transform(sources, new Function<Vertex, String>() {
       @Override
@@ -175,27 +178,35 @@ public class TaskCommunicatorContextImpl implements TaskCommunicatorContext, Ver
   @Override
   public int getVertexTotalTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return getDag().getVertex(vertexName).getTotalTasks();
+    DAG dag = getDag();
+    Vertex vertex = dag.getVertex(vertexName);
+    return vertex.getTotalTasks();
   }
 
   @Override
   public int getVertexCompletedTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return getDag().getVertex(vertexName).getCompletedTasks();
+    DAG dag = getDag();
+    Vertex vertex = dag.getVertex(vertexName);
+    return vertex.getCompletedTasks();
   }
 
   @Override
   public int getVertexRunningTaskCount(String vertexName) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
-    return getDag().getVertex(vertexName).getRunningTasks();
+    DAG dag = getDag();
+    Vertex vertex = dag.getVertex(vertexName);
+    return vertex.getRunningTasks();
   }
 
   @Override
   public long getFirstAttemptStartTime(String vertexName, int taskIndex) {
     Preconditions.checkArgument(vertexName != null, "VertexName must be specified");
     Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0");
-    return getDag().getVertex(vertexName).getTask(
-        taskIndex).getFirstAttemptStartTime();
+    DAG dag = getDag();
+    Vertex vertex = dag.getVertex(vertexName);
+    Task task = vertex.getTask(taskIndex);
+    return task.getFirstAttemptStartTime();
   }
 
   @Override


[09/50] [abbrv] tez git commit: TEZ-3619. Tez UI: Improve DAG Data download (sree)

Posted by zh...@apache.org.
TEZ-3619. Tez UI: Improve DAG Data download (sree)


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

Branch: refs/heads/TEZ-1190
Commit: daa8d3db0639ca93085072543dec2baac803e427
Parents: cd44c90
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Wed Feb 15 21:32:21 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Wed Feb 15 21:32:21 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../webapp/app/components/zip-download-modal.js |  2 +-
 .../webapp/app/styles/zip-download-modal.less   | 13 +++-
 .../templates/components/zip-download-modal.hbs | 12 ++--
 .../main/webapp/app/utils/download-dag-zip.js   | 75 ++++++++++++++++++--
 .../components/zip-download-modal-test.js       | 52 +++++++++++++-
 6 files changed, 140 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 337b394..31e141c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -195,6 +195,7 @@ ALL CHANGES:
   TEZ-3598. Tez UI: Text formatting changes
   TEZ-3602. Tez UI: Query Name field is not required
   TEZ-3615. Tez UI: Table changes
+  TEZ-3619. Tez UI: Improve DAG Data download
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/tez-ui/src/main/webapp/app/components/zip-download-modal.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/components/zip-download-modal.js b/tez-ui/src/main/webapp/app/components/zip-download-modal.js
index c55b34e..528c84e 100644
--- a/tez-ui/src/main/webapp/app/components/zip-download-modal.js
+++ b/tez-ui/src/main/webapp/app/components/zip-download-modal.js
@@ -23,7 +23,7 @@ export default Ember.Component.extend({
   content: null,
 
   _onSuccess: Ember.observer("content.downloader.succeeded", function () {
-    if(this.get("content.downloader.succeeded")) {
+    if(this.get("content.downloader.succeeded") && !this.get("content.downloader.partial")) {
       Ember.run.later(this, "close");
     }
   }),

http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/tez-ui/src/main/webapp/app/styles/zip-download-modal.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/zip-download-modal.less b/tez-ui/src/main/webapp/app/styles/zip-download-modal.less
index 4ed0fd2..49b5245 100644
--- a/tez-ui/src/main/webapp/app/styles/zip-download-modal.less
+++ b/tez-ui/src/main/webapp/app/styles/zip-download-modal.less
@@ -20,11 +20,22 @@
   .message {
     padding: 10px 15px;
 
+    i {
+      margin-right: 5px;
+    }
+
     .fa-spinner {
       color: green;
     }
-    .fa-exclamation-circle {
+    .error {
       color: red;
     }
+    .warning {
+      color: orange;
+    }
+
+    .progress {
+      margin: 5px 0 5px 0;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/tez-ui/src/main/webapp/app/templates/components/zip-download-modal.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/zip-download-modal.hbs b/tez-ui/src/main/webapp/app/templates/components/zip-download-modal.hbs
index 03b820e..937722c 100644
--- a/tez-ui/src/main/webapp/app/templates/components/zip-download-modal.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/zip-download-modal.hbs
@@ -18,17 +18,21 @@
 
 <div class="message">
   {{#if content.downloader.failed}}
-    <i class="fa fa-lg fa-exclamation-circle"></i>
+    <i class="fa fa-lg fa-exclamation-circle error"></i>
     Error downloading data!
+  {{else}}{{#if (and content.downloader.partial content.downloader.succeeded)}}
+    <i class="fa fa-lg fa-exclamation-circle warning"></i>
+    Data downloaded might be incomplete. Please check the zip!
   {{else}}
-    <i class="fa fa-lg fa-spinner fa-spin"></i>
     Downloading data for dag: <b>{{content.dag.entityID}}</b>
-  {{/if}}
+    {{em-progress value=content.downloader.percent striped=true}}
+    {{content.downloader.message}}
+  {{/if}}{{/if}}
 </div>
 
 
 <div class="form-actions">
-  {{#if content.downloader.failed}}
+  {{#if (or content.downloader.failed content.downloader.partial)}}
     <button type="button" class="btn btn-primary" data-dismiss="modal" aria-label="Close">Ok</button>
   {{else}}
     <button type="button" class="btn" data-dismiss="modal" aria-label="Close" {{action "cancel"}}>Cancel</button>

http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/tez-ui/src/main/webapp/app/utils/download-dag-zip.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/utils/download-dag-zip.js b/tez-ui/src/main/webapp/app/utils/download-dag-zip.js
index 9c91a92..8f10bde 100644
--- a/tez-ui/src/main/webapp/app/utils/download-dag-zip.js
+++ b/tez-ui/src/main/webapp/app/utils/download-dag-zip.js
@@ -110,6 +110,10 @@ var IO = {
       var reqID = getRequestId();
       pendingRequests[reqID] = xhr;
 
+      if(item.onFetch) {
+        item.onFetch(item.context);
+      }
+
       xhr.done(function(data/*, statusText, xhr*/) {
         delete pendingRequests[reqID];
 
@@ -129,8 +133,17 @@ var IO = {
       }).fail(function(xhr, statusText/*, errorObject*/) {
         delete pendingRequests[reqID];
         inProgress--;
-        if(item.onItemFail) {
-          item.onItemFail();
+
+        if(item.retryCount) {
+          itemList.unshift(item);
+          item.retryCount--;
+          if(item.onRetry) {
+            item.onRetry(item.context);
+          }
+          Ember.run.later(processNext, 3000 + Math.random() * 3000);
+        }
+        else if(item.onItemFail) {
+          item.onItemFail(xhr, item.context);
           processNext();
         }
         else {
@@ -286,27 +299,47 @@ export default function downloadDagZip(dag, options) {
         {
           url: getUrl('TEZ_APPLICATION', 'tez_' + dag.get("appID")),
           context: { name: 'application', type: 'TEZ_APPLICATION' },
-          onItemFetched: processSingleItem
+          onFetch: onFetch,
+          onRetry: onRetry,
+          onItemFetched: processSingleItem,
+          onItemFail: processFailure,
+          retryCount: 3,
         },
         {
           url: getUrl('TEZ_DAG_ID', dagID),
           context: { name: 'dag', type: 'TEZ_DAG_ID' },
-          onItemFetched: processSingleItem
+          onFetch: onFetch,
+          onRetry: onRetry,
+          onItemFetched: processSingleItem,
+          onItemFail: processFailure,
+          retryCount: 3,
         },
         {
           url: getUrl('TEZ_VERTEX_ID', null, dagID),
           context: { name: 'vertices', type: 'TEZ_VERTEX_ID', part: 0 },
-          onItemFetched: processMultipleItems
+          onFetch: onFetch,
+          onRetry: onRetry,
+          onItemFetched: processMultipleItems,
+          onItemFail: processFailure,
+          retryCount: 3,
         },
         {
           url: getUrl('TEZ_TASK_ID', null, dagID),
           context: { name: 'tasks', type: 'TEZ_TASK_ID', part: 0 },
-          onItemFetched: processMultipleItems
+          onFetch: onFetch,
+          onRetry: onRetry,
+          onItemFetched: processMultipleItems,
+          onItemFail: processFailure,
+          retryCount: 3,
         },
         {
           url: getUrl('TEZ_TASK_ATTEMPT_ID', null, dagID),
           context: { name: 'task_attempts', type: 'TEZ_TASK_ATTEMPT_ID', part: 0 },
-          onItemFetched: processMultipleItems
+          onFetch: onFetch,
+          onRetry: onRetry,
+          onItemFetched: processMultipleItems,
+          onItemFail: processFailure,
+          retryCount: 3,
         }
       ];
 
@@ -334,7 +367,9 @@ export default function downloadDagZip(dag, options) {
       }),
       downloaderProxy = Ember.Object.create({
         percent: 0,
+        message: "",
         succeeded: false,
+        partial: false,
         failed: false,
         cancel: function() {
           downloader.cancel();
@@ -353,6 +388,7 @@ export default function downloadDagZip(dag, options) {
         url = `${url}&fromId=${fromID}`;
       }
     }
+
     return url;
   }
 
@@ -367,6 +403,30 @@ export default function downloadDagZip(dag, options) {
     }
   }
 
+  function onFetch(context) {
+    downloaderProxy.set("message", `Fetching ${context.name} data.`);
+  }
+
+  function onRetry(context) {
+    downloaderProxy.set("message", `Downloading ${context.name} data failed. Retrying!`);
+  }
+
+  function processFailure(data, context) {
+    var obj = {};
+    try {
+      obj[context.name] = JSON.parse(data.responseText);
+    }
+    catch(e) {
+      obj[context.name] = data.responseText;
+    }
+
+    downloaderProxy.set("partial", true);
+    downloaderProxy.set("message", `Downloading ${context.name} data failed!`);
+
+    zipHelper.addFile({name: `error.${context.name}.json`, data: JSON.stringify(obj, null, 2)});
+    checkIfAllDownloaded();
+  }
+
   function processSingleItem(data, context) {
     var obj = {};
     obj[context.name] = data;
@@ -415,6 +475,7 @@ export default function downloadDagZip(dag, options) {
 
   zipHelper.then(function(zippedBlob) {
     saveAs(zippedBlob, `${dagID}.zip`);
+    downloaderProxy.set("message", `Download complete.`);
     downloaderProxy.set("succeeded", true);
   }, function() {
     Ember.Logger.error('zip Failed');

http://git-wip-us.apache.org/repos/asf/tez/blob/daa8d3db/tez-ui/src/main/webapp/tests/integration/components/zip-download-modal-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/integration/components/zip-download-modal-test.js b/tez-ui/src/main/webapp/tests/integration/components/zip-download-modal-test.js
index cd9a61a..8326054 100644
--- a/tez-ui/src/main/webapp/tests/integration/components/zip-download-modal-test.js
+++ b/tez-ui/src/main/webapp/tests/integration/components/zip-download-modal-test.js
@@ -34,7 +34,7 @@ test('Basic creation test', function(assert) {
   });
 
   this.render(hbs`{{zip-download-modal content=content}}`);
-  assert.equal(this.$(".message").text().trim(), expectedMessage);
+  assert.equal(this.$(".message").text().trim().indexOf(expectedMessage), 0);
 
   // Template block usage:" + EOL +
   this.render(hbs`
@@ -42,5 +42,53 @@ test('Basic creation test', function(assert) {
       template block text
     {{/zip-download-modal}}
   `);
-  assert.equal(this.$(".message").text().trim(), expectedMessage);
+  assert.equal(this.$(".message").text().trim().indexOf(expectedMessage), 0);
 });
+
+test('progress test', function(assert) {
+  this.set("content", {
+    downloader: {
+      percent: 0.5
+    }
+  });
+
+  this.render(hbs`{{zip-download-modal content=content}}`);
+  let text = this.$(".message").text().trim();
+  assert.equal(text.substr(-3), "50%");
+
+  assert.equal(this.$(".btn").length, 1);
+  assert.equal(this.$(".btn-primary").length, 0);
+});
+
+test('failed test', function(assert) {
+  var expectedMessage = "Error downloading data!";
+
+  this.set("content", {
+    downloader: {
+      failed: true
+    }
+  });
+
+  this.render(hbs`{{zip-download-modal content=content}}`);
+  assert.equal(this.$(".message").text().trim().indexOf(expectedMessage), 0);
+
+  assert.equal(this.$(".btn").length, 1);
+  assert.equal(this.$(".btn-primary").length, 1);
+});
+
+test('partial test', function(assert) {
+  var expectedMessage = "Data downloaded might be incomplete. Please check the zip!";
+
+  this.set("content", {
+    downloader: {
+      succeeded: true,
+      partial: true
+    }
+  });
+
+  this.render(hbs`{{zip-download-modal content=content}}`);
+  assert.equal(this.$(".message").text().trim().indexOf(expectedMessage), 0);
+
+  assert.equal(this.$(".btn").length, 1);
+  assert.equal(this.$(".btn-primary").length, 1);
+});
\ No newline at end of file


[25/50] [abbrv] tez git commit: TEZ-3644. Cleanup container list stored in AMNode. (sseth)

Posted by zh...@apache.org.
TEZ-3644. Cleanup container list stored in AMNode. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: 4ce6ea6ed867a67600dbc36a2f56c37bbec3d708
Parents: 1f2a935
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 2 16:02:16 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Mar 2 16:02:16 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../dag/app/rm/container/AMContainerImpl.java   |  62 +++++++--
 .../org/apache/tez/dag/app/rm/node/AMNode.java  |   3 +
 .../rm/node/AMNodeEventContainerCompleted.java  |  37 ++++++
 .../tez/dag/app/rm/node/AMNodeEventType.java    |   5 +-
 .../apache/tez/dag/app/rm/node/AMNodeImpl.java  |  67 ++++++++--
 .../tez/dag/app/rm/node/AMNodeTracker.java      |   5 +-
 .../dag/app/rm/node/PerSourceNodeTracker.java   |  11 +-
 .../dag/app/rm/container/TestAMContainer.java   | 128 ++++++++++++-------
 .../tez/dag/app/rm/node/TestAMNodeTracker.java  |  73 +++++++++++
 10 files changed, 322 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b8465de..07841bf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3644. Cleanup container list stored in AMNode.
   TEZ-3646. IFile.Writer has an extra output stream flush call
   TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances.
   TEZ-3637. TezMerger logs too much at INFO level

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index ac429c7..18e72a7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -32,9 +32,12 @@ import org.apache.tez.Utils;
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError;
+import org.apache.tez.dag.app.rm.node.AMNodeEventContainerCompleted;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.common.ContainerSignatureMatcher;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
+import org.apache.tez.state.OnStateChangedCallback;
+import org.apache.tez.state.StateMachineTez;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.security.Credentials;
@@ -48,7 +51,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
-import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tez.dag.app.AppContext;
@@ -118,6 +120,8 @@ public class AMContainerImpl implements AMContainer {
 
   private Credentials credentials;
   private boolean credentialsChanged = false;
+
+  private boolean completedMessageSent = false;
   
   // TODO Consider registering with the TAL, instead of the TAL pulling.
   // Possibly after splitting TAL and ContainerListener.
@@ -127,8 +131,11 @@ public class AMContainerImpl implements AMContainer {
 
   // TODO Create a generic ERROR state. Container tries informing relevant components in this case.
 
+  private final NonRunningStateEnteredCallback NON_RUNNING_STATE_ENTERED_CALLBACK = new NonRunningStateEnteredCallback();
+
+  private final StateMachineTez<AMContainerState, AMContainerEventType, AMContainerEvent, AMContainerImpl>
+      stateMachine;
 
-  private final StateMachine<AMContainerState, AMContainerEventType, AMContainerEvent> stateMachine;
   private static final StateMachineFactory
       <AMContainerImpl, AMContainerState, AMContainerEventType, AMContainerEvent>
       stateMachineFactory =
@@ -328,7 +335,19 @@ public class AMContainerImpl implements AMContainer {
     this.schedulerId = schedulerId;
     this.launcherId = launcherId;
     this.taskCommId = taskCommId;
-    this.stateMachine = stateMachineFactory.make(this);
+    this.stateMachine = new StateMachineTez<>(stateMachineFactory.make(this), this);
+    augmentStateMachine();
+  }
+
+
+  private void augmentStateMachine() {
+    stateMachine
+        .registerStateEnteredCallback(AMContainerState.STOP_REQUESTED,
+            NON_RUNNING_STATE_ENTERED_CALLBACK)
+        .registerStateEnteredCallback(AMContainerState.STOPPING,
+            NON_RUNNING_STATE_ENTERED_CALLBACK)
+        .registerStateEnteredCallback(AMContainerState.COMPLETED,
+            NON_RUNNING_STATE_ENTERED_CALLBACK);
   }
 
   @Override
@@ -422,7 +441,7 @@ public class AMContainerImpl implements AMContainer {
         LOG.error("Can't handle event " + event.getType()
             + " at current state " + oldState + " for ContainerId "
             + this.containerId, e);
-        inError = true;
+        setError();
         // TODO Can't set state to COMPLETED. Add a default error state.
       }
       if (oldState != getState()) {
@@ -482,7 +501,7 @@ public class AMContainerImpl implements AMContainer {
                 msg, e));
         // We have not registered with any of the listeners etc yet. Send out a deallocateContainer
         // message and return. The AM will shutdown shortly.
-        container.inError = true;
+        container.setError();
         container.deAllocate();
         return;
       }
@@ -515,7 +534,7 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
-      container.inError = true;
+      container.setError();
       container.registerFailedAttempt(event.getTaskAttemptId());
       container.maybeSendNodeFailureForFailedAssignment(event
           .getTaskAttemptId());
@@ -961,7 +980,7 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
-      container.inError = true;
+      container.setError();
       String errorMessage = "AttemptId: " + event.getTaskAttemptId() +
           " cannot be allocated to container: " + container.getContainerId() +
           " in " + container.getState() + " state";
@@ -1032,7 +1051,7 @@ public class AMContainerImpl implements AMContainer {
 
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
-      container.inError = true;
+      container.setError();
     }
   }
 
@@ -1046,7 +1065,7 @@ public class AMContainerImpl implements AMContainer {
       // think the container is still around and assign a task to it. The task
       // ends up getting a CONTAINER_KILLED message. Task could handle this by
       // asking for a reschedule in this case. Will end up FAILING the task instead of KILLING it.
-      container.inError = true;
+      container.setError();
       AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
       String errorMessage = "AttemptId: " + event.getTaskAttemptId()
           + " cannot be allocated to container: " + container.getContainerId()
@@ -1058,9 +1077,19 @@ public class AMContainerImpl implements AMContainer {
     }
   }
 
+  private static class NonRunningStateEnteredCallback
+      implements OnStateChangedCallback<AMContainerState, AMContainerImpl> {
+
+    @Override
+    public void onStateChanged(AMContainerImpl amContainer,
+                               AMContainerState amContainerState) {
+      amContainer.handleNonRunningStateEntered();
+    }
+  }
+
   private void handleExtraTAAssign(
       AMContainerEventAssignTA event, TezTaskAttemptID currentTaId) {
-    this.inError = true;
+    setError();
     String errorMessage = "AMScheduler Error: Multiple simultaneous " +
         "taskAttempt allocations to: " + this.getContainerId() +
         ". Attempts: " + currentTaId + ", " + event.getTaskAttemptId() +
@@ -1078,6 +1107,19 @@ public class AMContainerImpl implements AMContainer {
     this.unregisterFromContainerListener();
   }
 
+  private void setError() {
+    this.inError = true;
+    handleNonRunningStateEntered();
+  }
+
+  private void handleNonRunningStateEntered() {
+    if (!completedMessageSent) {
+      completedMessageSent = true;
+      sendEvent(new AMNodeEventContainerCompleted(getContainer().getNodeId(),
+          schedulerId, containerId));
+    }
+  }
+
   protected void registerFailedAttempt(TezTaskAttemptID taId) {
     failedAssignments.add(taId);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNode.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNode.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNode.java
index 1c34816..bc01e04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNode.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNode.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.app.dag.DAG;
 
 public interface AMNode extends EventHandler<AMNodeEvent> {
   
@@ -33,4 +34,6 @@ public interface AMNode extends EventHandler<AMNodeEvent> {
   public boolean isUnhealthy();
   public boolean isBlacklisted();
   public boolean isUsable();
+
+  void dagComplete(DAG dag);
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerCompleted.java
new file mode 100644
index 0000000..f999c3a
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventContainerCompleted.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.dag.app.rm.node;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+
+public class AMNodeEventContainerCompleted extends AMNodeEvent {
+
+  private final ContainerId containerId;
+
+  public AMNodeEventContainerCompleted(
+      NodeId nodeId,
+      int schedulerId, ContainerId containerId) {
+    super(nodeId, schedulerId, AMNodeEventType.N_CONTAINER_COMPLETED);
+    this.containerId = containerId;
+  }
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventType.java
index 86087d0..a141124 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeEventType.java
@@ -21,7 +21,10 @@ package org.apache.tez.dag.app.rm.node;
 public enum AMNodeEventType {
   //Producer: Scheduler
   N_CONTAINER_ALLOCATED,
-  
+
+  //Producer: Container
+  N_CONTAINER_COMPLETED,
+
   //Producer: TaskSchedulerEventHandler
   N_TA_SUCCEEDED,
 

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
index bcc38c6..f4ad032 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java
@@ -19,6 +19,8 @@
 package org.apache.tez.dag.app.rm.node;
 
 import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
@@ -26,6 +28,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
+import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -60,20 +63,19 @@ public class AMNodeImpl implements AMNode {
   private boolean blacklistingEnabled;
   private boolean ignoreBlacklisting = false;
   private boolean nodeUpdatesRescheduleEnabled;
-  private Set<TezTaskAttemptID> failedAttemptIds = Sets.newHashSet();
+  private final Set<TezTaskAttemptID> failedAttemptIds = Sets.newHashSet();
 
   @SuppressWarnings("rawtypes")
   protected EventHandler eventHandler;
 
   @VisibleForTesting
-  final List<ContainerId> containers = new LinkedList<ContainerId>();
+  final Set<ContainerId> containers = new LinkedHashSet<>();
+  final Set<ContainerId> completedContainers = new HashSet<>();
   int numFailedTAs = 0;
   int numSuccessfulTAs = 0;
-  
-  //Book-keeping only. In case of Health status change.
-  private final List<ContainerId> pastContainers = new LinkedList<ContainerId>();
-
 
+  private static final ContainerCompletedTransition CONTAINER_COMPLETED_TRANSITION =
+      new ContainerCompletedTransition();
 
   private final StateMachine<AMNodeState, AMNodeEventType, AMNodeEvent> stateMachine;
 
@@ -103,6 +105,8 @@ public class AMNodeImpl implements AMNode {
           new IgnoreBlacklistingStateChangeTransition(true))
       .addTransition(AMNodeState.ACTIVE, AMNodeState.ACTIVE,
           AMNodeEventType.N_TURNED_HEALTHY)
+      .addTransition(AMNodeState.ACTIVE, AMNodeState.ACTIVE,
+          AMNodeEventType.N_CONTAINER_COMPLETED, CONTAINER_COMPLETED_TRANSITION)
 
       // Transitions from BLACKLISTED state.
       .addTransition(AMNodeState.BLACKLISTED, AMNodeState.BLACKLISTED,
@@ -120,6 +124,8 @@ public class AMNodeImpl implements AMNode {
       .addTransition(AMNodeState.BLACKLISTED, AMNodeState.FORCED_ACTIVE,
           AMNodeEventType.N_IGNORE_BLACKLISTING_ENABLED,
           new IgnoreBlacklistingStateChangeTransition(true))
+      .addTransition(AMNodeState.BLACKLISTED, AMNodeState.BLACKLISTED,
+          AMNodeEventType.N_CONTAINER_COMPLETED, CONTAINER_COMPLETED_TRANSITION)
       .addTransition(
           AMNodeState.BLACKLISTED,
           AMNodeState.BLACKLISTED,
@@ -142,6 +148,8 @@ public class AMNodeImpl implements AMNode {
           EnumSet.of(AMNodeState.BLACKLISTED, AMNodeState.ACTIVE),
           AMNodeEventType.N_IGNORE_BLACKLISTING_DISABLED,
           new IgnoreBlacklistingDisabledTransition())
+      .addTransition(AMNodeState.FORCED_ACTIVE, AMNodeState.FORCED_ACTIVE,
+          AMNodeEventType.N_CONTAINER_COMPLETED, CONTAINER_COMPLETED_TRANSITION)
       .addTransition(
           AMNodeState.FORCED_ACTIVE,
           AMNodeState.FORCED_ACTIVE,
@@ -168,6 +176,8 @@ public class AMNodeImpl implements AMNode {
           EnumSet.of(AMNodeState.ACTIVE, AMNodeState.FORCED_ACTIVE),
           AMNodeEventType.N_TURNED_HEALTHY, new NodeTurnedHealthyTransition())
       .addTransition(AMNodeState.UNHEALTHY, AMNodeState.UNHEALTHY,
+          AMNodeEventType.N_CONTAINER_COMPLETED, CONTAINER_COMPLETED_TRANSITION)
+      .addTransition(AMNodeState.UNHEALTHY, AMNodeState.UNHEALTHY,
           AMNodeEventType.N_TURNED_UNHEALTHY, new GenericErrorTransition())
 
         .installTopology();
@@ -259,7 +269,6 @@ public class AMNodeImpl implements AMNode {
       sendEvent(new AMContainerEventNodeFailed(c, "Node blacklisted"));
     }
     // these containers are not useful anymore
-    pastContainers.addAll(containers);
     containers.clear();
     sendEvent(new AMSchedulerEventNodeBlacklistUpdate(getNodeId(), true, schedulerId));
   }
@@ -295,9 +304,9 @@ public class AMNodeImpl implements AMNode {
     @Override
     public AMNodeState transition(AMNodeImpl node, AMNodeEvent nEvent) {
       AMNodeEventTaskAttemptEnded event = (AMNodeEventTaskAttemptEnded) nEvent;
-      LOG.info("Attempt failed on node: " + node.getNodeId() + " TA: "
-          + event.getTaskAttemptId() + " failed: " + event.failed()
-          + " container: " + event.getContainerId() + " numFailedTAs: "
+      LOG.info("Attempt " + (event.failed() ? "failed" : "killed") + "on node: " + node.getNodeId()
+          + " TA: " + event.getTaskAttemptId()
+          + ", container: " + event.getContainerId() + ", numFailedTAs: "
           + node.numFailedTAs);
       if (event.failed()) {
         // ignore duplicate attempt ids
@@ -381,8 +390,6 @@ public class AMNodeImpl implements AMNode {
       AMNodeEventContainerAllocated event = (AMNodeEventContainerAllocated) nEvent;
       node.sendEvent(new AMContainerEvent(event.getContainerId(),
           AMContainerEventType.C_STOP_REQUEST));
-      // ZZZ CReuse: Should the scheduler check node state before scheduling a
-      // container on it ?
     }
   }
 
@@ -434,7 +441,6 @@ public class AMNodeImpl implements AMNode {
       MultipleArcTransition<AMNodeImpl, AMNodeEvent, AMNodeState> {
     @Override
     public AMNodeState transition(AMNodeImpl node, AMNodeEvent nEvent) {
-      node.pastContainers.addAll(node.containers);
       node.containers.clear();
       if (node.ignoreBlacklisting) {
         return AMNodeState.FORCED_ACTIVE;
@@ -444,6 +450,17 @@ public class AMNodeImpl implements AMNode {
     }
   }
 
+  protected static class ContainerCompletedTransition
+      implements SingleArcTransition<AMNodeImpl, AMNodeEvent> {
+
+    @Override
+    public void transition(AMNodeImpl amNode, AMNodeEvent amNodeEvent) {
+      AMNodeEventContainerCompleted cc =
+          (AMNodeEventContainerCompleted) amNodeEvent;
+      amNode.completedContainers.add(cc.getContainerId());
+    }
+  }
+
   @Override
   public boolean isUnhealthy() {
     this.readLock.lock();
@@ -468,4 +485,28 @@ public class AMNodeImpl implements AMNode {
   public boolean isUsable() {
     return !(isUnhealthy() || isBlacklisted());
   }
+
+  @Override
+  public void dagComplete(DAG dag) {
+    this.writeLock.lock();
+    try {
+      int countBefore = containers.size();
+      int countCompleted = completedContainers.size();
+
+
+      // Actual functionality.
+      containers.removeAll(completedContainers);
+      completedContainers.clear();
+
+      int countAfter = containers.size();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Node {}, cleaning up knownContainers. current={}, completed={}, postCleanup={}",
+            getNodeId(), countBefore, countCompleted, countAfter);
+      }
+
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
index fdc8a4c..1536170 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java
@@ -101,6 +101,7 @@ public class AMNodeTracker extends AbstractService implements
     // No synchronization required until there's multiple dispatchers.
     switch (rEvent.getType()) {
       case N_CONTAINER_ALLOCATED:
+      case N_CONTAINER_COMPLETED:
       case N_TA_SUCCEEDED:
       case N_TA_ENDED:
       case N_IGNORE_BLACKLISTING_ENABLED:
@@ -140,7 +141,9 @@ public class AMNodeTracker extends AbstractService implements
   }
 
   public void dagComplete(DAG dag) {
-    // TODO TEZ-2337 Maybe reset failures from previous DAGs
+    for (PerSourceNodeTracker perSourceNodeTracker : perSourceNodeTrackers.values()) {
+      perSourceNodeTracker.dagComplete(dag);
+    }
   }
 
   private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int schedulerId) {

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
index 72c3230..74c6176 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -106,7 +107,8 @@ public class PerSourceNodeTracker {
         }
         break;
       default:
-        nodeMap.get(nodeId).handle(rEvent);
+        amNode = nodeMap.get(nodeId);
+        amNode.handle(rEvent);
     }
   }
 
@@ -186,6 +188,13 @@ public class PerSourceNodeTracker {
     }
   }
 
+  public void dagComplete(DAG dag) {
+    for (AMNode amNode : nodeMap.values()) {
+      amNode.dagComplete(dag);
+    }
+    // TODO TEZ-2337 Maybe reset failures from previous DAGs
+  }
+
   @SuppressWarnings("unchecked")
   private void sendEvent(Event<?> event) {
     this.eventHandler.handle(event);

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index 4d1bbae..1b9df99 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.security.JobTokenIdentifier;
 import org.apache.tez.common.security.TokenCache;
 import org.apache.tez.dag.app.TaskCommunicatorWrapper;
+import org.apache.tez.dag.app.rm.node.AMNodeEventType;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.serviceplugins.api.ServicePluginException;
 import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
@@ -146,7 +147,9 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
-    wc.verifyNoOutgoingEvents();
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -196,7 +199,9 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
-    wc.verifyNoOutgoingEvents();
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -266,7 +271,9 @@ public class TestAMContainer {
     wc.containerCompleted();
     wc.verifyHistoryStopEvent();
     wc.verifyState(AMContainerState.COMPLETED);
-    wc.verifyNoOutgoingEvents();
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -288,9 +295,11 @@ public class TestAMContainer {
     wc.stopRequest();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
     // Event to NM to stop the container.
-    wc.verifyCountAndGetOutgoingEvents(1);
-    assertTrue(wc.verifyCountAndGetOutgoingEvents(1).get(0).getType() ==
-        ContainerLauncherEventType.CONTAINER_STOP_REQUEST);
+
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     wc.nmStopSent();
     wc.verifyState(AMContainerState.STOPPING);
@@ -323,9 +332,10 @@ public class TestAMContainer {
     wc.stopRequest();
     wc.verifyState(AMContainerState.STOP_REQUESTED);
     // Event to NM to stop the container.
-    wc.verifyCountAndGetOutgoingEvents(1);
-    assertTrue(wc.verifyCountAndGetOutgoingEvents(1).get(0).getType() ==
-        ContainerLauncherEventType.CONTAINER_STOP_REQUEST);
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     wc.nmStopFailed();
     wc.verifyState(AMContainerState.STOPPING);
@@ -366,11 +376,12 @@ public class TestAMContainer {
         "Multiple simultaneous taskAttempt");
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(4);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATING);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATING,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     assertTrue(wc.amContainer.isInErrorState());
 
     wc.nmStopSent();
@@ -405,11 +416,12 @@ public class TestAMContainer {
         "Multiple simultaneous taskAttempt");
     verify(wc.chh).unregister(wc.containerID);
     // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(4);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATING);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATING,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     assertTrue(wc.amContainer.isInErrorState());
 
     wc.nmStopSent();
@@ -442,10 +454,11 @@ public class TestAMContainer {
         "timed out");
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        ContainerLauncherEventType.CONTAINER_STOP_REQUEST);
+        ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     // TODO Should this be an RM DE-ALLOCATE instead ?
 
     wc.containerCompleted();
@@ -477,10 +490,11 @@ public class TestAMContainer {
         "received a STOP_REQUEST");
     verify(wc.chh).unregister(wc.containerID);
     // 1 to TA, 1 for RM de-allocate.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        ContainerLauncherEventType.CONTAINER_STOP_REQUEST);
+        ContainerLauncherEventType.CONTAINER_STOP_REQUEST,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     // TODO Should this be an RM DE-ALLOCATE instead ?
 
     wc.containerCompleted();
@@ -511,10 +525,11 @@ public class TestAMContainer {
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.LAUNCH_FAILED,
         "launchFailed");
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
+        AMSchedulerEventType.S_CONTAINER_DEALLOCATE,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     for (Event e : outgoingEvents) {
       if (e.getType() == TaskAttemptEventType.TA_CONTAINER_TERMINATING) {
         Assert.assertEquals(TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,
@@ -538,7 +553,9 @@ public class TestAMContainer {
 
     wc.containerCompleted();
     wc.verifyState(AMContainerState.COMPLETED);
-    wc.verifyNoOutgoingEvents();
+    List<Event> outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     assertFalse(wc.amContainer.isInErrorState());
   }
@@ -561,9 +578,10 @@ public class TestAMContainer {
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.COMPLETED, null);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     Assert.assertEquals(TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,
         ((TaskAttemptEventContainerTerminated)outgoingEvents.get(0)).getTerminationCause());
 
@@ -591,9 +609,10 @@ public class TestAMContainer {
     verify(wc.tal).registerRunningContainer(wc.containerID, 0);
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.OTHER, "DiskFailed");
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     Assert.assertEquals(TaskAttemptTerminationCause.NODE_DISK_ERROR,
         ((TaskAttemptEventContainerTerminatedBySystem)outgoingEvents.get(0)).getTerminationCause());
 
@@ -623,9 +642,10 @@ public class TestAMContainer {
     verifyUnregisterRunningContainer(wc.tal, wc.containerID, 0, ContainerEndReason.NODE_FAILED,
         "NodeFailed");
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     Assert.assertEquals(TaskAttemptTerminationCause.NODE_FAILED,
         ((TaskAttemptEventContainerTerminated)outgoingEvents.get(0)).getTerminationCause());
 
@@ -656,11 +676,12 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    wc.verifyCountAndGetOutgoingEvents(0);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     assertFalse(wc.amContainer.isInErrorState());
 
-    wc.verifyNoOutgoingEvents();
     wc.verifyHistoryStopEvent();
 
     assertFalse(wc.amContainer.isInErrorState());
@@ -685,9 +706,10 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     assertFalse(wc.amContainer.isInErrorState());
 
@@ -722,11 +744,14 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+
+    Event event = findEventByType(outgoingEvents, TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
     Assert.assertEquals(TaskAttemptTerminationCause.EXTERNAL_PREEMPTION,
-        ((TaskAttemptEventContainerTerminatedBySystem)outgoingEvents.get(0)).getTerminationCause());
+        ((TaskAttemptEventContainerTerminatedBySystem)event).getTerminationCause());
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     assertFalse(wc.amContainer.isInErrorState());
 
@@ -761,9 +786,10 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
     Assert.assertEquals(TaskAttemptTerminationCause.INTERNAL_PREEMPTION,
         ((TaskAttemptEventContainerTerminatedBySystem)outgoingEvents.get(0)).getTerminationCause());
 
@@ -799,11 +825,13 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    Event event = findEventByType(outgoingEvents, TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
     Assert.assertEquals(TaskAttemptTerminationCause.NODE_DISK_ERROR,
-        ((TaskAttemptEventContainerTerminatedBySystem)outgoingEvents.get(0)).getTerminationCause());
+        ((TaskAttemptEventContainerTerminatedBySystem)event).getTerminationCause());
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     assertFalse(wc.amContainer.isInErrorState());
 
@@ -862,11 +890,12 @@ public class TestAMContainer {
     wc.nodeFailed();
     // Expecting a complete event from the RM
     wc.verifyState(AMContainerState.STOPPING);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(4);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_NODE_FAILED,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
+        AMSchedulerEventType.S_CONTAINER_DEALLOCATE,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     for (Event event : outgoingEvents) {
       if (event.getType() == TaskAttemptEventType.TA_NODE_FAILED) {
@@ -904,11 +933,12 @@ public class TestAMContainer {
     wc.nodeFailed();
     // Expecting a complete event from the RM
     wc.verifyState(AMContainerState.STOPPING);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(4);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_NODE_FAILED,
         TaskAttemptEventType.TA_NODE_FAILED,
-        AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
+        AMSchedulerEventType.S_CONTAINER_DEALLOCATE,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     for (Event event : outgoingEvents) {
       if (event.getType() == TaskAttemptEventType.TA_NODE_FAILED) {
@@ -945,12 +975,13 @@ public class TestAMContainer {
     wc.nodeFailed();
     // Expecting a complete event from the RM
     wc.verifyState(AMContainerState.STOPPING);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(4);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(5);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_NODE_FAILED,
         TaskAttemptEventType.TA_NODE_FAILED,
         TaskAttemptEventType.TA_CONTAINER_TERMINATING,
-        AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
+        AMSchedulerEventType.S_CONTAINER_DEALLOCATE,
+        AMNodeEventType.N_CONTAINER_COMPLETED);
 
     for (Event event : outgoingEvents) {
       if (event.getType() == TaskAttemptEventType.TA_NODE_FAILED) {
@@ -1439,6 +1470,15 @@ public class TestAMContainer {
     assertTrue("Found unexpected events: " + eventsCopy
         + " in outgoing event list", eventsCopy.isEmpty());
   }
+
+  private Event findEventByType(List<Event> events, Enum<?> type) {
+    for (Event event : events) {
+      if (event.getType() == type) {
+        return event;
+      }
+    }
+    return null;
+  }
   
   private LocalResource createLocalResource(String name) {
     LocalResource lr = LocalResource.newInstance(URL.newInstance(null, "localhost", 2321, name),

http://git-wip-us.apache.org/repos/asf/tez/blob/4ce6ea6e/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
index e123dd1..11d3b7a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/node/TestAMNodeTracker.java
@@ -29,6 +29,7 @@ import static org.mockito.Mockito.mock;
 
 import java.util.List;
 
+import org.apache.tez.dag.app.dag.DAG;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -326,6 +327,78 @@ public class TestAMNodeTracker {
     }
   }
 
+  @Test(timeout = 10000L)
+  public void testNodeCompletedAndCleanup() {
+    AppContext appContext = mock(AppContext.class);
+    Configuration conf = new Configuration(false);
+    conf.setInt(TezConfiguration.TEZ_AM_MAX_TASK_FAILURES_PER_NODE, 2);
+    TestEventHandler handler = new TestEventHandler();
+    AMNodeTracker amNodeTracker = new AMNodeTracker(handler, appContext);
+    doReturn(amNodeTracker).when(appContext).getNodeTracker();
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    TaskSchedulerManager taskSchedulerManager =
+        mock(TaskSchedulerManager.class);
+    dispatcher.register(AMNodeEventType.class, amNodeTracker);
+    dispatcher.register(AMContainerEventType.class, amContainerMap);
+    dispatcher.register(AMSchedulerEventType.class, taskSchedulerManager);
+    amNodeTracker.init(conf);
+    amNodeTracker.start();
+
+    try {
+
+      NodeId nodeId = NodeId.newInstance("fakenode", 3333);
+      amNodeTracker.nodeSeen(nodeId, 0);
+
+      AMNode amNode = amNodeTracker.get(nodeId, 0);
+      ContainerId[] containerIds = new ContainerId[7];
+
+      // Start 5 containers.
+      for (int i = 0; i < 5; i++) {
+        containerIds[i] = mock(ContainerId.class);
+        amNodeTracker
+            .handle(new AMNodeEventContainerAllocated(nodeId, 0, containerIds[i]));
+      }
+      assertEquals(5, amNode.getContainers().size());
+
+      // Finnish 1st dag
+      amNodeTracker.dagComplete(mock(DAG.class));
+      assertEquals(5, amNode.getContainers().size());
+
+
+      // Mark 2 as complete. Finish 2nd dag.
+      for (int i = 0; i < 2; i++) {
+        amNodeTracker.handle(
+            new AMNodeEventContainerCompleted(nodeId, 0, containerIds[i]));
+      }
+      amNodeTracker.dagComplete(mock(DAG.class));
+      assertEquals(3, amNode.getContainers().size());
+
+      // Add 2 more containers. Mark all as complete. Finish 3rd dag.
+      for (int i = 5; i < 7; i++) {
+        containerIds[i] = mock(ContainerId.class);
+        amNodeTracker
+            .handle(new AMNodeEventContainerAllocated(nodeId, 0, containerIds[i]));
+      }
+      assertEquals(5, amNode.getContainers().size());
+      amNodeTracker.dagComplete(mock(DAG.class));
+      assertEquals(5, amNode.getContainers().size());
+      amNodeTracker.dagComplete(mock(DAG.class));
+      assertEquals(5, amNode.getContainers().size());
+
+      for (int i = 2; i < 7; i++) {
+        amNodeTracker.handle(
+            new AMNodeEventContainerCompleted(nodeId, 0, containerIds[i]));
+      }
+      assertEquals(5, amNode.getContainers().size());
+      amNodeTracker.dagComplete(mock(DAG.class));
+      assertEquals(0, amNode.getContainers().size());
+
+    } finally {
+      amNodeTracker.stop();
+    }
+
+  }
+
   @Test(timeout=10000)
   public void testNodeUnhealthyRescheduleTasksEnabled() throws Exception {
     _testNodeUnhealthyRescheduleTasks(true);


[43/50] [abbrv] tez git commit: TEZ-3659. AM/Task classpath should not contain hadoop conf directory. (harishjp)

Posted by zh...@apache.org.
TEZ-3659. AM/Task classpath should not contain hadoop conf directory. (harishjp)


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

Branch: refs/heads/TEZ-1190
Commit: 293c5933f27be96091320f4a5deb80815db33cfd
Parents: a339d05
Author: Harish JP <ha...@gmail.com>
Authored: Fri Mar 17 08:06:41 2017 +0530
Committer: Harish JP <ha...@gmail.com>
Committed: Fri Mar 17 08:07:55 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/tez/common/TezYARNUtils.java     |  7 +++----
 .../org/apache/tez/dag/api/TezConfiguration.java     | 13 +++++++++++++
 .../java/org/apache/tez/common/TestTezYARNUtils.java | 15 +++++++++++++++
 3 files changed, 31 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/293c5933/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java
index d5da672..bd6de11 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezYARNUtils.java
@@ -18,7 +18,6 @@
 package org.apache.tez.common;
 
 import java.io.File;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -103,10 +102,10 @@ public class TezYARNUtils {
         classpathBuilder.append(c.trim())
             .append(File.pathSeparator);
       }
-    } else {
+    } else if (conf.getBoolean(TezConfiguration.TEZ_CLASSPATH_ADD_HADOOP_CONF,
+        TezConfiguration.TEZ_CLASSPATH_ADD_HADOOP_CONF_DEFAULT)) {
       // Setup HADOOP_CONF_DIR after PWD and tez-libs, if it's required.
-      classpathBuilder.append(Environment.HADOOP_CONF_DIR.$())
-          .append(File.pathSeparator);
+      classpathBuilder.append(Environment.HADOOP_CONF_DIR.$()).append(File.pathSeparator);
     }
 
     if (!userClassesTakesPrecedence) {

http://git-wip-us.apache.org/repos/asf/tez/blob/293c5933/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 5900f5e..40f84e6 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1158,6 +1158,19 @@ public class TezConfiguration extends Configuration {
       TEZ_PREFIX + "cluster.additional.classpath.prefix";
 
   /**
+   * Boolean value.
+   * If this value is true then tez explicitly adds hadoop conf directory into classpath for AM and
+   * task containers. Default is false.
+   */
+  @Private
+  @Unstable
+  @ConfigurationScope(Scope.CLIENT)
+  @ConfigurationProperty(type="boolean")
+  public static final String TEZ_CLASSPATH_ADD_HADOOP_CONF = TEZ_PREFIX +
+      "classpath.add-hadoop-conf";
+  public static final boolean TEZ_CLASSPATH_ADD_HADOOP_CONF_DEFAULT = false;
+
+  /**
    * Session-related properties
    */
   @Private

http://git-wip-us.apache.org/repos/asf/tez/blob/293c5933/tez-api/src/test/java/org/apache/tez/common/TestTezYARNUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezYARNUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezYARNUtils.java
index 9f02801..4fdd61b 100644
--- a/tez-api/src/test/java/org/apache/tez/common/TestTezYARNUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/common/TestTezYARNUtils.java
@@ -70,6 +70,21 @@ public class TestTezYARNUtils {
     Assert.assertTrue(classpath.contains(TezConstants.TEZ_TAR_LR_NAME + File.separator + "*"));
     Assert.assertTrue(classpath.contains(TezConstants.TEZ_TAR_LR_NAME + File.separator
         + "lib" + File.separator + "*"));
+    Assert.assertTrue(!classpath.contains(Environment.HADOOP_CONF_DIR.$()));
+    Assert.assertTrue(classpath.indexOf(Environment.PWD.$()) <
+        classpath.indexOf(TezConstants.TEZ_TAR_LR_NAME));
+  }
+
+  @Test(timeout = 5000)
+  public void testNoHadoopConfInClasspath() {
+    Configuration conf = new Configuration(false);
+    conf.setBoolean(TezConfiguration.TEZ_CLASSPATH_ADD_HADOOP_CONF, true);
+    String classpath = TezYARNUtils.getFrameworkClasspath(conf, true);
+    Assert.assertTrue(classpath.contains(Environment.PWD.$()));
+    Assert.assertTrue(classpath.contains(Environment.PWD.$() + File.separator + "*"));
+    Assert.assertTrue(classpath.contains(TezConstants.TEZ_TAR_LR_NAME + File.separator + "*"));
+    Assert.assertTrue(classpath.contains(TezConstants.TEZ_TAR_LR_NAME + File.separator
+        + "lib" + File.separator + "*"));
     Assert.assertTrue(classpath.contains(Environment.HADOOP_CONF_DIR.$()));
     Assert.assertTrue(classpath.indexOf(Environment.PWD.$()) <
         classpath.indexOf(TezConstants.TEZ_TAR_LR_NAME));


[32/50] [abbrv] tez git commit: TEZ-3651. Add section for 0.8.6

Posted by zh...@apache.org.
TEZ-3651. Add section for 0.8.6


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

Branch: refs/heads/TEZ-1190
Commit: c6d4908ebf7ab80644e4519f25f180c1a63e2e3f
Parents: d40f3ad
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 7 11:35:03 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Mar 7 11:35:03 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c6d4908e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 248f329..0065a83 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -217,7 +217,14 @@ ALL CHANGES:
   TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
   TEZ-3503. Tez UI: Support search by queue name
 
-Release 0.8.5: Unreleased
+Release 0.8.6: Unreleased
+
+INCOMPATIBLE CHANGES
+
+ALL CHANGES:
+
+
+Release 0.8.5: 2016-03-13
 
 INCOMPATIBLE CHANGES
 


[38/50] [abbrv] tez git commit: TEZ-3660. Remove CHANGES.txt. (sseth)

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/5f953bfd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
deleted file mode 100644
index df0caa1..0000000
--- a/CHANGES.txt
+++ /dev/null
@@ -1,2294 +0,0 @@
-Apache Tez Change Log
-=====================
-
-Release 0.9.0: Unreleased
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-
-  TEZ-3650. Improve performance of FetchStatsLogger#logIndividualFetchComplete
-  TEZ-3655. Specify netty version instead of inheriting from hadoop dependency.
-  TEZ-3253. Remove special handling for last app attempt.
-  TEZ-3648. IFile.Write#close has an extra output stream flush
-  TEZ-3649. AsyncHttpConnection should add StopWatch start.
-  TEZ-3647. Add a setting which lets Tez determine Xmx.
-  TEZ-3644. Cleanup container list stored in AMNode.
-  TEZ-3646. IFile.Writer has an extra output stream flush call
-  TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances.
-  TEZ-3637. TezMerger logs too much at INFO level
-  TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism
-  TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.
-  TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.
-  TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes
-  TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
-  TEZ-3550. Provide access to sessionId/dagId via DagClient.
-  TEZ-3267. Publish queue name to ATS as part of dag summary.
-  TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
-  TEZ-3244. Allow overlap of input and output memory when they are not concurrent
-  TEZ-3581. Add different logger to enable suppressing logs for specific lines.
-  TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
-  TEZ-3600. Fix flaky test: TestTokenCache
-  TEZ-3589. add a unit test for amKeepAlive not being shutdown if an app takes a long time to launch.
-  TEZ-3417. Reduce sleep time on AM shutdown to reduce test runtimes
-  TEZ-3582. Exception swallowed in PipelinedSorter causing incorrect results.
-  TEZ-3584. amKeepAliveService in TezClient should shutdown in case of AM failure.
-  TEZ-3462. Task attempt failure during container shutdown loses useful container diagnostics
-  TEZ-3579. Wrong configuration key for max slow start fraction in CartesianProductVertexManager.
-  TEZ-3458. Auto grouping for cartesian product edge(unpartitioned case).
-  TEZ-3574. Container reuse won't pickup extra dag level local resource.
-  TEZ-3443. Remove a repeated/unused method from MRTask.
-  TEZ-3551. FrameworkClient created twice causing minor delay.
-  TEZ-3566. Avoid caching fs isntances in TokenCache after a point.
-  TEZ-3568. Update SecurityUtils configuration to pick user provided configuration.
-  TEZ-3561. Fix wrong tez tarball name in install.md.
-  TEZ-3565. amConfig should check queuename isEmpty.
-  TEZ-3559. TEZ_LIB_URIS doesn't work with schemes different than the defaultFS
-  TEZ-3558. CartesianProduct is missing from the ExampleDriver class
-  TEZ-3549. TaskAttemptImpl does not initialize TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS correctly
-  TEZ-3552. Shuffle split array when size-based sorting is turned off.
-  TEZ-3537. ArrayIndexOutOfBoundsException with empty environment variables/Port YARN-3768 to Tez
-  TEZ-3271. Provide mapreduce failures.maxpercent equivalent.
-  TEZ-3222. Reduce messaging overhead for auto-reduce parallelism case.
-  TEZ-3547. Add TaskAssignment Analyzer.
-  TEZ-3508. TestTaskScheduler cleanup.
-  TEZ-3536. NPE in WebUIService start when host resolution fails.
-  TEZ-3269. Provide basic fair routing and scheduling functionality via custom VertexManager and EdgeManager.
-  TEZ-3534. Differentiate thread names on Fetchers, minor changes to shuffle shutdown code.
-  TEZ-3491. Tez job can hang due to container priority inversion.
-  TEZ-3533. ShuffleScheduler should shutdown threadpool on exit.
-  TEZ-3477. MRInputHelpers generateInputSplitsToMem public API modified
-  TEZ-3465. Support broadcast edge into cartesian product vertex and forbid other edges.
-  TEZ-3493. DAG submit timeout cannot be set to a month
-  TEZ-3505. Move license to the file header for TezBytesWritableSerialization
-  TEZ-3486. COMBINE_OUTPUT_RECORDS/COMBINE_INPUT_RECORDS are not correct
-  TEZ-3247. Add more unit test coverage for container reuse.
-  TEZ-3215. Support for MultipleOutputs.
-  TEZ-3097. Flaky test: TestCommit.testDAGCommitStartedEventFail_OnDAGSuccess.
-  TEZ-3487. Improvements in travis yml file to get builds to work.
-  TEZ-3405. Support ability for AM to kill itself if there is no client heartbeating to it.
-  TEZ-3483. Create basic travis yml file for Tez.
-  TEZ-3452. Auto-reduce parallelism calculation can overflow with large inputs
-  TEZ-3439. Tez joinvalidate fails when first input argument size is bigger than the second.
-  TEZ-3430. Make split sorting optional.
-  TEZ-3466. Tez classpath building to mimic mapreduce classpath building.
-  TEZ-3453. Correct the downloaded ATS dag data location for analyzer.
-  TEZ-3449. Fix Spelling typos.
-  TEZ-3464. Fix findbugs warnings in tez-dag mainLoop
-  TEZ-3330. Propagate additional config parameters when running MR jobs via Tez.
-  TEZ-3335. DAG client thinks app is still running when app status is null
-  TEZ-3437. Improve synchronization and the progress report behavior for Inputs from TEZ-3317.
-  TEZ-3460. Fix precommit release audit warning.
-  TEZ-3368. NPE in DelayedContainerManager
-  TEZ-3440. Shuffling to memory can get out-of-sync when fetching multiple compressed map outputs
-  TEZ-3429. Set reconfigureDoneTime on VertexConfigurationDoneEvent properly.
-  TEZ-3000. Fix TestContainerReuse.
-  TEZ-3436. Check input and output count before start in MapProcessor.
-  TEZ-3163. Reuse and tune Inflaters and Deflaters to speed DME processing
-  TEZ-3434. Add unit tests for flushing of recovery events.
-  TEZ-3317. Speculative execution starts too early due to 0 progress.
-  TEZ-3404. Move blocking call for YARN Timeline domain creation from client side to AM.
-  TEZ-3272. Add AMContainerImpl and AMNodeImpl to StateMachine visualization list.
-  TEZ-3284. Synchronization for every write in UnOrderedKVWriter
-  TEZ-3426. Second AM attempt launched for session mode and recovery disabled for certain cases
-  TEZ-3230. Implement vertex manager and edge manager of cartesian product edge.
-  TEZ-3326. Display JVM system properties in AM and task logs.
-  TEZ-3009. Errors that occur during container task acquisition are not logged.
-  TEZ-2852. TestVertexImpl fails due to race in AsyncDispatcher.
-  TEZ-3395. Refactor ShuffleVertexManager to make parts of it re-usable in other plugins.
-  TEZ-3413. ConcurrentModificationException in HistoryEventTimelineConversion for AppLaunchedEvent.
-  TEZ-3352. MRInputHelpers getStringProperty() should not fail if property value is null.
-  TEZ-3409. Log dagId along with other information when submitting a dag.
-  TEZ-3384. Fix TestATSV15HistoryLoggingService::testDAGGroupingGroupingEnabled unit test.
-  TEZ-3382. Tez analyzer: Should be resilient to new counters.
-  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.
-  TEZ-3358. Support using the same TimelineGroupId for multiple DAGs.
-  TEZ-3357. Change TimelineCachePlugin to handle DAG grouping.
-  TEZ-3348. NullPointerException in Tez MROutput while trying to write using Parquet's DeprecatedParquetOutputFormat.
-  TEZ-3356. Fix initializing of stats when custom ShuffleVertexManager is used.
-  TEZ-3235. Modify Example TestOrderedWordCount job to test the IPC limit for large dag plans.
-  TEZ-3337. Do not log empty fields of TaskAttemptFinishedEvent to avoid confusion.
-  TEZ-3303. Have ShuffleVertexManager consume more precise partition stats.
-  TEZ-1248. Reduce slow-start should special case 1 reducer runs.
-  TEZ-3327. ATS Parser: Populate config details available in dag.
-  TEZ-3325. Flaky test in TestDAGImpl.testCounterLimits.
-  TEZ-3323. Update license and notice for xml-apis jar. Also update year in notice to 2016.
-  TEZ-3223. Support a NullHistoryLogger to disable history logging if needed.
-  TEZ-3313. ATSFileParser : Wrong args passed in VersionInfo.
-  TEZ-3286. Allow clients to set processor reserved memory per vertex (instead of per container).
-  TEZ-3293. Fetch failures can cause a shuffle hang waiting for memory merge that never starts.
-  TEZ-3314. Double counting input bytes in MultiMRInput.
-  TEZ-3308. Add counters to capture input split length.
-  TEZ-3302. Add a version of processorContext.waitForAllInputsReady and waitForAnyInputReady with a timeout.
-  TEZ-3291. Optimize splits grouping when locality information is not available.
-  TEZ-3305. TestAnalyzer fails on Hadoop 2.7.
-  TEZ-3304. TestHistoryParser fails with Hadoop 2.7.
-  TEZ-3216. Add support for more precise partition stats in VertexManagerEvent.
-  TEZ-3296. Tez job can hang if two vertices at the same root distance have different task requirements
-  TEZ-3294. DAG.createDag() does not clear local state on repeat calls.
-  TEZ-3297. Deadlock scenario in AM during ShuffleVertexManager auto reduce.
-  TEZ-3296. Tez fails to compile against hadoop 2.8 after MAPREDUCE-5870
-  TEZ-3295. TestOrderedWordCount should handle relative input/output paths.
-  TEZ-3290. Set full task attempt id string in MRInput configuration object.
-  TEZ-2846. Flaky test: TestCommit.testVertexCommit_OnDAGSuccess.
-  TEZ-3289. Tez Example MRRSleep job does not set Staging dir correctly on secure cluster.
-  TEZ-3276. Tez Example MRRSleep job fails when tez.staging-dir fs is not same as default FS.
-  TEZ-3280. LOG MRInputHelpers split generation message as INFO
-  TEZ-909.  Provide support for application tags
-  TEZ-3257. Fix flaky test TestUnorderedPartitionedKVWriter.
-  TEZ-3206. Have unordered partitioned KV output send partition stats via VertexManagerEvent.
-  TEZ-3237. Corrupted shuffle transfers to disk are not detected during transfer
-  TEZ-3240. Improvements to tez.lib.uris to allow for multiple tarballs and mixing tarballs and jars.
-  TEZ-3246. Improve diagnostics when DAG killed by user
-  TEZ-3258. Jvm Checker does not ignore DisableExplicitGC when checking JVM GC options.
-  TEZ-3256. [Backport HADOOP-11032] Remove Guava Stopwatch dependency
-  TEZ-2342. Reduce bytearray copy with TezEvent Serialization and deserialization
-  TEZ-3251. Allow ability to add custom counters to TaskRunner2Callable.
-  TEZ-3250. TezTaskRunner2 should accept ExecutorService.
-  TEZ-3245. Data race between addKnowInput and clearAndGetOnePartition of InputHost.
-  TEZ-3193. Deadlock in AM during task commit request.
-  TEZ-3203. DAG hangs when one of the upstream vertices has zero tasks
-  TEZ-3207. Add support for fetching multiple partitions from the same source task to UnorderedKVInput.
-  TEZ-3232. Disable randomFailingInputs in testFaulttolerance to unblock other tests.
-  TEZ-3219. Allow service plugins to define log locations link for remotely run task attempts.
-  TEZ-3224. User payload is not initialized before creating vertex manager plugin.
-  TEZ-3226. Tez UI 2: All DAGs UX improvements.
-  TEZ-3077. TezClient.waitTillReady should support timeout.
-  TEZ-3202. Reduce the memory need for jobs with high number of segments
-  TEZ-3165. Allow Inputs/Outputs to be initialized serially, control processor initialization relative to Inputs/Outputs
-  TEZ-3214. Tez UI 2: Pagination in All DAGs
-  TEZ-3210. Tez UI 2: license should account for numeral, more-js, loader.js , etc
-  TEZ-3087. Tez UI 2: Add log links in task & attempt details page
-  TEZ-3146. Tez UI 2: CSS & JS assets in the UI must be minified
-  TEZ-3227. Tez UI: Replace UI1 with UI2
-  TEZ-3233. Tez UI: Have LLAP information reflect in Tez UI
-  TEZ-3086. Tez UI: Backward compatibility changes
-  TEZ-3254. Tez UI: Consider downloading Hive/Pig explain plans
-  TEZ-3255. Tez UI: Hide swimlane while displaying running DAGs from old versions of Tez
-  TEZ-3259. Tez UI: Build issue - File saver package is not working well with bower
-  TEZ-3262. Tez UI : zip.js is not having a bower friendly versioning system
-  TEZ-3063. Tez UI: Display Input, Output, Processor, Source and Sink configurations under a vertex
-  TEZ-3281. Tez UI: Swimlane improvements
-  TEZ-3264. Tez UI: UI discrepancies
-  TEZ-3292. Tez UI: UTs breaking with timezone change
-  TEZ-3288. Tez UI: Display more details in the error bar
-  TEZ-3318. Tez UI: Polling is not restarted after RM recovery
-  TEZ-3329. Tez ATS data is incomplete for a vertex which fails or gets killed before initialization
-  TEZ-3333. Tez UI: Handle cases where Vertex/Task/Task Attempt data is missing
-  TEZ-3370. Tez UI: Display the log link as N/A if the app does not provide a log link
-  TEZ-3398. Tez UI: Bread crumb link to Application from Application details dag/configuration tab is broken
-  TEZ-3433. Tez UI: Searching using wrong ID causes error in all DAGs page
-  TEZ-3428. Tez UI: First Tab not needed for few entries in DAG listings
-  TEZ-3469. Tez UI: Bump Phantom JS version to 2.1.1
-  TEZ-3419. Tez UI: Applications page shows error, for users with only DAG level ACL permission
-  TEZ-3484. Tez UI: Remove .travis.yml from webapp folder
-  TEZ-3470. Tez UI: Make the build work in IBM PPC
-  TEZ-3502. Tez UI: Search in All DAGs page doesn't work with numeric values
-  TEZ-3546. Tez UI: On sorting asc - Not Available must be at the top
-  TEZ-3555. Tez UI: Build is failing in RHEL6
-  TEZ-3570. Tez UI: Wait for sometime before tooltips are displayed
-  TEZ-3504. Tez UI: Duration is displaying invalid values when start or end time is invalid
-  TEZ-3529. Tez UI: Add 'All Queries' table in the landing page along 'All DAGs' page
-  TEZ-3530. Tez UI: Add query details page, and link the page from All Queries table
-  TEZ-3531. Tez UI: All Queries table: Improve searchability
-  TEZ-3556. Tez UI: Display query configurations
-  TEZ-3496. Tez UI: Optimize display of all tasks table
-  TEZ-3571. Tez UI: Display a Total Timeline View for Hive Queries
-  TEZ-2712. Tez UI: Display the vertex description in the tooltip of vertex in DAG view UI
-  TEZ-3583. Tez UI: UTs are flaky because of a dependency issue
-  TEZ-3580. Tez UI: Pagination broken on queries page
-  TEZ-3575. RM have started forwarding origin. Use that in AMWebController for CORS support
-  TEZ-3494. Support relative url for tez-ui.history-url.base config
-  TEZ-3554. Add a link to get to all logs from Tez UI while job is running
-  TEZ-3591. Tez UI: Logs url in all DAGs doesn't open in a new window
-  TEZ-3592. Tez UI: Search issues
-  TEZ-3593. Tez UI: Issues in timeline page
-  TEZ-3594. Tez UI: Graphical view tooltip issues
-  TEZ-3598. Tez UI: Text formatting changes
-  TEZ-3602. Tez UI: Query Name field is not required
-  TEZ-3615. Tez UI: Table changes
-  TEZ-3619. Tez UI: Improve DAG Data download
-  TEZ-3629. Tez UI: Enable the UI to display log links from LLAP
-  TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly
-  TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling
-  TEZ-3639. Tez UI: Footer pagination is improper in landing page
-  TEZ-3640. Tez UI: Add associated llap application id to queries page
-  TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
-  TEZ-3503. Tez UI: Support search by queue name
-  TEZ-3656. Tez UI: Status correction is not working as expected
-  TEZ-3653. Tez UI: Swimlane tooltip is not proper for running DAGs
-
-Release 0.8.6: Unreleased
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-
-
-Release 0.8.5: 2016-03-13
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-
-  TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
-  TEZ-3550. Provide access to sessionId/dagId via DagClient.
-  TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
-  TEZ-3244. Allow overlap of input and output memory when they are not concurrent
-  TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
-  TEZ-3462. Task attempt failure during container shutdown loses useful container diagnostics
-  TEZ-3574. Container reuse won't pickup extra dag level local resource.
-  TEZ-3566. Avoid caching fs isntances in TokenCache after a point.
-  TEZ-3568. Update SecurityUtils configuration to pick user provided configuration.
-  TEZ-3559. TEZ_LIB_URIS doesn't work with schemes different than the defaultFS
-  TEZ-3549. TaskAttemptImpl does not initialize TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS correctly
-  TEZ-3537. ArrayIndexOutOfBoundsException with empty environment variables/Port YARN-3768 to Tez
-  TEZ-3536. NPE in WebUIService start when host resolution fails.
-  TEZ-3534. Differentiate thread names on Fetchers, minor changes to shuffle shutdown code.
-  TEZ-3491. Tez job can hang due to container priority inversion.
-  TEZ-3533. ShuffleScheduler should shutdown threadpool on exit.
-  TEZ-3493. DAG submit timeout cannot be set to a month
-  TEZ-3505. Move license to the file header for TezBytesWritableSerialization
-  TEZ-3486. COMBINE_OUTPUT_RECORDS/COMBINE_INPUT_RECORDS are not correct
-  TEZ-3097. Flaky test: TestCommit.testDAGCommitStartedEventFail_OnDAGSuccess.
-  TEZ-3437. Improve synchronization and the progress report behavior for Inputs from TEZ-3317.
-  TEZ-3317. Speculative execution starts too early due to 0 progress.
-  TEZ-3452. Auto-reduce parallelism calculation can overflow with large inputs
-  TEZ-3439. Tez joinvalidate fails when first input argument size is bigger than the second.
-  TEZ-3464. Fix findbugs warnings in tez-dag mainLoop
-  TEZ-3330. Propagate additional config parameters when running MR jobs via Tez.
-  TEZ-3335. DAG client thinks app is still running when app status is null
-  TEZ-3460. Fix precommit release audit warning.
-  TEZ-3368. NPE in DelayedContainerManager
-  TEZ-3440. Shuffling to memory can get out-of-sync when fetching multiple compressed map outputs
-  TEZ-3429. Set reconfigureDoneTime on VertexConfigurationDoneEvent properly.
-  TEZ-3000. Fix TestContainerReuse.
-  TEZ-3436. Check input and output count before start in MapProcessor.
-  TEZ-3426. Second AM attempt launched for session mode and recovery disabled for certain cases
-  TEZ-3326. Display JVM system properties in AM and task logs.
-  TEZ-3009. Errors that occur during container task acquisition are not logged.
-  TEZ-2852. TestVertexImpl fails due to race in AsyncDispatcher.
-  TEZ-3413. ConcurrentModificationException in HistoryEventTimelineConversion for AppLaunchedEvent.
-  TEZ-3352. MRInputHelpers getStringProperty() should not fail if property value is null.
-  TEZ-3409. Log dagId along with other information when submitting a dag.
-  TEZ-3384. Fix TestATSV15HistoryLoggingService::testDAGGroupingGroupingEnabled unit test.
-  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.
-  TEZ-3358. Support using the same TimelineGroupId for multiple DAGs.
-  TEZ-3357. Change TimelineCachePlugin to handle DAG grouping.
-  TEZ-3348. NullPointerException in Tez MROutput while trying to write using Parquet's DeprecatedParquetOutputFormat.
-  TEZ-3356. Fix initializing of stats when custom ShuffleVertexManager is used.
-  TEZ-3329. Tez ATS data is incomplete for a vertex which fails or gets killed before initialization
-  TEZ-3235. Modify Example TestOrderedWordCount job to test the IPC limit for large dag plans.
-  TEZ-3337. Do not log empty fields of TaskAttemptFinishedEvent to avoid confusion.
-  TEZ-1248. Reduce slow-start should special case 1 reducer runs.
-
-Release 0.8.4: 2016-07-08
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-
-  TEZ-3323. Update license and notice for xml-apis jar. Also update year in notice to 2016.
-  TEZ-3223. Support a NullHistoryLogger to disable history logging if needed.
-  TEZ-3286. Allow clients to set processor reserved memory per vertex (instead of per container).
-  TEZ-3293. Fetch failures can cause a shuffle hang waiting for memory merge that never starts.
-  TEZ-3314. Double counting input bytes in MultiMRInput.
-  TEZ-3308. Add counters to capture input split length.
-  TEZ-3302. Add a version of processorContext.waitForAllInputsReady and waitForAnyInputReady with a timeout.
-  TEZ-3291. Optimize splits grouping when locality information is not available.
-  TEZ-3305. TestAnalyzer fails on Hadoop 2.7.
-  TEZ-3304. TestHistoryParser fails with Hadoop 2.7.
-  TEZ-3296. Tez job can hang if two vertices at the same root distance have different task requirements
-  TEZ-3294. DAG.createDag() does not clear local state on repeat calls.
-  TEZ-3297. Deadlock scenario in AM during ShuffleVertexManager auto reduce.
-  TEZ-3296. Tez fails to compile against hadoop 2.8 after MAPREDUCE-5870
-  TEZ-3290. Set full task attempt id string in MRInput configuration object.
-  TEZ-3280. LOG MRInputHelpers split generation message as INFO
-  TEZ-3257. Fix flaky test TestUnorderedPartitionedKVWriter.
-  TEZ-3237. Corrupted shuffle transfers to disk are not detected during transfer
-  TEZ-3246. Improve diagnostics when DAG killed by user
-  TEZ-3258. Jvm Checker does not ignore DisableExplicitGC when checking JVM GC options.
-  TEZ-3256. [Backport HADOOP-11032] Remove Guava Stopwatch dependency
-  TEZ-2342. Reduce bytearray copy with TezEvent Serialization and deserialization
-  TEZ-3251. Allow ability to add custom counters to TaskRunner2Callable.
-  TEZ-3250. TezTaskRunner2 should accept ExecutorService.
-  TEZ-3193. Deadlock in AM during task commit request.
-  TEZ-3203. DAG hangs when one of the upstream vertices has zero tasks
-  TEZ-3219. Allow service plugins to define log locations link for remotely run task attempts.
-  TEZ-3224. User payload is not initialized before creating vertex manager plugin.
-  TEZ-3226. Tez UI 2: All DAGs UX improvements.
-  TEZ-3077. TezClient.waitTillReady should support timeout.
-  TEZ-3202. Reduce the memory need for jobs with high number of segments
-  TEZ-3165. Allow Inputs/Outputs to be initialized serially, control processor initialization relative to Inputs/Outputs
-  TEZ-3214. Tez UI 2: Pagination in All DAGs
-  TEZ-3210. Tez UI 2: license should account for numeral, more-js, loader.js , etc
-  TEZ-3087. Tez UI 2: Add log links in task & attempt details page
-  TEZ-3146. Tez UI 2: CSS & JS assets in the UI must be minified
-
-Release 0.8.3: 2016-04-14
-
-INCOMPATIBLE CHANGES
-  TEZ-3180. Update master docs to declare hadoop-2.6.x as a minimum requirement.
-  TEZ-3029. Add an onError method to service plugin contexts.
-  TEZ-3120. Remove TaskCommContext.getCurrentDagName, Identifier.
-  TEZ-3183. Change the taskFailed method on plugin contexts to specify the type of failure.
-  TEZ-3199. Rename getCredentials in TaskCommunicatorContext to be less confusing.
-
-ALL CHANGES:
-  TEZ-3188. Move tez.submit.hosts out of TezConfiguration to TezConfigurationConstants.
-  TEZ-3194. Tez UI: Swimlane improve in-progress experience.
-  TEZ-3196. java.lang.InternalError from decompression codec is fatal to a task during shuffle
-  TEZ-3161. Allow task to report different kinds of errors - fatal / kill.
-  TEZ-3177. Non-DAG events should use the session domain or no domain if the data does not need protection.
-  TEZ-3192. IFile#checkState creating unnecessary objects though auto-boxing
-  TEZ-3173. Update Tez AM REST APIs for more information for each vertex.
-  TEZ-3108. Add support for external services to local mode.
-  TEZ-3189. Pre-warm dags should not be counted in submitted dags count by DAGAppMaster.
-  TEZ-2967. Vertex start time should be that of first task start time in UI
-  TEZ-3175. Add tez client submit host
-  TEZ-3166. Fix a few cases where counters aren't fully updated and sent for failed tasks.
-  TEZ-2958. Recovered TA, whose commit cannot be recovered, should move to killed state
-  TEZ-2936. Create ATS implementation that enables support for YARN-4265 (ATS v1.5)
-  TEZ-3148. Invalid event TA_TEZ_EVENT_UPDATE on TaskAttempt.
-  TEZ-3105. TezMxBeanResourceCalculator does not work on IBM JDK 7 or 8 causing Tez failures.
-  TEZ-3155. Support a way to submit DAGs to a session where the DAG plan exceeds hadoop ipc limits.
-  TEZ-2863. Container, node, and logs not available in UI for tasks that fail to launch
-  TEZ-3140. Reduce AM memory usage during serialization
-  TEZ-2756. MergeManager close should not try merging files on close if invoked after a shuffle exception.
-  TEZ-3156. Tez client keeps trying to talk to RM even if RM does not know about the application.
-  TEZ-3115. Shuffle string handling adds significant memory overhead
-  TEZ-3151. Expose DAG credentials to plugins.
-  TEZ-3149. Tez-tools: Add username in DagInfo.
-  TEZ-2988. DAGAppMaster::shutdownTezAM should return with a no-op if it has been invoked earlier.
-  TEZ-3147. Intermediate mem-to-mem: Fix early exit when only one segment can fit into memory
-  TEZ-3141. mapreduce.task.timeout is not translated to container heartbeat timeout
-  TEZ-3128. Avoid stopping containers on the AM shutdown thread.
-  TEZ-3129. Tez task and task attempt UI needs application fails with NotFoundException
-  TEZ-3114. Shuffle OOM due to EventMetaData flood
-  TEZ-1911. MergeManager's unconditionalReserve() should check for memory limits before allocating.
-  TEZ-3102. Fetch failure of a speculated task causes job hang
-  TEZ-3124. Running task hangs due to missing event to initialize input in recovery.
-  TEZ-3135. tez-ext-service-tests, tez-plugins/tez-yarn-timeline-history and tez-tools/tez-javadoc-tools missing dependencies.
-  TEZ-3134. tez-dag should depend on commons-collections4.
-  TEZ-3126. Log reason for not reducing parallelism
-  TEZ-3131. Support a way to override test_root_dir for FaultToleranceTestRunner.
-  TEZ-3067. Links to tez configs documentation should be bubbled up to top-level release page.
-  TEZ-3123. Containers can get re-used even with conflicting local resources.
-  TEZ-3117. Deadlock in Edge and Vertex code
-  TEZ-3103. Shuffle can hang when memory to memory merging enabled
-  TEZ-3107. tez-tools: Log warn msgs in case ATS has wrong values (e.g startTime > finishTime).
-  TEZ-3104. Tez fails on Bzip2 intermediate output format on hadoop 2.7.1 and earlier
-  TEZ-3090. MRInput should make dagIdentifier, vertexIdentifier, etc available to the InputFormat jobConf.
-  TEZ-3093. CriticalPathAnalyzer should be accessible via zeppelin.
-  TEZ-3089. TaskConcurrencyAnalyzer can return negative task count with very large jobs.
-  TEZ-2307. Possible wrong error message when submitting new dag
-  TEZ-2974. Tez tools: TFileRecordReader in tez-tools should support reading >2 GB tfiles.
-  TEZ-3081. Update tez website for trademarks feedback.
-  TEZ-3076. Reduce merge memory overhead to support large number of in-memory mapoutputs
-  TEZ-3079. Fix tez-tfile parser documentation.
-  TEZ-3066. TaskAttemptFinishedEvent ConcurrentModificationException in recovery or history logging services.
-  TEZ-3036. Tez AM can hang on startup with no indication of error
-  TEZ-3052. Task internal error due to Invalid event: T_ATTEMPT_FAILED at FAILED
-  TEZ-2594. Fix LICENSE for missing entries for full and minimal tarballs.
-  TEZ-3053. Containers timeout if they do not receive a task within the container timeout interval.
-  TEZ-2898. tez tools : swimlanes.py is broken.
-  TEZ-2937. Can Processor.close() be called after closing inputs and outputs?
-  TEZ-3037. History URL should be set regardless of which history logging service is enabled.
-  TEZ-3032. DAG start time getting logged using system time instead of recorded time in startTime field.
-  TEZ-3101. Tez UI: Task attempt log link doesn't have the correct protocol.
-  TEZ-3143. Tez UI 2: Make the build faster
-  TEZ-3160. Tez UI 2: Swimlane - Create swimlane page & component (sree)
-  TEZ-3170. Tez UI 2: Swimlane - Display computed events, event bars & dependencies (sree)
-  TEZ-3152. Tez UI 2: Build fails when run by multiple users or when node_modules is old (sree)
-  TEZ-3171. Tez UI 2: Swimlane - Tooltip, zoom & redirection (sree)
-  TEZ-3172. Tez UI: Swimlane - In progress & Shadow (sree)
-  TEZ-3201. Tez-UI build broken (sree)
-
-TEZ-2980: Tez UI 2 - Umbrella:
-  TEZ-2982. Tez UI: Create tez-ui2 directory and get a basic dummy page working in ember 2.2
-  TEZ-3016. Tez UI 2: Make bower dependency silent
-  TEZ-2983. Tez UI 2: Get ember initializers functional
-  TEZ-3018. Tez UI 2: Add config.env
-  TEZ-3019. Tez UI 2: Replace BaseURL with Host
-  TEZ-2984. Tez UI 2: Create abstract classes
-  TEZ-3020. Tez UI 2: Add entity blueprint
-  TEZ-2985. Tez UI 2: Create loader and entity classes
-  TEZ-3021. Tez UI 2: Add env service & initializer
-  TEZ-3023. Tez UI 2: Abstract adapter and route
-  TEZ-3022. Tez UI 2: Add serializer & adapter for timeline server
-  TEZ-3026. Tez UI 2: Add adapters for RM & AM
-  TEZ-3027. Tez UI 2: Add header and footer elements
-  TEZ-2986. Tez UI 2: Implement All DAGs page
-  TEZ-3038. Tez UI 2: Create DAG details page
-  TEZ-3039. Tez UI 2: Create all sub-pages for DAG
-  TEZ-3040. Tez UI 2: Create Vertex details page & sub tables
-  TEZ-3041. Tez UI 2: Create Task & Attempt details page with sub tables
-  TEZ-3045. Tez UI 2: Create application details page with DAGs tab
-  TEZ-3048. Tez UI 2: Make PhantomJS a local dependency for build tests
-  TEZ-3042. Tez UI 2: Create Counters pages
-  TEZ-3043. Tez UI 2: Create configurations page
-  TEZ-3049. Tez UI 2: Add column selector
-  TEZ-3050. Tez UI 2: Add counter columns
-  TEZ-3064. Tez UI 2: Add All DAGs filters
-  TEZ-3059. Tez UI 2: Make refresh functional
-  TEZ-3070. Tez UI 2: Jenkins build is failing
-  TEZ-3060. Tez UI 2: Activate auto-refresh
-  TEZ-3061. Tez UI 2: Display in-progress vertex table in DAG details
-  TEZ-3069. Tez UI 2: Make error bar fully functional
-  TEZ-3062. Tez UI 2: Integrate graphical view
-  TEZ-3058. Tez UI 2: Add download data functionality
-  TEZ-3084. Tez UI 2: Display caller type and info
-  TEZ-3080. Tez UI 2: Ensure UI 2 is in-line with UI 1
-  TEZ-3092. Tez UI 2: Tuneups & Improvements
-  TEZ-3095. Tez UI 2: Tuneups & Improvements
-  TEZ-3088. Tez UI 2: Licenses of all the packages used by Tez Ui must be documented
-  TEZ-2916. Tez UI 2: Show counts of running tasks on the DAG visualization page
-  TEZ-3125. Tez UI 2: All auto-refresh pages refresh multiple times shortly after application complete
-  TEZ-3127. Tez UI 2: Release audit is failing
-
-Release 0.8.2: 2016-01-19
-
-INCOMPATIBLE CHANGES
-  TEZ-3024. Move TaskCommunicator to correct package.
-  TEZ-2679. Admin forms of launch env settings
-  TEZ-2948. Stop using dagName in the dagComplete notification to TaskCommunicators.
-  TEZ-2949. Allow duplicate dag names within session for Tez.
-  TEZ-604. Revert temporary changes made in TEZ-603 to kill the provided tez session, if running a MapReduce job.
-  TEZ-2972. Avoid task rescheduling when a node turns unhealthy
-
-ALL CHANGES:
-  TEZ-2669. Propagation of errors from plugins to the AM for error reporting.
-  TEZ-2978. Add an option to allow the SplitGrouper to generate node local only groups.
-  TEZ-2129. Task and Attempt views should contain links to the logs
-  TEZ-3025. InputInitializer creation should use the dag ugi.
-  TEZ-3017. HistoryACLManager does not have a close method for cleanup
-  TEZ-2914. Ability to limit vertex concurrency
-  TEZ-3011. Link Vertex Name in Dag Tasks/Task Attempts to Vertex
-  TEZ-3006. Remove unused import in TestHistoryParser.
-  TEZ-2910. Set caller context for tracing ( integrate with HDFS-9184 ).
-  TEZ-2976. Recovery fails when InputDescriptor is changed during input initialization.
-  TEZ-2997. Tez UI: Support searches by CallerContext ID for DAGs
-  TEZ-2996. TestAnalyzer fails in trunk after recovery redesign
-  TEZ-2987. TestVertexImpl.testTez2684 fails
-  TEZ-2995. Timeline primary filter should only be on callerId and not type.
-  TEZ-2994. LocalProgress in tez-runtime-library missing Apache header, rat check warnings from the new licenses after TEZ-2592 merge.
-  TEZ-2977. Make HadoopShim selection be overridable for distro-specific implementations.
-  TEZ-2472. Change slf4j version to 1.7.10.
-  TEZ-2920. org.apache.tez.client.TestTezClient.testStopRetriesUntilTimeout is flaky.
-  TEZ-2824. Add javadocs for Vertex.setConf and DAG.setConf.
-  TEZ-2911. Null location Strings can cause problems with GroupedSplit serialization.
-  TEZ-2990. Change test-patch.sh to run through all tests, despite failures in upstream modules
-  TEZ-2798. NPE when executing TestMemoryWithEvents::testMemoryScatterGather.
-  TEZ-2963. RecoveryService#handleSummaryEvent exception with HDFS transparent encryption + kerberos authentication.
-  TEZ-2966. Tez does not honor mapreduce.task.timeout
-  TEZ-2979. FlakyTest: org.apache.tez.history.TestHistoryParser.
-  TEZ-1491. Tez reducer-side merge's counter update is slow.
-  TEZ-2943. Change shuffle vertex manager to use per vertex data for auto
-  reduce and slow start
-  TEZ-2346. TEZ-UI: Lazy load other info / counter data
-  TEZ-2975. Bump up apache commons dependency.
-  TEZ-2970. Re-localization in TezChild does not use correct UGI.
-  TEZ-2968. Counter limits exception causes AM to crash.
-  TEZ-2960. Tez UI: Move hardcoded url namespace to the configuration file
-  TEZ-2581. Umbrella for Tez Recovery Redesign
-  TEZ-2956. Handle auto-reduce parallelism when the
-  totalNumBipartiteSourceTasks is 0
-  TEZ-2947. Tez UI: Timeline, RM & AM requests gets into a consecutive loop in counters page without any delay
-  TEZ-2946. Tez UI: At times RM return a huge error message making the yellow error bar to fill the whole screen
-  TEZ-2949. Allow duplicate dag names within session for Tez.
-  TEZ-2952. NPE in TestOnFileUnorderedKVOutput
-  TEZ-2480. Exception when closing output is ignored.
-  TEZ-2944. NPE in TestProcessorContext.
-  TEZ-2948. Stop using dagName in the dagComplete notification to TaskCommunicators.
-  TEZ-2945. TEZ-2740 addendum to update API with currently supported parameters
-  TEZ-2933. Tez UI: Load application details from RM when available
-  TEZ-2908. Tez UI: Errors are logged, but not displayed in the UI when AM fetch fails
-  TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts
-  TEZ-2924. Framework for Hadoop shims.
-  TEZ-2935. Add MR slow start translation for ShuffleVertexManager
-  TEZ-2918. Make progress notifications in IOs
-  TEZ-2940. Invalid shuffle max slow start setting causes vertex to hang indefinitely
-  TEZ-2930. Tez UI: Parent controller is not polling at times
-  TEZ-1670. Add tests for all converter functions in HistoryEventTimelineConversion.
-  TEZ-2879. While grouping splits, allow an alternate list of preferred locations to be provided per split.
-  TEZ-2929. Tez UI: Dag details page displays vertices to be running even when dag have completed
-  TEZ-1976. Findbug warning: Unread field:
-  org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$SplitHolder.split
-  TEZ-2927. Tez UI: Graciously fail when system-metrics-publisher is disabled
-  TEZ-2915. Tez UI: Getting back to the DAG details page is difficult
-  TEZ-2895. Tez UI: Add option to enable and disable in-progress
-  TEZ-2894. Tez UI: Disable sorting for few columns while in progress. Display an alert on trying to sort them
-  TEZ-2893. Tez UI: Retain vertex info displayed in DAG details page even after completion
-  TEZ-2878. Tez UI: AM error handling - Make the UI handle cases in which AM returns unexpected/no data
-  TEZ-2922. Tez Live UI gives access denied for admins
-  TEZ-2849. Implement Specific Workaround for JDK-8026049 & JDK-8073093.
-  TEZ-2828. Fix typo in "Shuffle assigned " log statement in shuffle.orderedgrouped.Shuffle.
-  TEZ-2909. Tez UI: Application link in All DAGs table is disable when applicationhistory is unavailable
-  TEZ-808. Handle task attempts that are not making progress
-  TEZ-2553. Tez UI: Tez UI Nits
-  TEZ-2814. ATSImportTool has a return statement in a finally block
-  TEZ-2906. Compilation fails with hadoop 2.2.0
-  TEZ-2900. Ignore V_INPUT_DATA_INFORMATION when vertex is in Failed/Killed/Error
-  TEZ-2244. PipelinedSorter: Progressive allocation for sort-buffers
-  TEZ-2904. Pig can't specify task specific command opts
-  TEZ-2888. Make critical path calculation resilient to AM crash
-  TEZ-2899. Tez UI: DAG getting created with huge horizontal gap in between vertices
-  TEZ-2907. NPE in IFile.Reader.getLength during final merge operation
-  TEZ-2903. Stop using proprietary APIs in RPCLoadGen.
-  TEZ-2882. Consider improving fetch failure handling
-  TEZ-2850. Tez MergeManager OOM for small Map Outputs
-  TEZ-1888. Fix javac warnings all over codebase.
-  TEZ-2886. Ability to merge AM credentials with DAG credentials.
-  TEZ-2896. Fix thread names used during Input/Output initialization.
-  TEZ-2866. Tez UI: Newly added columns wont be displayed by default in tables
-  TEZ-2887. Tez build failure due to missing dependency in pom files.
-  TEZ-1692. Reduce code duplication between TezMapredSplitsGrouper and TezMapreduceSplitsGrouper.
-  TEZ-2972. Avoid task rescheduling when a node turns unhealthy
-
-
-Release 0.8.1-alpha: 2015-10-12
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-  TEZ-2885. Remove counter logs from AMWebController
-  TEZ-2096. TEZ-UI : Add link to view AM log of finished & running apps
-  TEZ-2874. Improved logging for caller context.
-  TEZ-2875. Enable missing tests in TestAnalyzer
-  TEZ-2781. Fallback to send only TaskAttemptFailedEvent if taskFailed heartbeat fails
-  TEZ-1788. Allow vertex level disabling of speculation
-  TEZ-2868. Fix setting Caller Context in Tez Examples.
-  TEZ-2860. NPE in DAGClientImpl.
-  TEZ-2855. Fix a potential NPE while routing VertexManager events.
-  TEZ-2758. Remove append API in RecoveryService after TEZ-1909.
-  TEZ-2851. Support a way for upstream applications to pass in a caller context to Tez.
-  TEZ-2859. TestMergeManager.testLocalDiskMergeMultipleTasks failing
-  TEZ-2858. Stop using System.currentTimeMillis in TestInputReadyTracker.
-  TEZ-2857. Fix flakey tests in TestDAGImpl.
-  TEZ-2836. Avoid setting framework/system counters for tasks running in threads.
-  TEZ-2398. Flaky test: TestFaultTolerance
-  TEZ-2833. Don't create extra directory during ATS file download
-  TEZ-2834. Make Tez preemption resilient to incorrect free resource reported
-  by YARN
-  TEZ-2775. Improve and consolidate logging in Runtime components.
-  TEZ-2097. TEZ-UI Add dag logs backend support
-  TEZ-2812. Preemption sometimes does not respect heartbeats between preemptions
-  TEZ-814.  Improve heuristic for determining a task has failed outputs
-  TEZ-2832. Support tests for both SimpleHistory logging and ATS logging
-  TEZ-2827. Increase timeout for TestFetcher testInputAttemptIdentifierMap
-  TEZ-2774. Improvements and cleanup of logging for the AM and parts of the runtme.
-  TEZ-2825. Report progress in terms of completed tasks to reduce load on AM for Tez UI
-  TEZ-2812. Tez UI: Update task & attempt tables while in progress.
-  TEZ-2786. Tez UI: Update vertex, task & attempt details page while in progress.
-  TEZ-2612. Support for showing allocation delays due to internal preemption
-  TEZ-2808. Race condition between preemption and container assignment
-  TEZ-2807. Log data in the finish event instead of the start event
-  TEZ-2799. SimpleHistoryParser NPE
-  TEZ-2643. Minimize number of empty spills in Pipelined Sorter
-  TEZ-2783. Refactor analyzers to extend TezAnalyzerBase
-  TEZ-2784. optimize TaskImpl.isFinished()
-  TEZ-2788. Allow TezAnalyzerBase to parse SimpleHistory logs
-  TEZ-2782. VertexInfo.getAvgExecutionTimeInterval throws NPE when task does not have any valid attempts info
-  TEZ-2778. Improvements to handle multiple read errors with complex DAGs
-  TEZ-2768. Log a useful error message when the summary stream cannot be closed when shutting
-  down an AM.
-  TEZ-2745. ClassNotFoundException of user code should fail dag
-  TEZ-2754. Tez UI: StartTime & EndTime is not displayed with right format in Graphical View
-  TEZ-2752. logUnsuccessful completion in Attempt should write original finish
-  time to ATS
-  TEZ-2755. Fix findbugs warning in TezClient
-  TEZ-2767. Make TezMxBeanResourceCalculator the default resource calculator.
-  TEZ-2765. Change Xmlwriter to use defaultValue instead of value tag.
-  TEZ-2750. Shuffle may not shutdown in case of a fetch failure, causing it to hang.
-  TEZ-2294. Add tez-site-template.xml with description of config properties.
-  TEZ-2757. Fix download links for Tez releases.
-  TEZ-2742. VertexImpl.finished() terminationCause hides member var of the
-  same name
-  TEZ-2747. Update master to reflect 0.8.0-alpha release.
-  TEZ-2662. Provide a way to check whether AM or task opts are valid and error if not.
-  TEZ-2739. Improve handling of read errors in critical path analyzer
-
-Release 0.8.0-alpha: 2015-09-01
-
-INCOMPATIBLE CHANGES
-  TEZ-2048. Remove VertexManagerPluginContext.getTaskContainer()
-  TEZ-2565. Consider scanning unfinished tasks in VertexImpl::constructStatistics to reduce merge overhead.
-  TEZ-2468. Change the minimum Java version to Java 7.
-
-ALL CHANGES:
-  TEZ-2749. TaskInfo in history parser should not depend on the apache directory project. Fix master build against hadoop-2.4
-  TEZ-2748. Fix master build against hadoop-2.2.
-  TEZ-2743. Fix TezContainerLauncher logging tokens.
-  TEZ-2708. Rename classes and variables post TEZ-2003 changes.
-  TEZ-2740. Create a reconfigureVertex alias for deprecated
-  setVertexParallelism API
-  TEZ-2690. Add critical path analyser
-  TEZ-2734. Add a test to verify the filename generated by OnDiskMerge.
-  TEZ-2732. DefaultSorter throws ArrayIndex exceptions on 2047 Mb size sort buffers
-  TEZ-2687. ATS History shutdown happens before the min-held containers are released
-  TEZ-2629. LimitExceededException in Tez client when DAG has exceeds the default max counters
-  TEZ-2730. tez-api missing dependency on org.codehaus.jettison for json.
-  TEZ-2719. Consider reducing logs in unordered fetcher with shared-fetch option
-  TEZ-2646. Add scheduling casual dependency for attempts
-  TEZ-2647. Add input causality dependency for attempts
-  TEZ-2633. Allow VertexManagerPlugins to receive and report based on attempts
-  instead of tasks
-  TEZ-2650. Timing details on Vertex state changes
-  TEZ-2699. Internalize strings in ATF parser
-  TEZ-2701. Add time at which container was allocated to attempt
-  TEZ-2683. TestHttpConnection::testAsyncHttpConnectionInterrupt fails in certain environments.
-  TEZ-2692. bugfixes & enhancements related to job parser and analyzer.
-  TEZ-2663. SessionNotRunning exceptions are wrapped in a ServiceException from a dying AM.
-  TEZ-2630. TezChild receives IP address instead of FQDN.
-  TEZ-2684. ShuffleVertexManager.parsePartitionStats throws IllegalStateException: Stats should be initialized.
-  TEZ-2172. FetcherOrderedGrouped using List to store InputAttemptIdentifier can lead to some inefficiency during remove() operation.
-  TEZ-2613. Fetcher(unordered) using List to store InputAttemptIdentifier can lead to some inefficiency during remove() operation.
-  TEZ-2645. Provide standard analyzers for job analysis.
-  TEZ-2627. Support for Tez Job Priorities.
-  TEZ-2623. Fix module dependencies related to hadoop-auth.
-  TEZ-2464. Move older releases to dist archive.
-  TEZ-2239. Update Tez UI docs to explain how to configure history url for YARN.
-  TEZ-2602. Throwing EOFException when launching MR job.
-  TEZ-2496. Consider scheduling tasks in ShuffleVertexManager based on the partition sizes from the source.
-  TEZ-2616. Fix build warning by undefined version of maven-findbugs-plugin.
-  TEZ-2588. Improper argument name
-  TEZ-2575. Handle KeyValue pairs size which do not fit in a single block.
-  TEZ-2599. Don't send obsoleted data movement events to tasks
-  TEZ-2542. TezDAGID fromString array length check.
-  TEZ-2565. Consider scanning unfinished tasks in VertexImpl::constructStatistics to reduce merge overhead.
-  TEZ-2296. Add option to print counters for tez-examples.
-  TEZ-2570. Fix license header issue for eps image files.
-  TEZ-2378. In case Fetcher (unordered) fails to do local fetch, log in debug mode to reduce log size.
-  TEZ-2558. Upload additional Tez images.
-  TEZ-2486. Update tez website to include links based on
-    http://www.apache.org/foundation/marks/pmcs.html#navigation.
-  TEZ-2548. TezClient submitDAG can hang if the AM is in the process of shutting down.
-  TEZ-2473. Consider using RawLocalFileSystem in MapOutput.createDiskMapOutput.
-  TEZ-2538. ADDITIONAL_SPILL_COUNT wrongly populated for DefaultSorter with multiple partitions.
-  TEZ-2489. Disable warn log for Timeline ACL error when tez.allow.disabled.timeline-domains set to true.
-  TEZ-2376. Remove TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE
-  TEZ-2509. YarnTaskSchedulerService should not try to allocate containers if AM is shutting down.
-  TEZ-2506. TestAsyncHttpConnection failing.
-  TEZ-2503. findbugs version isn't reported properly in test-patch report.
-  TEZ-2198. Fix sorter spill counts.
-  TEZ-1883. Change findbugs version to 3.x.
-  TEZ-2440. Sorter should check for indexCacheList.size() in flush().
-  TEZ-2490. TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability.
-  TEZ-2450. support async http clients in ordered & unordered inputs.
-  TEZ-2454. Change FetcherOrderedGroup to work as Callables instead of blocking threads.
-  TEZ-2466. tez-history-parser breaks hadoop 2.2 compatability.
-  TEZ-2463. Update site for 0.7.0 release
-  TEZ-2461. tez-history-parser compile fails with hadoop-2.4.
-  TEZ-2076. Tez framework to extract/analyze data stored in ATS for specific dag.
-  TEZ-2436. Tez UI: Add cancel button in column selector.
-  TEZ-2351. Remove GroupByOrderByMRRTest example from tez-tests.
-  TEZ-2419. Inputs/Outputs should inform the Processor about Interrupts when interrupted during a blocking Op.
-  TEZ-1752. Inputs / Outputs in the Runtime library should be interruptable.
-  TEZ-1970. Fix javadoc warnings in SortMergeJoinExample.
-
-TEZ-2003: Support for External services CHANGES
-  TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
-  TEZ-2006. Task communication plane needs to be pluggable.
-  TEZ-2090. Add tests for jobs running in external services.
-  TEZ-2117. Add a manager for ContainerLaunchers running in the AM.
-  TEZ-2122. Setup pluggable components at AM/Vertex level.
-  TEZ-2123. Fix component managers to use pluggable components. (Enable hybrid mode)
-  TEZ-2125. Create a task communicator for local mode. Allow tasks to run in the AM.
-  TEZ-2131. Add additional tests for tasks running in the AM.
-  TEZ-2138. Fix minor bugs in adding default scheduler, getting launchers.
-  TEZ-2139. Update tez version to 0.7.0-TEZ-2003-SNAPSHOT.
-  TEZ-2175. Task priority should be available to the TaskCommunicator plugin.
-  TEZ-2187. Allow TaskCommunicators to report failed / killed attempts.
-  TEZ-2241. Miscellaneous fixes after last reabse.
-  TEZ-2283. Fixes after rebase 04/07.
-  TEZ-2284. Separate TaskReporter into an interface.
-  TEZ-2285. Allow TaskCommunicators to indicate task/container liveness.
-  TEZ-2302. Allow TaskCommunicators to subscribe for Vertex updates.
-  TEZ-2347. Expose additional information in TaskCommunicatorContext.
-  TEZ-2361. Propagate dag completion to TaskCommunicator.
-  TEZ-2381. Fixes after rebase 04/28.
-  TEZ-2388. Send dag identifier as part of the fetcher request string.
-  TEZ-2414. LogicalIOProcessorRuntimeTask, RuntimeTask, TezTaskRunner should handle interrupts & carry out necessary cleanups.
-  TEZ-2420. TaskRunner returning before executing the task.
-  TEZ-2433. Fixes after rebase 05/08
-  TEZ-2438. tez-tools version in the branch is incorrect.
-  TEZ-2434. Allow tasks to be killed in the Runtime.
-  TEZ-2443. TaskRunner2 should call abort, NPEs while cleaning up tasks.
-  TEZ-2465. Return the status of a kill request in TaskRunner2.
-  TEZ-2471. NPE in LogicalIOProcessorRuntimeTask while printing thread info.
-  TEZ-2495. Inform TaskCommunicaor about Task and Container termination reasons.
-  TEZ-2502. Fix for TezTaskRunner2 not killing tasks properly in all situations.
-  TEZ-2508. rebase 06/01
-  TEZ-2526. Fix version for tez-history-parser.
-  TEZ-2621. rebase 07/14
-  TEZ-2124. Change Node tracking to work per external container source.
-  TEZ-2004. Define basic interface for pluggable ContainerLaunchers.
-  TEZ-2005. Define basic interface for pluggable TaskScheduler.
-  TEZ-2651. Pluggable services should not extend AbstractService.
-  TEZ-2652. Cleanup the way services are specified for an AM and vertices.
-  TEZ-2653. Change service contexts to expose a user specified payload instead of the AM configuration.
-  TEZ-2441. Add tests for TezTaskRunner2.
-  TEZ-2657. Add tests for client side changes - specifying plugins, etc.
-  TEZ-2626. Fix log lines with DEBUG in messages, consolidate TEZ-2003 TODOs.
-  TEZ-2126. Add unit tests for verifying multiple schedulers, launchers, communicators.
-  TEZ-2698. rebase 08/05
-  TEZ-2675. Add javadocs for new pluggable components, fix problems reported by jenkins
-  TEZ-2678. Fix comments from reviews - part 1.
-  TEZ-2707. Fix comments from reviews - part 2.
-  TEZ-2713. Add tests for node handling when there's multiple schedulers.
-  TEZ-2721. rebase 08/14
-  TEZ-2714. Fix comments from review - part 3.
-  TEZ-2727. Fix findbugs warnings
-  TEZ-2670. Remove TaskAttempt holder used within TezTaskCommunicator.
-  TEZ-2735. rebase 08/21
-  TEZ-2736. Pre-merge: Update CHANGES.txt and version in branch.
-
-Release 0.7.2: Unreleased
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-
-  TEZ-3559. TEZ_LIB_URIS doesn't work with schemes different than the defaultFS
-  TEZ-3549. TaskAttemptImpl does not initialize TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS correctly
-  TEZ-3537. ArrayIndexOutOfBoundsException with empty environment variables/Port YARN-3768 to Tez
-  TEZ-3536. NPE in WebUIService start when host resolution fails.
-  TEZ-3493. DAG submit timeout cannot be set to a month
-  TEZ-3505. Move license to the file header for TezBytesWritableSerialization
-  TEZ-3486. COMBINE_OUTPUT_RECORDS/COMBINE_INPUT_RECORDS are not correct
-  TEZ-3437. Improve synchronization and the progress report behavior for Inputs from TEZ-3317.
-  TEZ-3317. Speculative execution starts too early due to 0 progress.
-  TEZ-3452. Auto-reduce parallelism calculation can overflow with large inputs
-  TEZ-3439. Tez joinvalidate fails when first input argument size is bigger than the second.
-  TEZ-3464. Fix findbugs warnings in tez-dag mainLoop
-  TEZ-3335. DAG client thinks app is still running when app status is null
-  TEZ-3460. Fix precommit release audit warning.
-  TEZ-3368. NPE in DelayedContainerManager
-  TEZ-3440. Shuffling to memory can get out-of-sync when fetching multiple compressed map outputs
-  TEZ-3426. Second AM attempt launched for session mode and recovery disabled for certain cases
-  TEZ-2852. TestVertexImpl fails due to race in AsyncDispatcher.
-  TEZ-3326. Display JVM system properties in AM and task logs.
-  TEZ-3009. Errors that occur during container task acquisition are not logged.
-  TEZ-3413. ConcurrentModificationException in HistoryEventTimelineConversion for AppLaunchedEvent.
-  TEZ-3286. Allow clients to set processor reserved memory per vertex (instead of per container).
-  TEZ-3223. Support a NullHistoryLogger to disable history logging if needed.
-  TEZ-3293. Fetch failures can cause a shuffle hang waiting for memory merge that never starts.
-  TEZ-3305. TestAnalyzer fails on Hadoop 2.7.
-  TEZ-3304. TestHistoryParser fails with Hadoop 2.7.
-  TEZ-3296. Tez job can hang if two vertices at the same root distance have different task requirements
-  TEZ-3294. DAG.createDag() does not clear local state on repeat calls.
-  TEZ-3297. Deadlock scenario in AM during ShuffleVertexManager auto reduce.
-  TEZ-3296. Tez fails to compile against hadoop 2.8 after MAPREDUCE-5870
-  TEZ-3280. LOG MRInputHelpers split generation message as INFO
-  TEZ-3257. Fix flaky test TestUnorderedPartitionedKVWriter.
-  TEZ-3237. Corrupted shuffle transfers to disk are not detected during transfer
-  TEZ-3258. Jvm Checker does not ignore DisableExplicitGC when checking JVM GC options.
-  TEZ-3256. [Backport HADOOP-11032] Remove Guava Stopwatch dependency
-  TEZ-2342. Reduce bytearray copy with TezEvent Serialization and deserialization
-
-Release 0.7.1: 2016-05-10
-
-INCOMPATIBLE CHANGES
-  TEZ-2679. Admin forms of launch env settings
-  TEZ-2949. Allow duplicate dag names within session for Tez.
-
-ALL CHANGES:
-  TEZ-3193. Deadlock in AM during task commit request.
-  TEZ-3203. DAG hangs when one of the upstream vertices has zero tasks
-  TEZ-3224. User payload is not initialized before creating vertex manager plugin.
-  TEZ-3165. Allow Inputs/Outputs to be initialized serially, control processor initialization relative to Inputs/Outputs
-  TEZ-3202. Reduce the memory need for jobs with high number of segments
-  TEZ-3188. Move tez.submit.hosts out of TezConfiguration to TezConfigurationConstants.
-  TEZ-3196. java.lang.InternalError from decompression codec is fatal to a task during shuffle
-  TEZ-3177. Non-DAG events should use the session domain or no domain if the data does not need protection.
-  TEZ-3192. IFile#checkState creating unnecessary objects though auto-boxing
-  TEZ-3189. Pre-warm dags should not be counted in submitted dags count by DAGAppMaster.
-  TEZ-2967. Vertex start time should be that of first task start time in UI
-  TEZ-3175. Add tez client submit host
-  TEZ-3166. Fix a few cases where counters aren't fully updated and sent for failed tasks.
-  TEZ-2958. Recovered TA, whose commit cannot be recovered, should move to killed state
-  TEZ-3105. TezMxBeanResourceCalculator does not work on IBM JDK 7 or 8 causing Tez failures.
-  TEZ-2863. Container, node, and logs not available in UI for tasks that fail to launch
-  TEZ-3140. Reduce AM memory usage during serialization
-  TEZ-3156. Tez client keeps trying to talk to RM even if RM does not know about the application.
-  TEZ-3115. Shuffle string handling adds significant memory overhead
-  TEZ-3149. Tez-tools: Add username in DagInfo.
-  TEZ-2988. DAGAppMaster::shutdownTezAM should return with a no-op if it has been invoked earlier.
-  TEZ-3141. mapreduce.task.timeout is not translated to container heartbeat timeout
-  TEZ-3129. Tez task and task attempt UI needs application fails with NotFoundException
-  TEZ-3114. Shuffle OOM due to EventMetaData flood
-  TEZ-3102. Fetch failure of a speculated task causes job hang
-  TEZ-3126. Log reason for not reducing parallelism
-  TEZ-3123. Containers can get re-used even with conflicting local resources.
-  TEZ-3117. Deadlock in Edge and Vertex code
-  TEZ-3103. Shuffle can hang when memory to memory merging enabled
-  TEZ-3107. tez-tools: Log warn msgs in case ATS has wrong values (e.g startTime > finishTime).
-  TEZ-3104. Tez fails on Bzip2 intermediate output format on hadoop 2.7.1 and earlier
-  TEZ-3093. CriticalPathAnalyzer should be accessible via zeppelin.
-  TEZ-3089. TaskConcurrencyAnalyzer can return negative task count with very large jobs.
-  TEZ-2307. Possible wrong error message when submitting new dag.
-  TEZ-3076. Reduce merge memory overhead to support large number of in-memory mapoutputs
-  TEZ-3066. TaskAttemptFinishedEvent ConcurrentModificationException in recovery or history logging services.
-  TEZ-3036. Tez AM can hang on startup with no indication of error
-  TEZ-3052. Task internal error due to Invalid event: T_ATTEMPT_FAILED at FAILED
-  TEZ-2937. Can Processor.close() be called after closing inputs and outputs?
-  TEZ-3037. History URL should be set regardless of which history logging service is enabled.
-  TEZ-3032. DAG start time getting logged using system time instead of recorded time in startTime field.
-  TEZ-2129. Task and Attempt views should contain links to the logs
-  TEZ-3025. InputInitializer creation should use the dag ugi.
-  TEZ-3017. HistoryACLManager does not have a close method for cleanup
-  TEZ-2914. Ability to limit vertex concurrency
-  TEZ-2918. Make progress notifications in IOs
-  TEZ-2952. NPE in TestOnFileUnorderedKVOutput
-  TEZ-808. Handle task attempts that are not making progress
-  TEZ-2987. TestVertexImpl.testTez2684 fails
-  TEZ-2599. Don't send obsoleted data movement events to tasks
-  TEZ-2943. Change shuffle vertex manager to use per vertex data for auto
-  TEZ-2633. Allow VertexManagerPlugins to receive and report based on attempts
-  TEZ-3011. Link Vertex Name in Dag Tasks/Task Attempts to Vertex
-  TEZ-2538. ADDITIONAL_SPILL_COUNT wrongly populated for DefaultSorter with multiple partitions.
-  TEZ-3006. Remove unused import in TestHistoryParser.
-  TEZ-2979. FlakyTest: org.apache.tez.history.TestHistoryParser.
-  TEZ-2684. ShuffleVertexManager.parsePartitionStats throws IllegalStateException: Stats should be initialized.
-  TEZ-2496. Consider scheduling tasks in ShuffleVertexManager based on the partition sizes from the source.
-  TEZ-2995. Timeline primary filter should only be on callerId and not type.
-  TEZ-2824. Add javadocs for Vertex.setConf and DAG.setConf.
-  TEZ-2963. RecoveryService#handleSummaryEvent exception with HDFS transparent encryption + kerberos authentication.
-  TEZ-2966. Tez does not honor mapreduce.task.timeout
-  TEZ-2346. TEZ-UI: Lazy load other info / counter data
-  TEZ-2975. Bump up apache commons dependency.
-  TEZ-2970. Re-localization in TezChild does not use correct UGI.
-  TEZ-2968. Counter limits exception causes AM to crash.
-  TEZ-2947. Tez UI: Timeline, RM & AM requests gets into a consecutive loop in counters page without any delay
-  TEZ-2949. Allow duplicate dag names within session for Tez.
-  TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts
-  TEZ-2935. Add MR slow start translation for ShuffleVertexManager
-  TEZ-2940. Invalid shuffle max slow start setting causes vertex to hang indefinitely
-  TEZ-1670. Add tests for all converter functions in HistoryEventTimelineConversion.
-  TEZ-2922. Tez Live UI gives access denied for admins
-  TEZ-2828. Fix typo in "Shuffle assigned " log statement in shuffle.orderedgrouped.Shuffle.
-  TEZ-2900. Ignore V_INPUT_DATA_INFORMATION when vertex is in Failed/Killed/Error
-  TEZ-2904. Pig can't specify task specific command opts
-  TEZ-2899. Tez UI: DAG getting created with huge horizontal gap in between vertices
-  TEZ-2882. Consider improving fetch failure handling
-  TEZ-2907. NPE in IFile.Reader.getLength during final merge operation
-  TEZ-2850. Tez MergeManager OOM for small Map Outputs
-  TEZ-2886. Ability to merge AM credentials with DAG credentials.
-  TEZ-2896. Fix thread names used during Input/Output initialization.
-  TEZ-2866. Tez UI: Newly added columns wont be displayed by default in tables
-  TEZ-2885. Remove counter logs from AMWebController.
-  TEZ-2887. Tez build failure due to missing dependency in pom files.
-  TEZ-2096. TEZ-UI : Add link to view AM log of finished & running apps
-  TEZ-2874. Improved logging for caller context.
-  TEZ-2781. Fallback to send only TaskAttemptFailedEvent if taskFailed heartbeat fails
-  TEZ-2868. Fix setting Caller Context in Tez Examples.
-  TEZ-2860. NPE in DAGClientImpl.
-  TEZ-2855. Fix a potential NPE while routing VertexManager events.
-  TEZ-2758. Remove append API in RecoveryService after TEZ-1909.
-  TEZ-2851. Support a way for upstream applications to pass in a caller context to Tez.
-  TEZ-2858. Stop using System.currentTimeMillis in TestInputReadyTracker.
-  TEZ-2857. Fix flakey tests in TestDAGImpl.
-  TEZ-2398. Flaky test: TestFaultTolerance
-  TEZ-2808. Race condition between preemption and container assignment
-  TEZ-2853. Tez UI: task attempt page is coming empty
-  TEZ-2716. DefaultSorter.isRleNeeded not thread safe
-  TEZ-2847. Tez UI: Task details doesn't gets updated on manual refresh after job complete
-  TEZ-2843. Tez UI: Show error if in progress fails due to AM not reachable
-  TEZ-2842. Tez UI: Update Tez App details page while in-progress
-  TEZ-2834. Make Tez preemption resilient to incorrect free resource reported
-  by YARN
-  TEZ-2775. Improve and consolidate logging in Runtime components.
-  TEZ-2097. TEZ-UI Add dag logs backend support
-  TEZ-2812. Preemption sometimes does not respect heartbeats between preemptions
-  TEZ-814.  Improve heuristic for determining a task has failed outputs
-  TEZ-2829. Tez UI: minor fixes to in-progress update of UI from AM
-  TEZ-2663. SessionNotRunning exceptions are wrapped in a ServiceException from a dying AM.
-  TEZ-2825. Report progress in terms of completed tasks to reduce load on AM for Tez UI
-  TEZ-2812. Tez UI: Update task & attempt tables while in progress.
-  TEZ-2786. Tez UI: Update vertex, task & attempt details page while in progress.
-  TEZ-2817. Tez UI: update in progress counter data for the dag vertices and tasks table
-  TEZ-2813. Tez UI: add counter data for rest api calls to AM Web Services v2
-  TEZ-2660. Tez UI: need to show application page even if system metrics publish is disabled.
-  TEZ-2787. Tez AM should have java.io.tmpdir=./tmp to be consistent with tasks
-  TEZ-2780. Tez UI: Update All Tasks page while in progress
-  TEZ-2792. Add AM web service API for tasks
-  TEZ-2807. Log data in the finish event instead of the start event
-  TEZ-2766. Tez UI: Add vertex in-progress info in DAG details
-  TEZ-2768. Log a useful error message when the summary stream cannot be closed when shutting
-  down an AM.
-  TEZ-2745. ClassNotFoundException of user code should fail dag
-  TEZ-2761. Tez UI: update the progress on the dag and vertices pages with info from AM
-  TEZ-2731. Fix Tez GenericCounter performance bottleneck
-  TEZ-2752. logUnsuccessful completion in Attempt should write original finish
-  time to ATS
-  TEZ-2755. Fix findbugs warning in TezClient
-  TEZ-2767. Make TezMxBeanResourceCalculator the default resource calculator.
-  TEZ-2602. Throwing EOFException when launching MR job
-  TEZ-2575. Handle KeyValue pairs size which do not fit in a single block in PipelinedSorter
-  TEZ-2198. Fix sorter spill counts
-  TEZ-2440. Sorter should check for indexCacheList.size() in flush()
-  TEZ-2742. VertexImpl.finished() terminationCause hides member var of the
-  same name
-  TEZ-2662. Provide a way to check whether AM or task opts are valid and error if not.
-  TEZ-2300. TezClient.stop() takes a lot of time or does not work sometimes
-  TEZ-2734. Add a test to verify the filename generated by OnDiskMerge.
-  TEZ-2732. DefaultSorter throws ArrayIndex exceptions on 2047 Mb size sort buffers
-  TEZ-2687. ATS History shutdown happens before the min-held containers are released
-  TEZ-2629. LimitExceededException in Tez client when DAG has exceeds the default max counters
-  TEZ-2540. Create both tez-dist minimal and minimal.tar.gz formats as part of build
-  TEZ-2630. TezChild receives IP address instead of FQDN.
-  TEZ-2211. Tez UI: Allow users to configure timezone
-  TEZ-2623. Fix module dependencies related to hadoop-auth.
-  TEZ-1314. Port MAPREDUCE-5821 to Tez.
-  TEZ-2568. V_INPUT_DATA_INFORMATION may happen after vertex is initialized
-  TEZ-2291. TEZ UI: Improper vertex name in tables.
-  TEZ-2567. Tez UI: download dag data does not work within ambari
-  TEZ-2559. tez-ui fails compilation due to version dependency of frontend-maven-plugin
-  TEZ-2545. It is not necessary to start the vertex group commit when DAG is in TERMINATING
-  TEZ-2554. Tez UI: View log link does not correctly propagate login crendential to read log from yarn web.
-  TEZ-2548. TezClient submitDAG can hang if the AM is in the process of shutting down.
-  TEZ-2547. Tez UI: Download Data fails on secure, cross-origin clusters
-  TEZ-1961. Remove misleading exception "No running dag" from AM logs.
-  TEZ-2546. Tez UI: Fetch hive query text from timeline if dagInfo is not set.
-  TEZ-2513. Tez UI: Allow filtering by DAG ID on All dags table.
-  TEZ-2541. DAGClientImpl enable TimelineClient check is wrong.
-  TEZ-2539. Tez UI: Pages are not updating in IE.
-  TEZ-2535. Tez UI: Failed task attempts link in vertex details page is broken.
-  TEZ-2489. Disable warn log for Timeline ACL error when tez.allow.disabled.timeline-domains set to true.
-  TEZ-2528. Tez UI: Column selector buttons gets clipped, and table scroll bar not visible in mac.
-  TEZ-2391. TestVertexImpl timing out at times on jenkins builds.
-  TEZ-2509. YarnTaskSchedulerService should not try to allocate containers if AM is shutting down.
-  TEZ-2527. Tez UI: Application hangs on entering erroneous RegEx in counter table search box
-  TEZ-2523. Tez UI: derive applicationId from dag/vertex id instead of relying on json data
-  TEZ-2505. PipelinedSorter uses Comparator objects concurrently from multiple threads.
-  TEZ-2504. Tez UI: tables - show status column without scrolling, numeric 0 shown as Not available
-  TEZ-2478. Move OneToOne routing to store events in Tasks.
-  TEZ-2482. Tez UI: Mouse events not working on IE11
-  TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster.
-  TEZ-2481. Tez UI: graphical view does not render properly on IE11
-  TEZ-2474. The old taskNum is logged incorrectly when parallelism is changed
-  TEZ-2460. Temporary solution for issue due to YARN-2560
-  TEZ-2455. Tez UI: Dag view caching, error handling and minor layout changes
-  TEZ-2453. Tez UI: show the dagInfo is the application has set the same.
-  TEZ-2447. Tez UI: Generic changes based on feedbacks.
-  TEZ-2409. Allow different edges to have different routing plugins
-
-Release 0.7.0: 2015-05-18
-
-INCOMPATIBLE CHANGES
-  TEZ-2176. Move all logging to slf4j. (commons-logging jar no longer part of Tez tar)
-  TEZ-1993. Implement a pluggable InputSizeEstimator for grouping fairly.
-  TEZ-2424. Bump up max counter group name length limit to account for per_io counters.
-    Default max limit increased. Should not affect existing users.
-
-ALL CHANGES:
-  TEZ-2446. Tez UI: Add tezVersion details when downloading timeline data for offline use
-  TEZ-2432. Syntax error in DOAP file release section
-  TEZ-2445. Disable the object cleanup in local mode in LogicalIOProcessorRuntimeTask.
-  TEZ-2057. tez-dag/pom.xml contains versions for dependencies.
-  TEZ-2435. Add public key to KEYS
-  TEZ-2421. Deadlock in AM because attempt and vertex locking each other out
-  TEZ-2426. Ensure the eventRouter thread completes before switching to a new task and thread safety fixes in IPOContexts.
-  TEZ-2412. Should kill vertex in DAGImpl#VertexRerunWhileCommitting
-  TEZ-2410. VertexGroupCommitFinishedEvent & VertexCommitStartedEvent is not logged correctly
-  TEZ-776. Reduce AM mem usage caused by storing TezEvents
-  TEZ-2423. Tez UI: Remove Attempt Index column from task->attempts page
-  TEZ-2416. Tez UI: Make tooltips display faster.
-  TEZ-2404. Handle DataMovementEvent before its TaskAttemptCompletedEvent
-  TEZ-2424. Bump up max counter group name length limit to account for per_io counters.
-  TEZ-2417. Tez UI: Counters are blank in the Attempts page if all attempts failed
-  TEZ-2366. Pig tez MiniTezCluster unit tests fail intermittently after TEZ-2333
-  TEZ-2406. Tez UI: Display per-io counter columns in task and attempt pages under vertex
-  TEZ-2384. Add warning message in the case of prewarn under non-session mode.
-  TEZ-2415. PMC RDF needs to use asfext:pmc, not asfext:PMC.
-  TEZ-2401. Tez UI: All-dag page has duration keep counting for KILLED dag.
-  TEZ-2392. Have all readers throw an Exception on incorrect next() usage.
-  TEZ-2408. TestTaskAttempt fails to compile against hadoop-2.4 and hadoop-2.2.
-  TEZ-2405. PipelinedSorter can throw NPE with custom compartor.
-  TEZ-1897. Create a concurrent version of AsyncDispatcher
-  TEZ-2394. Issues when there is an error in VertexManager callbacks
-  TEZ-2386. Tez UI: Inconsistent usage of icon colors
-  TEZ-2395. Tez UI: Minimum/Maximum Duration show a empty bracket next to 0 secs when you purposefully failed a job.
-  TEZ-2360. per-io counters flag should generate both overall and per-edge counters
-  TEZ-2389. Tez UI: Sort by attempt-no is incorrect in attempts pages.
-  TEZ-2363: Fix off-by-one error in REDUCE_INPUT_RECORDS counter
-  TEZ-2383. Cleanup input/output/processor contexts in LogicalIOProcessorRuntimeTask.
-  TEZ-2084. Tez UI: Stacktrace format info is lost in diagnostics
-  TEZ-2374. Fix build break against hadoop-2.2 due to TEZ-2325.
-  TEZ-2314. Tez task attempt failures due to bad event serialization
-  TEZ-2368. Make a dag identifier available in Context classes.
-  TEZ-2325. Route status update event directly to the attempt.
-  TEZ-2358. Pipelined Shuffle: MergeManager assumptions about 1 merge per source-task.
-  TEZ-2342. TestFaultTolerance.testRandomFailingTasks fails due to timeout.
-  TEZ-2362. State Change Notifier Thread should be stopped when dag is
-  completed
-  TEZ-2364. Resolve config parameter replacement on the client, before sending them to the AM.
-  TEZ-2298. Avoid logging full exception trace in TaskRunner when it's not the main error reason and is ignored.
-  TEZ-2248. VertexImpl/DAGImpl.checkForCompletion have too many termination cause checks
-  TEZ-2341. TestMockDAGAppMaster.testBasicCounters fails on windows
-  TEZ-2352. Move getTaskStatistics into the RuntimeTask class.
-  TEZ-2357. Tez UI: misc.js.orig is committed by accident
-  TEZ-2261. Should add diagnostics in DAGAppMaster when recovery error happens
-  TEZ-2340. TestRecoveryParser fails
-  TEZ-2345. Tez UI: Enable cell level loading in all DAGs table
-  TEZ-2330. Create reconfigureVertex() API for input based initialization
-  TEZ-2292. Add e2e test for error reporting when vertex manager invokes
-  plugin APIs
-  TEZ-2308. Add set/get of record counts in task/vertex statistics
-  TEZ-2344. Tez UI: Equip basic-ember-table's cell level loading for all use cases in all DAGs table
-  TEZ-2313. Regression in handling obsolete events in ShuffleScheduler.
-  TEZ-2212. Notify components on DAG completion.
-  TEZ-2328. Add tez.runtime.sorter.class & rename tez.runtime.sort.threads
-  to tez.runtime.pipelined.sorter.sort.threads.
-  TEZ-2333. Enable local fetch optimization by default.
-  TEZ-2310. Deadlock caused by StateChangeNotifier sending notifications on
-  thread holding locks
-  TEZ-1969. Stop the DAGAppMaster when a local mode client is stopped
-  TEZ-714. OutputCommitters should not run in the main AM dispatcher thread
-  TEZ-2323. Fix TestOrderedWordcount to use MR memory configs.
-  TEZ-1482. Fix memory issues for Local Mode running concurrent tasks
-  TEZ-2033. Update TestOrderedWordCount to add processor configs as history text
-  and use MR configs correctly
-  TEZ-2318. Tez UI: source and sink page is broken as they are not populated.
-  TEZ-2016. Tez UI: Dag View Fit and Finish
-  TEZ-2252. Tez UI: in graphical view some of the sinks are hidden as they overlap
-  TEZ-2275. Tez UI: enable faster loading and caching of data in tables
-  TEZ-2234. Add API for statistics information - allow vertex managers to get
-  output size per source vertex
-  TEZ-2274. Tez UI: full data loading, client side search and sort for other pages
-  TEZ-2301. Switch Tez Pre-commit builds to use tezqa user.
-  TEZ-2299. Invalid dag creation in MRRSleepJob post TEZ-2293.
-  TEZ-2290. Scale memory for Default Sorter down to a max of 2047 MB if configured higher.
-  TEZ-2233. Allow EdgeProperty of an edge to be changed by VertexManager
-  TEZ-2293. When running in "mr" mode, always use MR config settings.
-  TEZ-2273. Tez UI: Support client side searching & sorting for dag tasks page
-  TEZ-2223. TestMockDAGAppMaster fails due to TEZ-2210 on mac.
-  TEZ-2236. Tez UI: Support loading of all tasks in the dag tasks page
-  TEZ-2159. Tez UI: download timeline data for offline use.
-  TEZ-2269. DAGAppMaster becomes unresponsive (post TEZ-2149).
-  TEZ-2243. documentation should explicitly specify protobuf 2.5.0.
-  TEZ-2232. Allow setParallelism to be called multiple times before tasks get
-  scheduled
-  TEZ-2265. All inputs/outputs in a task share the same counter object
-  TEZ-2251. Race condition in VertexImpl & Edge causes DAG to hang.
-  TEZ-2264. Remove unused taskUmbilical reference in TezTaskRunner, register as running late.
-  TEZ-2149. Optimizations for the timed version of DAGClient.getStatus.
-  TEZ-2213. For the ordered case, enabling pipelined shuffle should automatically disable final merge.
-  TEZ-2204. TestAMRecovery increasingly flaky on jenkins builds.
-  TEZ-2209. Fix pipelined shuffle to fetch data from any one attempt
-  TEZ-2210. Record DAG AM CPU usage stats
-  TEZ-2203. Intern strings in tez counters
-  TEZ-2202. Fix LocalTaskExecutionThread ID to the standard thread numbering.
-  TEZ-2059. Remove TaskEventHandler in TestDAGImpl
-  TEZ-2191. Simulation improvements to MockDAGAppMaster
-  TEZ-2195. TestTezJobs::testInvalidQueueSubmission/testInvalidQueueSubmissionToSession
-    fail with hadoop branch-2.
-  TEZ-1827. MiniTezCluster takes 10 minutes to shut down.
-  TEZ-2178. YARN-3122 breaks tez compilation with hadoop 2.7.0.
-  TEZ-2174. Make task priority available to TaskAttemptListener.
-  TEZ-2169. Add NDC context to various threads and pools.
-  TEZ-2171. Remove unused metrics code.
-  TEZ-2001. Support pipelined data transfer for ordered output.
-  TEZ-2170. Incorrect its in README.md.
-  TEZ-2070. Controller class of output should be committer rather than initializer in DAG's dot file.
-  TEZ-2083. Make PipelinedSorter as the default sorter.
-  TEZ-1967. Add a monitoring API on DAGClient which returns after a time interval or on DAG final state change.
-  TEZ-2130. Send the sessionToken as part of the AM CLC.
-  TEZ-1935. Organization should be removed from http://tez.apache.org/team-list.html.
-  TEZ-2009. Change license/copyright headers to 2015.
-  TEZ-2085. PipelinedSorter should bail out (on BufferOverflowException) instead of retrying continuously.
-  TEZ-167. Create tests for MR Combiner.
-  TEZ-2080. LocalClient should be using tezconf in init instead of yarnconf.
-  TEZ-2072. Add missing Private annotation to createDAG in the DAG API class.
-  TEZ-2095. master branch fails to compile against hadoop-2.4.
-  TEZ-2093. Add events to MockDAGAppMaster and add e2e test for event routing
-  TEZ-2075. Incompatible issue caused by TEZ-1233 that TezConfiguration.TEZ_SITE_XML is made private
-  TEZ-2082. Race condition in TaskAttemptListenerImpTezDag.getTask()
-  TEZ-1233. Allow configuration of framework parameters per vertex.
-  TEZ-2045. TaskAttemptListener should not pull Tasks from AMContainer. Instead these should be registered with the listener.
-  TEZ-1914. VertexManager logic should not run on the central dispatcher
-  TEZ-2023. Refactor logIndividualFetchComplete() to be common for both shuffle-schedulers.
-  TEZ-1999. IndexOutOfBoundsException during merge.
-  TEZ-2000. Source vertex exists error during DAG submission.
-  TEZ-2008. Add methods to SecureShuffleUtils to verify a reply based on a provided Key.
-  TEZ-1995. Build failure against hadoop 2.2.
-  TEZ-1997. Remove synchronization DefaultSorter::isRLENeeded() (Causes sorter to hang indefinitely in large jobs).
-  TEZ-1996. Update Website after 0.6.0
-  TEZ-1803. Support > 2gb sort buffer in pipelinedsorter.
-  TEZ-1826. Add option to disable split grouping and local mode option for tez-examples.
-  TEZ-1982. TezChild setupUgi should not be using environment.
-  TEZ-1980. Suppress tez-dag findbugs warnings until addressed.
-  TEZ-1855. Avoid scanning for previously written files within Inputs / Outputs.
-  TEZ-1902. Fix findbugs warnings in tez-mapreduce.
-  TEZ-1963. Fix post memory merge to be > 2 GB.
-  TEZ-1901. Fix findbugs warnings in tez-examples.
-  TEZ-1941. Memory provided by *Context.getAvailableMemory needs to be setup explicitly.
-  TEZ-1879. Create local UGI instances for each task and the AM, when running in LocalMode.
-  TEZ-1661. LocalTaskScheduler hangs when shutdown.
-  TEZ-1951. Fix general findbugs warnings in tez-dag.
-  TEZ-1905. Fix findbugs warnings in tez-tests.
-  TEZ-1945. Remove 2 GB memlimit restriction in MergeManager.
-  TEZ-1913. Reduce deserialize cost in ValuesIterator.
-  TEZ-1917. Examples should extend TezExampleBase.
-  TEZ-1892. Add hashCode and equals for Vertex/VertexGroup/Edge/GroupInputEdge.
-  TEZ-1904. Fix findbugs warnings in tez-runtime-library module.
-  TEZ-1903. Fix findbugs warnings in tez-runtime-internal module.
-  TEZ-1896. Move the default heartbeat timeout and checkinterval to TezConfiguration.
-  TEZ-1274. Remove Key/Value type checks in IFile.
-  TEZ-1912. Merge exceptions are thrown when enabling tez.runtime.shuffle.memory-to-memory.enable && tez.runtime.shuffle.memory-to-memory.segments.
-  TEZ-1922. Fix comments: add UNSORTED_OUTPUT to TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS.
-  TEZ-485. Get rid of TezTaskStatus.
-  TEZ-1899. Fix findbugs warnings in tez-common module.
-  TEZ-1898. Fix findbugs warnings in tez-api module.
-  TEZ-1906. Fix findbugs warnings in tez-yarn-timeline-history-with-acls.
-  TEZ-1767. Enable RLE in reducer side merge codepath.
-  TEZ-1837. Restrict usage of Environment variables to main methods.
-  TEZ-1867. Create new central dispatcher for Tez AM
-  TEZ-1844. Shouldn't invoke system.exit in local mode when AM is failed to start.
-  TEZ-1889. Fix test-patch to provide correct findbugs report.
-  TEZ-1313. Setup pre-commit build to test submitted patches.
-  TEZ-1856. Remove LocalOnFileSortedOutput, LocalMergedInput, LocalTaskOutputFiles.
-  TEZ-1949. Whitelist TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH for broadcast edges.
-  TEZ-1593. Refactor PipelinedSorter to remove all MMAP based ByteBuffer references.
-
-Release 0.6.3: Unreleased
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-  TEZ-2907. NPE in IFile.Reader.getLength during final merge operation
-  TEZ-2850. Tez MergeManager OOM for small Map Outputs
-  TEZ-2781. Fallback to send only TaskAttemptFailedEvent if taskFailed heartbeat fails
-  TEZ-2855. Fix a potential NPE while routing VertexManager events.
-  TEZ-2716. DefaultSorter.isRleNeeded not thread safe
-  TEZ-2758. Remove append API in RecoveryService after TEZ-1909.
-  TEZ-2851. Support a way for upstream applications to pass in a caller context to Tez.
-  TEZ-2398. Flaky test: TestFaultTolerance
-  TEZ-2808. Race condition between preemption and container assignment
-  TEZ-2834. Make Tez preemption resilient to incorrect free resource reported
-  by YARN
-  TEZ-2097. TEZ-UI Add dag logs backend support
-  TEZ-2812. Preemption sometimes does not respect heartbeats between preemptions
-  TEZ-814.  Improve heuristic for determining a task has failed outputs
-  TEZ-2809. Minimal distribution compiled on 2.6 fails to run on 2.7
-  TEZ-2768. Log a useful error message when the summary stream cannot be closed when shutting
-  down an AM.
-  TEZ-2745. ClassNotFoundException of user code should fail dag
-  TEZ-2752. logUnsuccessful completion in Attempt should write original finish
-  time to ATS
-  TEZ-2742. VertexImpl.finished() terminationCause hides member var of the
-  same name
-  TEZ-2732. DefaultSorter throws ArrayIndex exceptions on 2047 Mb size sort buffers
-  TEZ-2290. Scale memory for Default Sorter down to a max of 2047 MB if configured higher.
-  TEZ-2734. Add a test to verify the filename generated by OnDiskMerge.
-  TEZ-2687. ATS History shutdown happens before the min-held containers are released
-  TEZ-2629. LimitExceededException in Tez client when DAG has exceeds the default max counters
-  TEZ-2630. TezChild receives IP address instead of FQDN.
-
-Release 0.6.2: 2015-08-07
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-  TEZ-2311. AM can hang if kill received while recovering from previous attempt.
-  TEZ-2623. Fix module dependencies related to hadoop-auth.
-  TEZ-2560. fix tex-ui build for maven 3.3+
-  TEZ-2600. When used with HDFS federation(viewfs) ,tez will throw a error
-  TEZ-2579. Incorrect comparison of TaskAttemptId
-  TEZ-2549. Reduce Counter Load on the Timeline Server
-  TEZ-2548. TezClient submitDAG can hang if the AM is in the process of shutting down.
-  TEZ-2534. Error handling summary event when shutting down AM.
-  TEZ-2511. Add exitCode to diagnostics when container fails.
-  TEZ-2489. Disable warn log for Timeline ACL error when tez.allow.disabled.timeline-domains set to true.
-  TEZ-2509. YarnTaskSchedulerService should not try to allocate containers if AM is shutting down.
-  TEZ-2483. Tez should close task if processor fail
-
-Release 0.6.1: 2015-05-18
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-  TEZ-2437. FilterLinesByWord NPEs when run in Localmode
-  TEZ-2057. tez-dag/pom.xml contains versions for dependencies.
-  TEZ-2282. Delimit reused yarn container logs (stderr, stdout, syslog) with task attempt start/stop events
-  TEZ-2396. pig-tez-tfile-parser pom is hard coded to depend on 0.6.0-SNAPSHOT version.
-  TEZ-2237. Valid events should be sent out when an Output is not started.
-  TEZ-1988. Tez UI: does not work when using file:// in a browser
-  TEZ-2390. tez-tools swimlane tool fails to parse large jobs >8K containers
-  TEZ-2256. Avoid use of BufferTooSmallException to signal end of buffer in UnorderedPartitionedKVWriter
-  TEZ-2380. Disable fall back to reading from timeline if timeline disabled.
-  TEZ-2226. Disable writing history to timeline if domain creation fails.
-  TEZ-2259. Push additional data to Timeline for Recovery for better consumption in UI.
-  TEZ-2365. Update tez-ui war's license/notice to reflect OFL license correctly.
-  TEZ-2329. UI Query on final dag status performance improvement
-  TEZ-2287. Deprecate VertexManagerPluginContext.getTaskContainer().
-  TEZ-1909. Remove need to copy over all events from attempt 1 to attempt 2 dir
-  TEZ-2061. Tez UI: vertex id column and filter on tasks page should be changed to vertex name
-  TEZ-2242. Refactor ShuffleVertexManager code
-  TEZ-2205. Tez still tries to post to ATS when yarn.timeline-service.enabled=false.
-  TEZ-2047. Build fails against hadoop-2.2 post TEZ-2018
-  TEZ-2064. SessionNotRunning Exception not thrown is all cases
-  TEZ-2189. Tez UI live AM tracking url only works for localhost addresses
-  TEZ-2179. Timeline relatedentries missing cause exaggerated warning.
-  TEZ-2168. Fix application dependencies on mutually exclusive artifacts: tez-yarn-timeline-history
-    and tez-yarn-timeline-history-with-acls.
-  TEZ-2190. TestOrderedWordCount fails when generateSplitsInClient set to true.
-  TEZ-2091. Add support for hosting TEZ_UI with nodejs.
-  TEZ-2165. Tez UI: DAG shows running status if killed by RM in some cases.
-  TEZ-2158. TEZ UI: Display dag/vertex names, and task/attempt index in breadcrumb.
-  TEZ-2160. Tez UI: App tracking URL should support navigation back.
-  TEZ-2147. Swimlanes: Improved tooltip
-  TEZ-2142. TEZ UI: Breadcrumb border color looks out of place in wrapped mode.
-  TEZ-2134. TEZ UI: On request failure, display request URL and server name in error bar.
-  TEZ-2136. Some enhancements to the new Tez UI.
-  TEZ-2135. ACL checks handled incorrectly in AMWebController.
-  TEZ-1990. Tez UI: DAG details page shows Nan for end time when a DAG is running.
-  TEZ-2116. Tez UI: dags page filter does not work if more than one filter is specified.
-  TEZ-2106. TEZ UI: Display data load time, and add a refresh button for items that can be refreshed.
-  TEZ-2114. Tez UI: task/task attempt status is not available when its running.
-  TEZ-2112. Tez UI: fix offset calculation, add home button to breadcrumbs.
-  TEZ-2038. TEZ-UI DAG is always running in tez-ui when the app is failed but no DAGFinishedEvent is logged.
-  TEZ-2102. Tez UI: DAG view has hidden edges, dragging DAG by holding vertex causes unintended click.
-  TEZ-2101. Tez UI: Issues on displaying a table.
-  TEZ-2092. Tez UI history url handler injects spurious trailing slash.
-  TEZ-2098. Tez UI: Dag details should be the default page for dag, fix invalid time entries for failed Vertices.
-  TEZ-2024. TaskFinishedEvent may not be logged in recovery.
-  TEZ-2031. Tez UI: horizontal scrollbars do not appear in tables, causing them to look truncated.
-  TEZ-2073. SimpleHistoryLoggingService cannot be read by log aggregation (umask)
-  TEZ-2078. Tez UI: Task logs url use in-progress url causing various errors.
-  TEZ-2077. Tez UI: No diagnostics on Task Attempt Details page if task attempt failed.
-  TEZ-2065. Setting up tez.tez-ui.history-url.base with a trailing slash can result in failures to redirect correctly.
-  TEZ-2068. Tez UI: Dag view should use full window height, disable webuiservice in localmode.
-  TEZ-2079. Tez UI: trailing slash in timelineBaseUrl in ui should be handled.
-  TEZ-2069. Tez UI: appId should link to application in dag details view.
-  TEZ-2063. Tez UI: Flaky log url in tasks table.
-  TEZ-2062. Tez UI: Showing 50 elements not working properly.
-  TEZ-2056. Tez UI: fix VertexID filter,show only tez configs by default,fix appattemptid.
-  TEZ-2052. Tez UI: log view fixes, show version from build, better handling of ats url config.
-  TEZ-2043. Tez UI: add progress info from am webservice to dag and vertex views.
-  TEZ-2032. Update CHANGES.txt to show 0.6.0 is released
-  TEZ-2018. App Tracking and History URL should point to the Tez UI.
-  TEZ-2035. Make timeline server putDomain exceptions non-fatal - work-around
-  TEZ-1929. pre-empted tasks should be marked as killed instead of failed
-  TEZ-2017. TEZ UI - Dag view throwing error whild re-displaying additionals in some dags.
-  TEZ-2013. TEZ UI - App Details Page UI Nits
-  TEZ-2014. Tez UI: Nits : All tables, Vertices Page UI.
-  TEZ-2012. TEZ UI: Show page number in all tables, and display more readable task/attempt ids.
-  TEZ-1973. Dag View
-  TEZ-2010. History payload generated from conf has ${var} placeholders.
-  TEZ-1946. Tez UI: add source & sink views, add counters to vertices/all task views.
-  TEZ-1987. Tez UI non-standalone mode uses invalid protocol.
-  TEZ-1983. Tez UI swimlane task attempt link is broken
-
-Release 0.6.0: 2015-01-23
-
-INCOMPATIBLE CHANGES
-
-ALL CHANGES:
-  TEZ-1977. Fixup CHANGES.txt with Tez UI jiras
-  TEZ-1743. Add versions-maven-plugins artifacts to gitignore
-  TEZ-1968. Tez UI - All vertices of DAG are not listed in vertices page
-  TEZ-1890. tez-ui web.tar.gz also being uploaded to maven repository
-  TEZ-1938. Build warning duplicate jersey-json definitions
-  TEZ-1910. Build fails against hadoop-2.2.0.
-  TEZ-1882. Tez UI build does not work on Windows
-  TEZ-1915. Add public key to KEYS
-  TEZ-1907. Fix javadoc warnings in tez codebase
-  TEZ-1891. Incorrect number of Javadoc warnings reported
-  TEZ-1762. Lots of unit tests do not have timeout parameter set.
-  TEZ-1886. remove deprecation warnings for tez-ui on the console.
-  TEZ-1875. dropdown filters do not work on vertices and task attempts page.
-  TEZ-1873. TestTezAMRMClient fails due to host resolution timing out.
-  TEZ-1881. Setup initial test-patch script for TEZ-1313.
-  TEZ-1864. move initialization code dependent on config params to App.ready.
-  TEZ-1870. Time displayed in the UI is in GMT.
-  TEZ-1858. Docs for deploying/using the Tez UI.
-  TEZ-1859. TestGroupedSplits has commented out test: testGzip.
-  TEZ-1868. Document how to do Windows builds due to with ACL symlink build changes.
-  TEZ-1872. docs/src/site/custom/project-info-report.properties needs license header.
-  TEZ-1850. Enable deploy for tez-ui war.
-  TEZ-1841. Remove range versions for dependencies in tez-ui.
-  TEZ-1854. Failing tests due to host resolution timing out.
-  TEZ-1860. mvn apache-rat:check broken for tez-ui.
-  TEZ-1866. remove the "original" directory under tez-ui
-  TEZ-1591. Add multiDAG session test and move TestLocalMode to tez-tests
-  TEZ-1769. ContainerCompletedWhileRunningTransition should inherit from TerminatedWhileRunningTransition
-  TEZ-1849. Fix tez-ui war file licensing.
-  TEZ-1840. Document TezTaskOutput.
-  TEZ-1576. Class level comment in {{MiniTezCluster}} ends abruptly.
-  TEZ-1838. tez-ui/src/main/webapp/bower.json gets updated after compiling source code.
-  TEZ-1789. Move speculator processing off the central dispatcher.
-  TEZ-1610. Add additional task counters for fetchers, merger.
-  TEZ-1847. Fix package name for MiniTezClusterWithTimeline.
-  TEZ-1846. Build fails with package org.apache.tez.dag.history.logging.ats does not exist.
-  TEZ-1696. Make Tez use the domain-based timeline ACLs.
-  TEZ-1835. TestFaultTolerance#testRandomFailingTasks is timing out
-  TEZ-1832. Test

<TRUNCATED>

[07/50] [abbrv] tez git commit: TEZ-3615. Tez UI: Table changes (sree)

Posted by zh...@apache.org.
TEZ-3615. Tez UI: Table changes (sree)


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

Branch: refs/heads/TEZ-1190
Commit: e1b0b28128ff7ec99ddb46e955fa3b4b0583c068
Parents: 16b93de
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Thu Feb 9 17:12:48 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Thu Feb 9 17:12:48 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../main/webapp/app/controllers/home/queries.js | 20 +++++-----
 tez-ui/src/main/webapp/app/controllers/table.js |  4 +-
 tez-ui/src/main/webapp/package.json             |  2 +-
 .../tests/unit/controllers/home/queries-test.js |  4 +-
 .../webapp/tests/unit/controllers/table-test.js | 39 ++++++++++++++++++++
 6 files changed, 55 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 215cb08..6991f05 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -193,6 +193,7 @@ ALL CHANGES:
   TEZ-3594. Tez UI: Graphical view tooltip issues
   TEZ-3598. Tez UI: Text formatting changes
   TEZ-3602. Tez UI: Query Name field is not required
+  TEZ-3615. Tez UI: Table changes
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/tez-ui/src/main/webapp/app/controllers/home/queries.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js
index 648f98e..49da719 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/queries.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js
@@ -85,6 +85,7 @@ export default TableController.extend({
     headerTitle: 'Query ID',
     contentPath: 'entityID',
     cellComponentName: 'em-table-linked-cell',
+    minWidth: "250px",
     getCellContent: function (row) {
       return {
         routeName: "query",
@@ -96,11 +97,13 @@ export default TableController.extend({
     id: 'requestUser',
     headerTitle: 'User',
     contentPath: 'requestUser',
+    minWidth: "100px",
   },{
     id: 'status',
     headerTitle: 'Status',
     contentPath: 'status',
     cellComponentName: 'em-table-status-cell',
+    minWidth: "105px",
   },{
     id: 'queryText',
     headerTitle: 'Query',
@@ -110,6 +113,7 @@ export default TableController.extend({
     headerTitle: 'DAG ID',
     contentPath: 'dag.firstObject.entityID',
     cellComponentName: 'em-table-linked-cell',
+    minWidth: "250px",
     getCellContent: function (row) {
       return {
         routeName: "dag",
@@ -129,6 +133,7 @@ export default TableController.extend({
     id: 'clientAddress',
     headerTitle: 'Client Address',
     contentPath: 'clientAddress',
+    hiddenByDefault: true,
   },{
     id: 'startTime',
     headerTitle: 'Start Time',
@@ -166,22 +171,17 @@ export default TableController.extend({
     id: 'executionMode',
     headerTitle: 'Execution Mode',
     contentPath: 'executionMode',
+    minWidth: "100px",
   },{
     id: 'hiveAddress',
     headerTitle: 'Hive Server 2 Address',
-    contentPath: 'hiveAddress'
+    contentPath: 'hiveAddress',
+    hiddenByDefault: true,
   },{
     id: 'instanceType',
     headerTitle: 'Client Type',
-    contentPath: 'instanceType'
-  },{
-    id: 'sessionID',
-    headerTitle: 'Session ID',
-    contentPath: 'sessionID',
-  },{
-    id: 'threadName',
-    headerTitle: 'Thread Name',
-    contentPath: 'threadName',
+    contentPath: 'instanceType',
+    minWidth: "100px",
   }]),
 
   getCounterColumns: function () {

http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/tez-ui/src/main/webapp/app/controllers/table.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/table.js b/tez-ui/src/main/webapp/app/controllers/table.js
index f0bce1c..d472e02 100644
--- a/tez-ui/src/main/webapp/app/controllers/table.js
+++ b/tez-ui/src/main/webapp/app/controllers/table.js
@@ -63,8 +63,8 @@ export default AbstractController.extend({
     var visibleColumnIDs = this.get("localStorage").get(this.get("storageID")) || {};
 
     this.get('columns').forEach(function (config) {
-      if(visibleColumnIDs[config.id] !== false) {
-        visibleColumnIDs[config.id] = true;
+      if(visibleColumnIDs[config.id] === undefined) {
+        visibleColumnIDs[config.id] = !Ember.get(config, "hiddenByDefault");
       }
     });
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/tez-ui/src/main/webapp/package.json
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json
index 149c81a..8985f82 100644
--- a/tez-ui/src/main/webapp/package.json
+++ b/tez-ui/src/main/webapp/package.json
@@ -58,7 +58,7 @@
   },
   "dependencies": {
     "em-helpers": "0.8.0",
-    "em-table": "0.4.0",
+    "em-table": "0.6.0",
     "em-tgraph": "0.0.10"
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
index cf36c7a..ccec36c 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
@@ -37,11 +37,11 @@ test('Basic creation test', function(assert) {
 
   assert.ok(controller.breadcrumbs);
   assert.ok(controller.headerComponentNames);
-  assert.ok(controller.headerComponentNames.length, 3);
+  assert.equal(controller.headerComponentNames.length, 3);
 
   assert.ok(controller.definition);
   assert.ok(controller.columns);
-  assert.ok(controller.columns.length, 9);
+  assert.equal(controller.columns.length, 16);
 
   assert.ok(controller.getCounterColumns);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e1b0b281/tez-ui/src/main/webapp/tests/unit/controllers/table-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/table-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/table-test.js
index 110fec2..c12f156 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/table-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/table-test.js
@@ -63,3 +63,42 @@ test('Basic creation test', function(assert) {
   assert.ok(controller.actions.openColumnSelector);
   assert.ok(controller.actions.columnsSelected);
 });
+
+test('initVisibleColumns test', function(assert) {
+  let controller = this.subject({
+    send: Ember.K,
+    localStorage: Ember.Object.create(),
+    columns: []
+  });
+
+  controller.set("columns", [{
+    id: "c1",
+  }, {
+    id: "c2",
+  }, {
+    id: "c3",
+  }]);
+  controller.initVisibleColumns();
+  assert.equal(controller.get("visibleColumnIDs.c1"), true);
+  assert.equal(controller.get("visibleColumnIDs.c2"), true);
+  assert.equal(controller.get("visibleColumnIDs.c3"), true);
+
+  controller.set("columns", [{
+    id: "c1",
+    hiddenByDefault: true,
+  }, {
+    id: "c2",
+  }, {
+    id: "c3",
+    hiddenByDefault: true,
+  }]);
+  controller.initVisibleColumns();
+  assert.equal(controller.get("visibleColumnIDs.c1"), false);
+  assert.equal(controller.get("visibleColumnIDs.c2"), true);
+  assert.equal(controller.get("visibleColumnIDs.c3"), false);
+
+  controller.initVisibleColumns();
+  assert.equal(controller.get("visibleColumnIDs.c1"), false);
+  assert.equal(controller.get("visibleColumnIDs.c2"), true);
+  assert.equal(controller.get("visibleColumnIDs.c3"), false);
+});


[29/50] [abbrv] tez git commit: TEZ-3253. Remove special handling for last app attempt. Contributed by Akira Ajisaka.

Posted by zh...@apache.org.
TEZ-3253. Remove special handling for last app attempt. Contributed by Akira Ajisaka.


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

Branch: refs/heads/TEZ-1190
Commit: e84231ebc9f984b9ecfc2fd8ff489ddfc627092b
Parents: a5ffdea
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Mar 6 18:17:04 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Mar 6 18:17:04 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../java/org/apache/tez/client/LocalClient.java |  2 +-
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 23 ++------------------
 .../apache/tez/dag/app/MockDAGAppMaster.java    |  2 +-
 .../apache/tez/dag/app/TestDAGAppMaster.java    |  8 +++----
 5 files changed, 9 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e84231eb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4c28405..a3a74bc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3253. Remove special handling for last app attempt.
   TEZ-3648. IFile.Write#close has an extra output stream flush
   TEZ-3649. AsyncHttpConnection should add StopWatch start.
   TEZ-3647. Add a setting which lets Tez determine Xmx.

http://git-wip-us.apache.org/repos/asf/tez/blob/e84231eb/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index db7fc2c..6baea48 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -362,7 +362,7 @@ public class LocalClient extends FrameworkClient {
 
     return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
         new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs,
-        versionInfo.getVersion(), 1, credentials, jobUserName, amPluginDescriptorProto);
+        versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto);
   }
 
   private AMPluginDescriptorProto getPluginDescriptorInfo(Configuration conf,

http://git-wip-us.apache.org/repos/asf/tez/blob/e84231eb/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index de02c18..fc24f04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -268,7 +268,6 @@ public class DAGAppMaster extends AbstractService {
   private HistoryEventHandler historyEventHandler;
   private final Map<String, LocalResource> amResources = new HashMap<String, LocalResource>();
   private final Map<String, LocalResource> cumulativeAdditionalResources = new HashMap<String, LocalResource>();
-  private final int maxAppAttempts;
   private final List<String> diagnostics = new ArrayList<String>();
   private String containerLogs;
 
@@ -346,7 +345,7 @@ public class DAGAppMaster extends AbstractService {
   public DAGAppMaster(ApplicationAttemptId applicationAttemptId,
       ContainerId containerId, String nmHost, int nmPort, int nmHttpPort,
       Clock clock, long appSubmitTime, boolean isSession, String workingDirectory,
-      String [] localDirs, String[] logDirs, String clientVersion, int maxAppAttempts,
+      String [] localDirs, String[] logDirs, String clientVersion,
       Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) {
     super(DAGAppMaster.class.getName());
     this.clock = clock;
@@ -365,7 +364,6 @@ public class DAGAppMaster extends AbstractService {
     this.shutdownHandler = createShutdownHandler();
     this.dagVersionInfo = new TezDagVersionInfo();
     this.clientVersion = clientVersion;
-    this.maxAppAttempts = maxAppAttempts;
     this.amCredentials = credentials;
     this.amPluginDescriptorProto = pluginDescriptorProto;
     this.appMasterUgi = UserGroupInformation
@@ -461,8 +459,6 @@ public class DAGAppMaster extends AbstractService {
         TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK,
         TezConfiguration.TEZ_AM_DISABLE_CLIENT_VERSION_CHECK_DEFAULT);
 
-    isLastAMRetry = appAttemptID.getAttemptId() >= maxAppAttempts;
-
     // Check client - AM version compatibility
     LOG.info("Comparing client version with AM version"
         + ", clientVersion=" + clientVersion
@@ -591,13 +587,6 @@ public class DAGAppMaster extends AbstractService {
       addIfServiceDependency(taskSchedulerManager, webUIService);
     }
 
-    if (isLastAMRetry) {
-      LOG.info("AM will unregister as this is the last attempt"
-          + ", currentAttempt=" + appAttemptID.getAttemptId()
-          + ", maxAttempts=" + maxAppAttempts);
-      this.taskSchedulerManager.setShouldUnregisterFlag();
-    }
-
     dispatcher.register(AMSchedulerEventType.class,
         taskSchedulerManager);
     addIfServiceDependency(taskSchedulerManager, clientRpcServer);
@@ -2401,14 +2390,6 @@ public class DAGAppMaster extends AbstractService {
         clientVersion = VersionInfo.UNKNOWN;
       }
 
-      // TODO Should this be defaulting to 1. Was there a version of YARN where this was not setup ?
-      int maxAppAttempts = 1;
-      String maxAppAttemptsEnv = System.getenv(
-          ApplicationConstants.MAX_APP_ATTEMPTS_ENV);
-      if (maxAppAttemptsEnv != null) {
-        maxAppAttempts = Integer.parseInt(maxAppAttemptsEnv);
-      }
-
       validateInputParam(appSubmitTimeStr,
           ApplicationConstants.APP_SUBMIT_TIME_ENV);
 
@@ -2465,7 +2446,7 @@ public class DAGAppMaster extends AbstractService {
               System.getenv(Environment.PWD.name()),
               TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name())),
               TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOG_DIRS.name())),
-              clientVersion, maxAppAttempts, credentials, jobUserName, amPluginDescriptorProto);
+              clientVersion, credentials, jobUserName, amPluginDescriptorProto);
       ShutdownHookManager.get().addShutdownHook(
         new DAGAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e84231eb/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
index b021a36..893e03d 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -497,7 +497,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
       AtomicBoolean launcherGoFlag, boolean initFailFlag, boolean startFailFlag,
       Credentials credentials, String jobUserName, int handlerConcurrency, int numConcurrentContainers) {
     super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
-        isSession, workingDirectory, localDirs, logDirs,  new TezApiVersionInfo().getVersion(), 1,
+        isSession, workingDirectory, localDirs, logDirs,  new TezApiVersionInfo().getVersion(),
         credentials, jobUserName, null);
     shutdownHandler = new MockDAGAppMasterShutdownHandler();
     this.launcherGoFlag = launcherGoFlag;

http://git-wip-us.apache.org/repos/asf/tez/blob/e84231eb/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
index 56d1f96..570c6dc 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
@@ -109,7 +109,7 @@ public class TestDAGAppMaster {
     // finishing an in-progress DAG.
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 2);
-    DAGAppMasterForTest dam = new DAGAppMasterForTest(attemptId, true, 3);
+    DAGAppMasterForTest dam = new DAGAppMasterForTest(attemptId, true);
     TezConfiguration conf = new TezConfiguration(false);
     conf.setBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, false);
     dam.init(conf);
@@ -427,7 +427,7 @@ public class TestDAGAppMaster {
         "127.0.0.1", 0, 0, new SystemClock(), 1, true,
         TEST_DIR.toString(), new String[] {TEST_DIR.toString()},
         new String[] {TEST_DIR.toString()},
-        new TezApiVersionInfo().getVersion(), 1, amCreds,
+        new TezApiVersionInfo().getVersion(), amCreds,
         "someuser", null);
     am.init(conf);
     am.start();
@@ -544,11 +544,11 @@ public class TestDAGAppMaster {
     private DAGAppMasterShutdownHandler mockShutdown;
     private TaskSchedulerManager mockScheduler = mock(TaskSchedulerManager.class);
 
-    public DAGAppMasterForTest(ApplicationAttemptId attemptId, boolean isSession, int maxAttempts) {
+    public DAGAppMasterForTest(ApplicationAttemptId attemptId, boolean isSession) {
       super(attemptId, ContainerId.newContainerId(attemptId, 1), "hostname", 12345, 12346,
           new SystemClock(), 0, isSession, TEST_DIR.getAbsolutePath(),
           new String[] { TEST_DIR.getAbsolutePath() }, new String[] { TEST_DIR.getAbsolutePath() },
-          new TezDagVersionInfo().getVersion(), maxAttempts, createCredentials(), "jobname", null);
+          new TezDagVersionInfo().getVersion(), createCredentials(), "jobname", null);
     }
 
     private static Credentials createCredentials() {


[40/50] [abbrv] tez git commit: TEZ-1526. LoadingCache for TezTaskID slow for large jobs (jeagles)

Posted by zh...@apache.org.
TEZ-1526. LoadingCache for TezTaskID slow for large jobs (jeagles)


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

Branch: refs/heads/TEZ-1190
Commit: 57c857d267f17dd4e47b53d7691996d73c4476a1
Parents: 5f953bf
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Tue Mar 14 15:11:36 2017 -0500
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Tue Mar 14 15:11:36 2017 -0500

----------------------------------------------------------------------
 .../org/apache/tez/dag/records/TezDAGID.java    | 64 +++++++-------------
 .../java/org/apache/tez/dag/records/TezID.java  | 21 +++++++
 .../tez/dag/records/TezTaskAttemptID.java       | 57 ++++++-----------
 .../org/apache/tez/dag/records/TezTaskID.java   | 51 ++++++----------
 .../org/apache/tez/dag/records/TezVertexID.java | 48 ++++++---------
 ...tesianProductVertexManagerUnpartitioned.java |  6 +-
 6 files changed, 103 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
index b7a2c8f..2e3309e 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
@@ -21,15 +21,12 @@ package org.apache.tez.dag.records;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 
 import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import org.apache.tez.util.FastNumberFormat;
 
 /**
  * TezDAGID represents the immutable and unique identifier for
@@ -43,16 +40,7 @@ import com.google.common.cache.LoadingCache;
  */
 public class TezDAGID extends TezID {
 
-  private static LoadingCache<TezDAGID, TezDAGID> dagIdCache = CacheBuilder.newBuilder().softValues().
-      build(
-          new CacheLoader<TezDAGID, TezDAGID>() {
-            @Override
-            public TezDAGID load(TezDAGID key) throws Exception {
-              return key;
-            }
-          }
-      );
-  
+  private static TezIDCache<TezDAGID> tezDAGIDCache = new TezIDCache<>();
   private ApplicationId applicationId;
 
   /**
@@ -65,13 +53,12 @@ public class TezDAGID extends TezID {
     // will be short-lived.
     // Alternately the cache can be keyed by the hash of the incoming paramters.
     Preconditions.checkArgument(applicationId != null, "ApplicationID cannot be null");
-    return dagIdCache.getUnchecked(new TezDAGID(applicationId, id));
+    return tezDAGIDCache.getInstance(new TezDAGID(applicationId, id));
   }
 
   @InterfaceAudience.Private
   public static void clearCache() {
-    dagIdCache.invalidateAll();
-    dagIdCache.cleanUp();
+    tezDAGIDCache.clear();
   }
   
   /**
@@ -85,7 +72,7 @@ public class TezDAGID extends TezID {
     // will be short-lived.
     // Alternately the cache can be keyed by the hash of the incoming paramters.
     Preconditions.checkArgument(yarnRMIdentifier != null, "yarnRMIdentifier cannot be null");
-    return dagIdCache.getUnchecked(new TezDAGID(yarnRMIdentifier, appId, id));
+    return tezDAGIDCache.getInstance(new TezDAGID(yarnRMIdentifier, appId, id));
   }
   
   // Public for Writable serialization. Verify if this is actually required.
@@ -151,25 +138,14 @@ public class TezDAGID extends TezID {
 
   // DO NOT CHANGE THIS. DAGClient replicates this code to create DAG id string
   public static final String DAG = "dag";
-  static final ThreadLocal<NumberFormat> tezAppIdFormat = new ThreadLocal<NumberFormat>() {
+  static final ThreadLocal<FastNumberFormat> tezAppIdFormat = new ThreadLocal<FastNumberFormat>() {
     @Override
-    public NumberFormat initialValue() {
-      NumberFormat fmt = NumberFormat.getInstance();
-      fmt.setGroupingUsed(false);
+    public FastNumberFormat initialValue() {
+      FastNumberFormat fmt = FastNumberFormat.getInstance();
       fmt.setMinimumIntegerDigits(4);
       return fmt;
     }
   };
-  
-  static final ThreadLocal<NumberFormat> tezDagIdFormat = new ThreadLocal<NumberFormat>() {
-    @Override
-    public NumberFormat initialValue() {
-      NumberFormat fmt = NumberFormat.getInstance();
-      fmt.setGroupingUsed(false);
-      fmt.setMinimumIntegerDigits(1);
-      return fmt;
-    }
-  };
 
   @Override
   public String toString() {
@@ -190,10 +166,15 @@ public class TezDAGID extends TezID {
       throw new IllegalArgumentException("numDagsPerGroup has to be more than one. Got: " +
           numDagsPerGroup);
     }
-    return DAG_GROUPID_PREFIX + SEPARATOR +
-        getApplicationId().getClusterTimestamp() + SEPARATOR +
-        tezAppIdFormat.get().format(getApplicationId().getId()) + SEPARATOR +
-        tezDagIdFormat.get().format((getId() - 1) / numDagsPerGroup);
+    StringBuilder sb = new StringBuilder();
+    sb.append(DAG_GROUPID_PREFIX);
+    sb.append(SEPARATOR);
+    sb.append(getApplicationId().getClusterTimestamp());
+    sb.append(SEPARATOR);
+    tezAppIdFormat.get().format(getApplicationId().getId(), sb);
+    sb.append(SEPARATOR);
+    sb.append((id - 1) / numDagsPerGroup);
+    return sb.toString();
   }
 
   public static TezDAGID fromString(String dagId) {
@@ -225,12 +206,11 @@ public class TezDAGID extends TezID {
    * @return the builder that was passed in
    */
   protected StringBuilder appendTo(StringBuilder builder) {
-    return builder.append(SEPARATOR).
-                 append(applicationId.getClusterTimestamp()).
-                 append(SEPARATOR).
-                 append(tezAppIdFormat.get().format(applicationId.getId())).
-                 append(SEPARATOR).
-                 append(tezDagIdFormat.get().format(id));
+    builder.append(SEPARATOR);
+    builder.append(applicationId.getClusterTimestamp());
+    builder.append(SEPARATOR);
+    tezAppIdFormat.get().format(applicationId.getId(), builder);
+    return builder.append(SEPARATOR).append(id);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java
index 7efbd9a..cd7b27d 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java
@@ -21,6 +21,8 @@ package org.apache.tez.dag.records;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.lang.ref.WeakReference;
+import java.util.WeakHashMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -40,6 +42,25 @@ public abstract class TezID implements WritableComparable<TezID> {
   public static final char SEPARATOR = '_';
   protected int id;
 
+  public static class TezIDCache<T> {
+    private final WeakHashMap<T, WeakReference<T>> cache = new WeakHashMap<>();
+
+    synchronized T getInstance(final T id) {
+      final WeakReference<T> cached = cache.get(id);
+      if (cached != null) {
+        final T value = cached.get();
+        if (value != null)
+          return value;
+      }
+      cache.put(id, new WeakReference<T>(id));
+      return id;
+    }
+
+    synchronized void clear() {
+      cache.clear();
+    }
+  }
+
   /** constructs an ID object from the given int */
   public TezID(int id) {
     this.id = id;

http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java
index 296d577..7aee80f 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java
@@ -21,15 +21,10 @@ package org.apache.tez.dag.records;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
 /**
  * TezTaskAttemptID represents the immutable and unique identifier for
  * a task attempt. Each task attempt is one particular instance of a Tez Task
@@ -50,17 +45,9 @@ import com.google.common.cache.LoadingCache;
 public class TezTaskAttemptID extends TezID {
   public static final String ATTEMPT = "attempt";
   private TezTaskID taskId;
-  
-  private static LoadingCache<TezTaskAttemptID, TezTaskAttemptID> taskAttemptIDCache = CacheBuilder.newBuilder().softValues().
-      build(
-          new CacheLoader<TezTaskAttemptID, TezTaskAttemptID>() {
-            @Override
-            public TezTaskAttemptID load(TezTaskAttemptID key) throws Exception {
-              return key;
-            }
-          }
-      );
-  
+
+  private static TezIDCache<TezTaskAttemptID> tezTaskAttemptIDCache = new TezIDCache<>();
+
   // Public for Writable serialization. Verify if this is actually required.
   public TezTaskAttemptID() {
   }
@@ -71,13 +58,12 @@ public class TezTaskAttemptID extends TezID {
    * @param id the task attempt number
    */
   public static TezTaskAttemptID getInstance(TezTaskID taskID, int id) {
-    return taskAttemptIDCache.getUnchecked(new TezTaskAttemptID(taskID, id));
+    return tezTaskAttemptIDCache.getInstance(new TezTaskAttemptID(taskID, id));
   }
 
   @InterfaceAudience.Private
   public static void clearCache() {
-    taskAttemptIDCache.invalidateAll();
-    taskAttemptIDCache.cleanUp();
+    tezTaskAttemptIDCache.clear();
   }
 
   private TezTaskAttemptID(TezTaskID taskId, int id) {
@@ -108,7 +94,9 @@ public class TezTaskAttemptID extends TezID {
    * @return the builder that was passed in.
    */
   protected StringBuilder appendTo(StringBuilder builder) {
-    return taskId.appendTo(builder).append(SEPARATOR).append(id);
+    taskId.appendTo(builder);
+    builder.append(SEPARATOR);
+    return builder.append(id);
   }
   
   @Override
@@ -151,25 +139,20 @@ public class TezTaskAttemptID extends TezID {
     super.write(out);
   }
 
-  protected static final ThreadLocal<NumberFormat> tezTaskAttemptIdFormat = new ThreadLocal<NumberFormat>() {
-    @Override
-    public NumberFormat initialValue() {
-      NumberFormat fmt = NumberFormat.getInstance();
-      fmt.setGroupingUsed(false);
-      fmt.setMinimumIntegerDigits(1);
-      return fmt;
-    }
-  };
-
   public static TezTaskAttemptID fromString(String taIdStr) {
     try {
-      String[] split = taIdStr.split("_");
-      String rmId = split[1];
-      int appId = TezDAGID.tezAppIdFormat.get().parse(split[2]).intValue();
-      int dagId = TezDAGID.tezDagIdFormat.get().parse(split[3]).intValue();
-      int vId = TezVertexID.tezVertexIdFormat.get().parse(split[4]).intValue();
-      int taskId = TezTaskID.tezTaskIdFormat.get().parse(split[5]).intValue();
-      int id = tezTaskAttemptIdFormat.get().parse(split[6]).intValue();
+      int pos1 = taIdStr.indexOf(SEPARATOR);
+      int pos2 = taIdStr.indexOf(SEPARATOR, pos1 + 1);
+      int pos3 = taIdStr.indexOf(SEPARATOR, pos2 + 1);
+      int pos4 = taIdStr.indexOf(SEPARATOR, pos3 + 1);
+      int pos5 = taIdStr.indexOf(SEPARATOR, pos4 + 1);
+      int pos6 = taIdStr.indexOf(SEPARATOR, pos5 + 1);
+      String rmId = taIdStr.substring(pos1 + 1, pos2);
+      int appId = Integer.parseInt(taIdStr.substring(pos2 + 1, pos3));
+      int dagId = Integer.parseInt(taIdStr.substring(pos3 + 1, pos4));
+      int vId = Integer.parseInt(taIdStr.substring(pos4 + 1, pos5));
+      int taskId = Integer.parseInt(taIdStr.substring(pos5 + 1, pos6));
+      int id = Integer.parseInt(taIdStr.substring(pos6 + 1));
 
       return TezTaskAttemptID.getInstance(
           TezTaskID.getInstance(

http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
index 3d28348..3295f6a 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java
@@ -21,16 +21,12 @@ package org.apache.tez.dag.records;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
+import org.apache.tez.util.FastNumberFormat;
 
 /**
  * TaskID represents the immutable and unique identifier for
@@ -46,26 +42,16 @@ public class TezTaskID extends TezID {
   public static final String TASK = "task";
   private final int serializingHash;
   
-  static final ThreadLocal<NumberFormat> tezTaskIdFormat = new ThreadLocal<NumberFormat>() {
+  static final ThreadLocal<FastNumberFormat> tezTaskIdFormat = new ThreadLocal<FastNumberFormat>() {
     @Override
-    public NumberFormat initialValue() {
-      NumberFormat fmt = NumberFormat.getInstance();
-      fmt.setGroupingUsed(false);
+    public FastNumberFormat initialValue() {
+      FastNumberFormat fmt = FastNumberFormat.getInstance();
       fmt.setMinimumIntegerDigits(6);
       return fmt;
     }
   };
 
-  private static LoadingCache<TezTaskID, TezTaskID> taskIDCache = CacheBuilder.newBuilder().softValues().
-      build(
-          new CacheLoader<TezTaskID, TezTaskID>() {
-            @Override
-            public TezTaskID load(TezTaskID key) throws Exception {
-              return key;
-            }
-          }
-      );
-  
+  private static TezIDCache<TezTaskID> tezTaskIDCache = new TezIDCache<>();
   private TezVertexID vertexId;
 
   /**
@@ -75,13 +61,12 @@ public class TezTaskID extends TezID {
    */
   public static TezTaskID getInstance(TezVertexID vertexID, int id) {
     Preconditions.checkArgument(vertexID != null, "vertexID cannot be null");
-    return taskIDCache.getUnchecked(new TezTaskID(vertexID, id));
+    return tezTaskIDCache.getInstance(new TezTaskID(vertexID, id));
   }
 
   @InterfaceAudience.Private
   public static void clearCache() {
-    taskIDCache.invalidateAll();
-    taskIDCache.cleanUp();
+    tezTaskIDCache.clear();
   }
 
   private TezTaskID(TezVertexID vertexID, int id) {
@@ -130,9 +115,9 @@ public class TezTaskID extends TezID {
    * @return the builder that was passed in
    */
   protected StringBuilder appendTo(StringBuilder builder) {
-    return vertexId.appendTo(builder).
-                 append(SEPARATOR).
-                 append(tezTaskIdFormat.get().format(id));
+    vertexId.appendTo(builder);
+    builder.append(SEPARATOR);
+    return tezTaskIdFormat.get().format(id, builder);
   }
 
   @Override
@@ -170,12 +155,16 @@ public class TezTaskID extends TezID {
 
   public static TezTaskID fromString(String taskIdStr) {
     try {
-      String[] split = taskIdStr.split("_");
-      String rmId = split[1];
-      int appId = TezDAGID.tezAppIdFormat.get().parse(split[2]).intValue();
-      int dagId = TezDAGID.tezDagIdFormat.get().parse(split[3]).intValue();
-      int vId = TezVertexID.tezVertexIdFormat.get().parse(split[4]).intValue();
-      int id = tezTaskIdFormat.get().parse(split[5]).intValue();
+      int pos1 = taskIdStr.indexOf(SEPARATOR);
+      int pos2 = taskIdStr.indexOf(SEPARATOR, pos1 + 1);
+      int pos3 = taskIdStr.indexOf(SEPARATOR, pos2 + 1);
+      int pos4 = taskIdStr.indexOf(SEPARATOR, pos3 + 1);
+      int pos5 = taskIdStr.indexOf(SEPARATOR, pos4 + 1);
+      String rmId = taskIdStr.substring(pos1 + 1, pos2);
+      int appId = Integer.parseInt(taskIdStr.substring(pos2 + 1, pos3));
+      int dagId = Integer.parseInt(taskIdStr.substring(pos3 + 1, pos4));
+      int vId = Integer.parseInt(taskIdStr.substring(pos4 + 1, pos5));
+      int id = Integer.parseInt(taskIdStr.substring(pos5 + 1));
 
       return TezTaskID.getInstance(
               TezVertexID.getInstance(

http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java
index d30df16..b56c9ad 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java
@@ -21,15 +21,12 @@ package org.apache.tez.dag.records;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
+import org.apache.tez.util.FastNumberFormat;
 
 /**
  * TezVertexID represents the immutable and unique identifier for
@@ -46,27 +43,17 @@ import com.google.common.cache.LoadingCache;
 @InterfaceStability.Stable
 public class TezVertexID extends TezID {
   public static final String VERTEX = "vertex";
-  static final ThreadLocal<NumberFormat> tezVertexIdFormat = new ThreadLocal<NumberFormat>() {
+  static final ThreadLocal<FastNumberFormat> tezVertexIdFormat = new ThreadLocal<FastNumberFormat>() {
 
     @Override
-    public NumberFormat initialValue() {
-      NumberFormat fmt = NumberFormat.getInstance();
-      fmt.setGroupingUsed(false);
+    public FastNumberFormat initialValue() {
+      FastNumberFormat fmt = FastNumberFormat.getInstance();
       fmt.setMinimumIntegerDigits(2);
       return fmt;
     }
   };
 
-  private static LoadingCache<TezVertexID, TezVertexID> vertexIDCache = CacheBuilder.newBuilder().softValues().
-      build(
-          new CacheLoader<TezVertexID, TezVertexID>() {
-            @Override
-            public TezVertexID load(TezVertexID key) throws Exception {
-              return key;
-            }
-          }
-      );
-  
+  private static TezIDCache<TezVertexID> tezVertexIDCache = new TezIDCache<>();
   private TezDAGID dagId;
 
   // Public for Writable serialization. Verify if this is actually required.
@@ -80,13 +67,12 @@ public class TezVertexID extends TezID {
    */
   public static TezVertexID getInstance(TezDAGID dagId, int id) {
     Preconditions.checkArgument(dagId != null, "DagID cannot be null");
-    return vertexIDCache.getUnchecked(new TezVertexID(dagId, id));
+    return tezVertexIDCache.getInstance(new TezVertexID(dagId, id));
   }
 
   @InterfaceAudience.Private
   public static void clearCache() {
-    vertexIDCache.invalidateAll();
-    vertexIDCache.cleanUp();
+    tezVertexIDCache.clear();
   }
 
   private TezVertexID(TezDAGID dagId, int id) {
@@ -146,9 +132,9 @@ public class TezVertexID extends TezID {
    * @return the builder that was passed in
    */
   protected StringBuilder appendTo(StringBuilder builder) {
-    return dagId.appendTo(builder).
-        append(SEPARATOR).
-        append(tezVertexIdFormat.get().format(id));
+    dagId.appendTo(builder);
+    builder.append(SEPARATOR);
+    return tezVertexIdFormat.get().format(id, builder);
   }
 
   @Override
@@ -158,12 +144,14 @@ public class TezVertexID extends TezID {
 
   public static TezVertexID fromString(String vertexIdStr) {
     try {
-      String[] split = vertexIdStr.split("_");
-      String rmId = split[1];
-      int appId = TezDAGID.tezAppIdFormat.get().parse(split[2]).intValue();
-      int dagId = TezDAGID.tezDagIdFormat.get().parse(split[3]).intValue();
-      int id = tezVertexIdFormat.get().parse(split[4]).intValue();
-
+      int pos1 = vertexIdStr.indexOf(SEPARATOR);
+      int pos2 = vertexIdStr.indexOf(SEPARATOR, pos1 + 1);
+      int pos3 = vertexIdStr.indexOf(SEPARATOR, pos2 + 1);
+      int pos4 = vertexIdStr.indexOf(SEPARATOR, pos3 + 1);
+      String rmId = vertexIdStr.substring(pos1 + 1, pos2);
+      int appId = Integer.parseInt(vertexIdStr.substring(pos2 + 1, pos3));
+      int dagId = Integer.parseInt(vertexIdStr.substring(pos3 + 1, pos4));
+      int id = Integer.parseInt(vertexIdStr.substring(pos4 + 1));
       return TezVertexID.getInstance(
               TezDAGID.getInstance(rmId, appId, dagId),
               id);

http://git-wip-us.apache.org/repos/asf/tez/blob/57c857d2/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerUnpartitioned.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerUnpartitioned.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerUnpartitioned.java
index 31a3941..d2ce378 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerUnpartitioned.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerUnpartitioned.java
@@ -290,11 +290,10 @@ public class TestCartesianProductVertexManagerUnpartitioned {
     VertexManagerEvent vmEvent =
       VertexManagerEvent.create("cp vertex", proto.toByteString().asReadOnlyByteBuffer());
 
-    Formatter formatter = new Formatter();
     for (int i = 0; i < desiredBytesPerGroup/outputBytesPerTaskV0; i++) {
       vmEvent.setProducerAttemptIdentifier(
         new TaskAttemptIdentifierImpl("dag", "v0", TezTaskAttemptID.fromString(
-          formatter.format("attempt_1441301219877_0109_1_00_%06d_0", i).toString())));
+          String.format("attempt_1441301219877_0109_1_00_%06d_0", i))));
       vertexManager.onVertexManagerEventReceived(vmEvent);
     }
     verify(context, never()).reconfigureVertex(anyInt(), any(VertexLocationHint.class),
@@ -313,10 +312,9 @@ public class TestCartesianProductVertexManagerUnpartitioned {
         anyMapOf(String.class, EdgeProperty.class));
       vmEvent.setProducerAttemptIdentifier(
         new TaskAttemptIdentifierImpl("dag", "v1", TezTaskAttemptID.fromString(
-          formatter.format("attempt_1441301219877_0109_1_01_%06d_0", i).toString())));
+          String.format("attempt_1441301219877_0109_1_01_%06d_0", i))));
       vertexManager.onVertexManagerEventReceived(vmEvent);
     }
-    formatter.close();
     verify(context, times(1)).reconfigureVertex(parallelismCaptor.capture(),
       isNull(VertexLocationHint.class), edgePropertiesCaptor.capture());
     Map<String, EdgeProperty> edgeProperties = edgePropertiesCaptor.getValue();


[14/50] [abbrv] tez git commit: TEZ-3629. Tez UI: Enable the UI to display log links from LLAP (sree)

Posted by zh...@apache.org.
TEZ-3629. Tez UI: Enable the UI to display log links from LLAP (sree)


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

Branch: refs/heads/TEZ-1190
Commit: 391434246f6e3fba8db03cae8c8084895b16c504
Parents: 8c311e4
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Wed Feb 22 16:32:53 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Wed Feb 22 16:32:53 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 tez-ui/src/main/webapp/app/initializers/env.js  |  1 +
 tez-ui/src/main/webapp/app/models/abstract.js   |  1 +
 tez-ui/src/main/webapp/app/models/attempt.js    | 17 ++++++++++-
 .../src/main/webapp/app/serializers/attempt.js  | 15 ++--------
 .../webapp/tests/unit/models/abstract-test.js   | 27 +++++++++++++++++
 .../webapp/tests/unit/models/attempt-test.js    | 31 ++++++++++++++++++++
 .../tests/unit/serializers/attempt-test.js      |  9 +++---
 8 files changed, 84 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a5c59ca..54cddc8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -200,6 +200,7 @@ ALL CHANGES:
   TEZ-3602. Tez UI: Query Name field is not required
   TEZ-3615. Tez UI: Table changes
   TEZ-3619. Tez UI: Improve DAG Data download
+  TEZ-3629. Tez UI: Enable the UI to display log links from LLAP
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/app/initializers/env.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/initializers/env.js b/tez-ui/src/main/webapp/app/initializers/env.js
index 591d989..9edc74c 100644
--- a/tez-ui/src/main/webapp/app/initializers/env.js
+++ b/tez-ui/src/main/webapp/app/initializers/env.js
@@ -21,6 +21,7 @@ export function initialize(application) {
   application.inject('route', 'env', 'service:env');
   application.inject('adapter', 'env', 'service:env');
   application.inject('serializer', 'env', 'service:env');
+  application.inject('model', 'env', 'service:env');
 }
 
 export default {

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/app/models/abstract.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/abstract.js b/tez-ui/src/main/webapp/app/models/abstract.js
index 24bcbd3..1109914 100644
--- a/tez-ui/src/main/webapp/app/models/abstract.js
+++ b/tez-ui/src/main/webapp/app/models/abstract.js
@@ -52,6 +52,7 @@ export default DS.Model.extend({
       case "FINISHED":
       case "FAILED":
       case "KILLED":
+      case "ERROR":
         return true;
     }
     return false;

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/app/models/attempt.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/attempt.js b/tez-ui/src/main/webapp/app/models/attempt.js
index ab54b22..6e0b05e 100644
--- a/tez-ui/src/main/webapp/app/models/attempt.js
+++ b/tez-ui/src/main/webapp/app/models/attempt.js
@@ -72,7 +72,22 @@ export default AMTimelineModel.extend({
   containerID: DS.attr('string'),
   nodeID: DS.attr('string'),
 
-  logURL: DS.attr('string'),
+  inProgressLogsURL: DS.attr('string'),
+  completedLogsURL: DS.attr('string'),
+  logURL: Ember.computed("entityID", "inProgressLogsURL", "completedLogsURL", "dag.isComplete", function () {
+    var logURL = this.get("inProgressLogsURL");
+
+    if(logURL) {
+      if(logURL.indexOf("://") === -1) {
+        let attemptID = this.get("entityID"),
+            yarnProtocol = this.get('env.app.yarnProtocol');
+        return `${yarnProtocol}://${logURL}/syslog_${attemptID}`;
+      }
+      else { // LLAP log link
+        return this.get("dag.isComplete") ? this.get("completedLogsURL") : logURL;
+      }
+    }
+  }),
 
   containerLogURL: DS.attr('string'),
 });

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/app/serializers/attempt.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/serializers/attempt.js b/tez-ui/src/main/webapp/app/serializers/attempt.js
index 60f7a89..dd54135 100644
--- a/tez-ui/src/main/webapp/app/serializers/attempt.js
+++ b/tez-ui/src/main/webapp/app/serializers/attempt.js
@@ -20,21 +20,11 @@ import Ember from 'ember';
 
 import TimelineSerializer from './timeline';
 
-function createLogURL(source) {
-  var logURL = Ember.get(source, 'otherinfo.inProgressLogsURL'),
-      attemptID = Ember.get(source, 'entity'),
-      yarnProtocol = this.get('env.app.yarnProtocol');
-
-  if(logURL) {
-    return `${yarnProtocol}://${logURL}/syslog_${attemptID}`;
-  }
-}
-
 function createContainerLogURL(source) {
   var logURL = Ember.get(source, 'otherinfo.inProgressLogsURL'),
       yarnProtocol = this.get('env.app.yarnProtocol');
 
-  if(logURL) {
+  if(logURL && logURL.indexOf("://") === -1) {
     return `${yarnProtocol}://${logURL}`;
   }
 }
@@ -48,7 +38,8 @@ export default TimelineSerializer.extend({
     containerID: 'otherinfo.containerId',
     nodeID: 'otherinfo.nodeId',
 
-    logURL: createLogURL,
+    inProgressLogsURL: "otherinfo.inProgressLogsURL",
+    completedLogsURL: "otherinfo.completedLogsURL",
 
     containerLogURL: createContainerLogURL
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/tests/unit/models/abstract-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/abstract-test.js b/tez-ui/src/main/webapp/tests/unit/models/abstract-test.js
index bd6f141..5a012f5 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/abstract-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/abstract-test.js
@@ -41,6 +41,33 @@ test('Basic test for existence', function(assert) {
   assert.ok(model.isComplete);
 });
 
+test('isComplete test', function(assert) {
+  let model = this.subject();
+  assert.equal(model.get("isComplete"), false);
+
+  Ember.run(function () {
+    model.set("status", "SUCCEEDED");
+    assert.equal(model.get("isComplete"), true);
+
+    model.set("status", null);
+    assert.equal(model.get("isComplete"), false);
+    model.set("status", "FINISHED");
+    assert.equal(model.get("isComplete"), true);
+
+    model.set("status", null);
+    model.set("status", "FAILED");
+    assert.equal(model.get("isComplete"), true);
+
+    model.set("status", null);
+    model.set("status", "KILLED");
+    assert.equal(model.get("isComplete"), true);
+
+    model.set("status", null);
+    model.set("status", "ERROR");
+    assert.equal(model.get("isComplete"), true);
+  });
+});
+
 test('_notifyProperties test - will fail if _notifyProperties implementation is changed in ember-data', function(assert) {
   let model = this.subject();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/tests/unit/models/attempt-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/attempt-test.js b/tez-ui/src/main/webapp/tests/unit/models/attempt-test.js
index faa27ad..a15e3c9 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/attempt-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/attempt-test.js
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+import Ember from 'ember';
 import { moduleForModel, test } from 'ember-qunit';
 
 moduleForModel('attempt', 'Unit | Model | attempt', {
@@ -44,7 +45,10 @@ test('Basic creation test', function(assert) {
   assert.ok(model.containerID);
   assert.ok(model.nodeID);
 
+  assert.ok(model.inProgressLogsURL);
+  assert.ok(model.completedLogsURL);
   assert.ok(model.logURL);
+  assert.ok(model.containerLogURL);
 });
 
 test('index test', function(assert) {
@@ -76,3 +80,30 @@ test('vertexName test', function(assert) {
 
   assert.equal(model.get("vertexName"), testVertexName);
 });
+
+test('logURL test', function(assert) {
+  let model = this.subject({
+        entityID: "id_1",
+        dag: Ember.Object.create(),
+        env: {
+          app: {
+            yarnProtocol: "ptcl"
+          }
+        },
+        completedLogsURL: "http://abc.com/completed/link.log.done"
+      });
+
+  Ember.run(function () {
+    // Normal Tez log link
+    model.set("inProgressLogsURL", "abc.com/test/link");
+    assert.equal(model.get("logURL"), "ptcl://abc.com/test/link/syslog_id_1");
+
+    // LLAP log link - In Progress
+    model.set("inProgressLogsURL", "http://abc.com/in-progress/link.log");
+    assert.equal(model.get("logURL"), "http://abc.com/in-progress/link.log");
+
+    // LLAP log link - Completed
+    model.set("dag.isComplete", true);
+    assert.equal(model.get("logURL"), "http://abc.com/completed/link.log.done");
+  });
+});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/39143424/tez-ui/src/main/webapp/tests/unit/serializers/attempt-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/attempt-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/attempt-test.js
index aaeab62..4715acd 100644
--- a/tez-ui/src/main/webapp/tests/unit/serializers/attempt-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/serializers/attempt-test.js
@@ -27,10 +27,10 @@ test('Basic creation test', function(assert) {
   let serializer = this.subject();
 
   assert.ok(serializer);
-  assert.ok(serializer.maps.logURL);
+  assert.equal(Object.keys(serializer.maps).length, 8 + 8);
 });
 
-test('logURL test', function(assert) {
+test('containerLogURL test', function(assert) {
   let serializer = this.subject({
     env: {
       app: {
@@ -39,10 +39,9 @@ test('logURL test', function(assert) {
     }
   });
 
-  assert.equal(serializer.maps.logURL.call(serializer, {
-    entity: "id_1",
+  assert.equal(serializer.maps.containerLogURL.call(serializer, {
     otherinfo: {
       inProgressLogsURL: "abc.com/test/link",
     }
-  }), "ptcl://abc.com/test/link/syslog_id_1");
+  }), "ptcl://abc.com/test/link");
 });


[06/50] [abbrv] tez git commit: TEZ-3267. Publish queue name to ATS as part of dag summary. Contributed by Harish Jaiprakash.

Posted by zh...@apache.org.
TEZ-3267. Publish queue name to ATS as part of dag summary. Contributed by Harish Jaiprakash.


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

Branch: refs/heads/TEZ-1190
Commit: 16b93de8f31a815cab63e0be0dc563549a688566
Parents: 11815a7
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 8 18:32:26 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Feb 8 18:32:26 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/tez/common/ATSConstants.java     |  1 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 11 ++++++++-
 .../dag/history/events/DAGSubmittedEvent.java   | 17 +++++++++++--
 .../impl/HistoryEventJsonConversion.java        |  3 +++
 tez-dag/src/main/proto/HistoryEvents.proto      |  1 +
 .../apache/tez/dag/app/TestRecoveryParser.java  | 26 ++++++++++----------
 .../dag/history/TestHistoryEventHandler.java    |  2 +-
 .../TestHistoryEventsProtoConversion.java       |  4 ++-
 .../impl/TestHistoryEventJsonConversion.java    | 12 +++++++--
 .../history/recovery/TestRecoveryService.java   |  2 +-
 .../ats/acls/TestATSHistoryWithACLs.java        |  4 +--
 .../ats/TestATSV15HistoryLoggingService.java    |  2 +-
 .../ats/HistoryEventTimelineConversion.java     |  3 +++
 .../ats/TestATSHistoryLoggingService.java       |  2 +-
 .../ats/TestHistoryEventTimelineConversion.java | 10 +++++---
 16 files changed, 72 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0949339..215cb08 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3267. Publish queue name to ATS as part of dag summary.
   TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3581. Add different logger to enable suppressing logs for specific lines.

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
index c56582c..03c9fa1 100644
--- a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
@@ -55,6 +55,7 @@ public class ATSConstants {
   public static final String DAG_PLAN = "dagPlan";
   public static final String DAG_NAME = "dagName";
   public static final String DAG_STATE = "dagState";
+  public static final String DAG_SUBMITTED_QUEUE_NAME = "submittedQueueName";
   public static final String DAG_AM_WEB_SERVICE_VERSION = "amWebServiceVersion";
   public static final String RECOVERY_FAILURE_REASON = "recoveryFailureReason";
   public static final String VERTEX_NAME = "vertexName";

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index eaaf18b..7f27064 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -2578,7 +2578,7 @@ public class DAGAppMaster extends AbstractService {
     // for an app later
     final DAGSubmittedEvent submittedEvent = new DAGSubmittedEvent(newDAG.getID(),
         submitTime, dagPlan, this.appAttemptID, cumulativeAdditionalResources,
-        newDAG.getUserName(), newDAG.getConf(), containerLogs);
+        newDAG.getUserName(), newDAG.getConf(), containerLogs, getSubmittedQueueName());
     boolean dagLoggingEnabled = newDAG.getConf().getBoolean(
         TezConfiguration.TEZ_DAG_HISTORY_LOGGING_ENABLED,
         TezConfiguration.TEZ_DAG_HISTORY_LOGGING_ENABLED_DEFAULT);
@@ -2671,6 +2671,15 @@ public class DAGAppMaster extends AbstractService {
     });
   }
 
+  private String getSubmittedQueueName() {
+    // TODO: Replace this with constant once the yarn patch is backported. (JIRA: TEZ-3279)
+    String submittedQueueName = System.getenv("YARN_RESOURCEMANAGER_APPLICATION_QUEUE");
+    if (submittedQueueName == null) {
+      submittedQueueName = amConf.get(TezConfiguration.TEZ_QUEUE_NAME);
+    }
+    return submittedQueueName;
+  }
+
   @SuppressWarnings("unchecked")
   private void sendEvent(Event<?> event) {
     dispatcher.getEventHandler().handle(event);

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
index 07d7c07..1b1fdf3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
@@ -57,6 +57,7 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
   private boolean historyLoggingEnabled = true;
   private Configuration conf;
   private String containerLogs;
+  private String queueName;
 
   public DAGSubmittedEvent() {
   }
@@ -64,7 +65,7 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
   public DAGSubmittedEvent(TezDAGID dagID, long submitTime,
       DAGProtos.DAGPlan dagPlan, ApplicationAttemptId applicationAttemptId,
       Map<String, LocalResource> cumulativeAdditionalLocalResources,
-      String user, Configuration conf, String containerLogs) {
+      String user, Configuration conf, String containerLogs, String queueName) {
     this.dagID = dagID;
     this.dagName = dagPlan.getName();
     this.submitTime = submitTime;
@@ -74,6 +75,7 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
     this.user = user;
     this.conf = conf;
     this.containerLogs = containerLogs;
+    this.queueName = queueName;
   }
 
   @Override
@@ -97,6 +99,9 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
         .setApplicationAttemptId(applicationAttemptId.toString())
         .setDagPlan(dagPlan)
         .setSubmitTime(submitTime);
+    if (queueName != null) {
+      builder.setQueueName(queueName);
+    }
     if (cumulativeAdditionalLocalResources != null && !cumulativeAdditionalLocalResources.isEmpty()) {
       builder.setCumulativeAdditionalAmResources(DagTypeConverters
           .convertFromLocalResources(cumulativeAdditionalLocalResources));
@@ -111,6 +116,9 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
     this.submitTime = proto.getSubmitTime();
     this.applicationAttemptId = ConverterUtils.toApplicationAttemptId(
         proto.getApplicationAttemptId());
+    if (proto.hasQueueName()) {
+      this.queueName = proto.getQueueName();
+    }
     if (proto.hasCumulativeAdditionalAmResources()) {
       this.cumulativeAdditionalLocalResources = DagTypeConverters.convertFromPlanLocalResources(proto
           .getCumulativeAdditionalAmResources());
@@ -134,7 +142,8 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
   @Override
   public String toString() {
     return "dagID=" + dagID
-        + ", submitTime=" + submitTime;
+        + ", submitTime=" + submitTime
+        + ", queueName=" + queueName;
   }
 
   @Override
@@ -203,4 +212,8 @@ public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent {
   public String getContainerLogs() {
     return containerLogs;
   }
+
+  public String getQueueName() {
+    return queueName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
index a767fbf..69c40e4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
@@ -515,6 +515,9 @@ public class HistoryEventJsonConversion {
       otherInfo.put(ATSConstants.CALLER_CONTEXT_TYPE,
           event.getDAGPlan().getCallerContext().getCallerType());
     }
+    if (event.getQueueName() != null) {
+      otherInfo.put(ATSConstants.DAG_SUBMITTED_QUEUE_NAME, event.getQueueName());
+    }
     jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
 
     return jsonObject;

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/main/proto/HistoryEvents.proto
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto
index ff3707d..7671469 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -54,6 +54,7 @@ message DAGSubmittedProto {
   optional int64 submit_time = 3;
   optional string application_attempt_id = 4;
   optional PlanLocalResourcesProto cumulative_additional_am_resources = 5;
+  optional string queue_name = 6;
 }
 
 message DAGInitializedProto {

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
index f4edf9e..6673b39 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
@@ -169,7 +169,7 @@ public class TestRecoveryParser {
     rService.start();
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGInitializedEvent(dagID, 1L, "user", dagPlan.getName(), null)));
     // only for testing, DAGCommitStartedEvent is not supposed to happen at this time.
@@ -215,7 +215,7 @@ public class TestRecoveryParser {
     rService.start();
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGInitializedEvent(dagID, 1L, "user", dagPlan.getName(), null)));
     rService.handle(new DAGHistoryEvent(dagID,
@@ -264,7 +264,7 @@ public class TestRecoveryParser {
     rService.start();
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // wait until DAGSubmittedEvent is handled in the RecoveryEventHandling thread
     rService.await();
     rService.outputStreamMap.get(dagID).writeUTF("INVALID_DATA");
@@ -310,7 +310,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // write an corrupted SummaryEvent
     rService.summaryStream.writeChars("INVALID_DATA");
     rService.stop();
@@ -344,7 +344,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGCommitStartedEvent(dagID, 0L)));
@@ -376,7 +376,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGCommitStartedEvent(dagID, 0L)));
@@ -412,7 +412,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     rService.handle(new DAGHistoryEvent(dagID,
         new VertexCommitStartedEvent(TezVertexID.getInstance(dagID, 0), 0L)));
@@ -445,7 +445,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
     rService.handle(new DAGHistoryEvent(dagID,
@@ -482,7 +482,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     rService.handle(new DAGHistoryEvent(dagID,
         new VertexGroupCommitStartedEvent(dagID, "group_1", 
@@ -516,7 +516,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     TezVertexID v0 = TezVertexID.getInstance(dagID, 0);
     TezVertexID v1 = TezVertexID.getInstance(dagID, 1);
@@ -565,7 +565,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
     rService.handle(new DAGHistoryEvent(dagID,
@@ -601,7 +601,7 @@ public class TestRecoveryParser {
     // write a DAGSubmittedEvent first to initialize summaryStream
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     // It should be fine to skip other events, just for testing.
     TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
     rService.handle(new DAGHistoryEvent(dagID,
@@ -640,7 +640,7 @@ public class TestRecoveryParser {
     // DAG  DAGSubmittedEvent -> DAGInitializedEvent -> DAGStartedEvent
     rService.handle(new DAGHistoryEvent(dagID,
         new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
-            null, "user", new Configuration(), null)));
+            null, "user", new Configuration(), null, null)));
     DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagID, 100L, 
         "user", "dagName", null);
     DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagID, 0L, "user", "dagName");

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
index 4c0fe3f..5a71a42 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/TestHistoryEventHandler.java
@@ -219,7 +219,7 @@ public class TestHistoryEventHandler {
         new AMStartedEvent(attemptId, time, user)));
     historyEvents.add(new DAGHistoryEvent(dagId,
         new DAGSubmittedEvent(dagId, time, DAGPlan.getDefaultInstance(), attemptId, null, user,
-            conf, null)));
+            conf, null, "default")));
     TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
     historyEvents.add(new DAGHistoryEvent(dagId,
         new VertexStartedEvent(vertexID, time, time)));

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 3d29a5d..47d8389 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -171,12 +171,13 @@ public class TestHistoryEventsProtoConversion {
     logEvents(event, deserializedEvent);
   }
 
+  private final String QUEUE_NAME = "TEST_QUEUE_NAME";
   private void testDAGSubmittedEvent() throws Exception {
     DAGSubmittedEvent event = new DAGSubmittedEvent(TezDAGID.getInstance(
         ApplicationId.newInstance(0, 1), 1), 1001l,
         DAGPlan.newBuilder().setName("foo").build(),
         ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(0, 1), 1), null, "", null, null);
+            ApplicationId.newInstance(0, 1), 1), null, "", null, null, QUEUE_NAME);
     DAGSubmittedEvent deserializedEvent = (DAGSubmittedEvent)
         testProtoConversion(event);
     Assert.assertEquals(event.getApplicationAttemptId(),
@@ -189,6 +190,7 @@ public class TestHistoryEventsProtoConversion {
         deserializedEvent.getSubmitTime());
     Assert.assertEquals(event.getDAGPlan(),
         deserializedEvent.getDAGPlan());
+    Assert.assertEquals(event.getQueueName(), deserializedEvent.getQueueName());
     logEvents(event, deserializedEvent);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
index 9477118..1bbecd3 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
@@ -124,7 +124,7 @@ public class TestHistoryEventJsonConversion {
           break;
         case DAG_SUBMITTED:
           event = new DAGSubmittedEvent(tezDAGID, random.nextInt(), dagPlan, applicationAttemptId,
-              null, user, null, null);
+              null, user, null, null, "Q_" + eventType.name());
           break;
         case DAG_INITIALIZED:
           event = new DAGInitializedEvent(tezDAGID, random.nextInt(), user, dagPlan.getName(), null);
@@ -200,7 +200,15 @@ public class TestHistoryEventJsonConversion {
       if (event == null || !event.isHistoryEvent()) {
         continue;
       }
-      HistoryEventJsonConversion.convertToJson(event);
+      JSONObject json = HistoryEventJsonConversion.convertToJson(event);
+      if (eventType == HistoryEventType.DAG_SUBMITTED) {
+        try {
+          Assert.assertEquals("Q_" + eventType.name(), json.getJSONObject(ATSConstants.OTHER_INFO)
+              .getString(ATSConstants.DAG_SUBMITTED_QUEUE_NAME));
+        } catch (JSONException ex) {
+          Assert.fail("Exception: " + ex.getMessage() + " for type: " + eventType);
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java
index 3dec1d7..790e2d8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java
@@ -333,7 +333,7 @@ public class TestRecoveryService {
     DAGPlan dagPlan = DAGPlan.newBuilder().setName("test_dag").build();
     // This writes to recovery immediately.
     recoveryService.handle(new DAGHistoryEvent(dagId, new DAGSubmittedEvent(
-        dagId, startTime, dagPlan, appAttemptId, null, "nobody", conf, null)));
+        dagId, startTime, dagPlan, appAttemptId, null, "nobody", conf, null, "default")));
     waitForDrain(-1);
     verify(summaryFos, times(1)).hflush();
     verify(dagFos, times(1)).hflush();

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java
index 6b3ebd7..8e5c95c 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java
+++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java
@@ -401,7 +401,7 @@ public class TestATSHistoryWithACLs {
     DAGPlan dagPlan = DAGPlan.newBuilder().setName("DAGPlanMock").build();
     DAGSubmittedEvent submittedEvent = new DAGSubmittedEvent(tezDAGID,
           1, dagPlan, appAttemptId, null,
-          "usr", tezConf, null);
+          "usr", tezConf, null, null);
     submittedEvent.setHistoryLoggingEnabled(false);
     DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent);
     historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent));
@@ -446,7 +446,7 @@ public class TestATSHistoryWithACLs {
     DAGPlan dagPlan = DAGPlan.newBuilder().setName("DAGPlanMock").build();
     DAGSubmittedEvent submittedEvent = new DAGSubmittedEvent(tezDAGID,
             1, dagPlan, appAttemptId, null,
-            "usr", tezConf, null);
+            "usr", tezConf, null, null);
     submittedEvent.setHistoryLoggingEnabled(true);
     DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent);
     historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent));

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
index 9111195..cbded35 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
+++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java
@@ -450,7 +450,7 @@ public class TestATSV15HistoryLoggingService {
         new AMStartedEvent(attemptId, time, user)));
     historyEvents.add(new DAGHistoryEvent(dagId,
         new DAGSubmittedEvent(dagId, time, DAGPlan.getDefaultInstance(), attemptId, null, user,
-            conf, null)));
+            conf, null, "default")));
     TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
     historyEvents.add(new DAGHistoryEvent(dagId,
         new VertexStartedEvent(vertexID, time, time)));

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
index 96239c3..8d0c547 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
@@ -423,6 +423,9 @@ public class HistoryEventTimelineConversion {
       atsEntity.addOtherInfo(ATSConstants.CALLER_CONTEXT_TYPE,
           event.getDAGPlan().getCallerContext().getCallerType());
     }
+    if (event.getQueueName() != null) {
+      atsEntity.addOtherInfo(ATSConstants.DAG_SUBMITTED_QUEUE_NAME, event.getQueueName());
+    }
 
     return atsEntity;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java
index da57eb2..a641cda 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java
@@ -444,7 +444,7 @@ public class TestATSHistoryLoggingService {
     Configuration conf = new Configuration(service.getConfig());
     historyEvents.add(new DAGHistoryEvent(null, new AMStartedEvent(attemptId, time, "user")));
     historyEvents.add(new DAGHistoryEvent(dagId, new DAGSubmittedEvent(dagId, time,
-        DAGPlan.getDefaultInstance(), attemptId, null, "user", conf, null)));
+        DAGPlan.getDefaultInstance(), attemptId, null, "user", conf, null, "default")));
     TezVertexID vertexID = TezVertexID.getInstance(dagId, 1);
     historyEvents.add(new DAGHistoryEvent(dagId, new VertexStartedEvent(vertexID, time, time)));
     TezTaskID tezTaskID = TezTaskID.getInstance(vertexID, 1);

http://git-wip-us.apache.org/repos/asf/tez/blob/16b93de8/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index 62fb335..bb189d3 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -145,7 +145,7 @@ public class TestHistoryEventTimelineConversion {
           break;
         case DAG_SUBMITTED:
           event = new DAGSubmittedEvent(tezDAGID, random.nextInt(), dagPlan, applicationAttemptId,
-              null, user, null, containerLogs);
+              null, user, null, containerLogs, null);
           break;
         case DAG_INITIALIZED:
           event = new DAGInitializedEvent(tezDAGID, random.nextInt(), user, dagPlan.getName(), null);
@@ -473,8 +473,9 @@ public class TestHistoryEventTimelineConversion {
   public void testConvertDAGSubmittedEvent() {
     long submitTime = random.nextLong();
 
+    final String queueName = "TEST_DAG_SUBMITTED";
     DAGSubmittedEvent event = new DAGSubmittedEvent(tezDAGID, submitTime, dagPlan,
-        applicationAttemptId, null, user, null, containerLogs);
+        applicationAttemptId, null, user, null, containerLogs, queueName);
 
     TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
     Assert.assertEquals(EntityTypes.TEZ_DAG_ID.name(), timelineEntity.getEntityType());
@@ -509,7 +510,7 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(ATSConstants.USER).contains(user));
 
-    Assert.assertEquals(8, timelineEntity.getOtherInfo().size());
+    Assert.assertEquals(9, timelineEntity.getOtherInfo().size());
     Assert.assertTrue(timelineEntity.getOtherInfo().containsKey(ATSConstants.DAG_PLAN));
     Assert.assertEquals(applicationId.toString(),
         timelineEntity.getOtherInfo().get(ATSConstants.APPLICATION_ID));
@@ -530,7 +531,8 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals(
         timelineEntity.getOtherInfo().get(ATSConstants.CALLER_CONTEXT_TYPE),
             dagPlan.getCallerContext().getCallerType());
-
+    Assert.assertEquals(
+        queueName, timelineEntity.getOtherInfo().get(ATSConstants.DAG_SUBMITTED_QUEUE_NAME));
 
   }
 


[31/50] [abbrv] tez git commit: TEZ-3503. Tez UI: Support search by queue name (sree)

Posted by zh...@apache.org.
TEZ-3503. Tez UI: Support search by queue name (sree)


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

Branch: refs/heads/TEZ-1190
Commit: d40f3ad719e6713d4b1dbf1dc1e9646f79f21c3d
Parents: af0d4d7
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Tue Mar 7 17:12:52 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Tue Mar 7 17:12:52 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 tez-ui/src/main/webapp/app/adapters/timeline.js |  1 +
 .../webapp/app/components/dags-page-search.js   |  2 ++
 .../main/webapp/app/controllers/home/index.js   | 23 ++++++++++++++----
 tez-ui/src/main/webapp/app/entities/entity.js   |  4 ++++
 tez-ui/src/main/webapp/app/models/dag.js        | 10 ++++++--
 tez-ui/src/main/webapp/app/routes/home/index.js |  5 +++-
 tez-ui/src/main/webapp/app/serializers/dag.js   |  4 +++-
 .../webapp/app/styles/dags-page-search.less     |  4 ++--
 .../templates/components/dags-page-search.hbs   |  8 +++++++
 .../src/main/webapp/app/templates/dag/index.hbs |  4 ++++
 .../main/webapp/app/templates/home/index.hbs    |  5 ++++
 tez-ui/src/main/webapp/package.json             |  2 +-
 .../components/dags-page-search-test.js         |  4 ++--
 .../webapp/tests/unit/adapters/timeline-test.js |  2 +-
 .../tests/unit/controllers/home/index-test.js   | 16 +++++++++++++
 .../webapp/tests/unit/entities/entity-test.js   | 25 ++++++++++++++++++++
 .../main/webapp/tests/unit/models/dag-test.js   | 18 ++++++++++++++
 .../webapp/tests/unit/routes/home/index-test.js |  2 +-
 .../webapp/tests/unit/serializers/dag-test.js   |  2 ++
 20 files changed, 127 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1d2af07..248f329 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -215,6 +215,7 @@ ALL CHANGES:
   TEZ-3639. Tez UI: Footer pagination is improper in landing page
   TEZ-3640. Tez UI: Add associated llap application id to queries page
   TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
+  TEZ-3503. Tez UI: Support search by queue name
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/adapters/timeline.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/adapters/timeline.js b/tez-ui/src/main/webapp/app/adapters/timeline.js
index 41cbd1c..2d8f474 100644
--- a/tez-ui/src/main/webapp/app/adapters/timeline.js
+++ b/tez-ui/src/main/webapp/app/adapters/timeline.js
@@ -41,6 +41,7 @@ export default AbstractAdapter.extend({
     requestuser: "requestuser",
     executionMode: "executionmode",
     callerId: "callerId",
+    queueName: "queueName",
 
     tablesRead: "tablesread",
     tablesWritten: "tableswritten",

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/components/dags-page-search.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/components/dags-page-search.js b/tez-ui/src/main/webapp/app/components/dags-page-search.js
index f1cc71b..0adbf84 100644
--- a/tez-ui/src/main/webapp/app/components/dags-page-search.js
+++ b/tez-ui/src/main/webapp/app/components/dags-page-search.js
@@ -27,6 +27,7 @@ export default Ember.Component.extend({
   status: Ember.computed.oneWay("tableDefinition.status"),
   appID: Ember.computed.oneWay("tableDefinition.appID"),
   callerID: Ember.computed.oneWay("tableDefinition.callerID"),
+  queue: Ember.computed.oneWay("tableDefinition.queue"),
 
   sendSearch: function () {
     this.get('parentView').sendAction('search', {
@@ -36,6 +37,7 @@ export default Ember.Component.extend({
       status: this.get("status"),
       appID: this.get("appID"),
       callerID: this.get("callerID"),
+      queue: this.get("queue"),
     });
   },
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/controllers/home/index.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js
index 9194fd6..74050e3 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/index.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/index.js
@@ -24,13 +24,16 @@ import TableDefinition from 'em-table/utils/table-definition';
 
 export default TableController.extend({
 
-  queryParams: ["dagName", "dagID", "submitter", "status", "appID", "callerID", "appid", "id", "user", "dag_name"],
+  queryParams: ["dagName", "dagID", "submitter", "status", "appID", "callerID", "queue",
+      "appid", "id", "user", "dag_name"],
   dagName: "",
   dagID: "",
   submitter: "",
   status: "",
   appID: "",
   callerID: "",
+  queue: "",
+
   appid: "",
   id: "",
   user: "",
@@ -53,8 +56,8 @@ export default TableController.extend({
 
   _definition: TableDefinition.create(),
   // Using computed, as observer won't fire if the property is not used
-  definition: Ember.computed("dagName", "dagID", "submitter", "status",
-      "appID", "callerID", "pageNum", "moreAvailable", "loadingMore", function () {
+  definition: Ember.computed("dagName", "dagID", "submitter", "status", "appID", "callerID", "queue",
+      "pageNum", "moreAvailable", "loadingMore", function () {
 
     var definition = this.get("_definition");
     if (!this.get("appID")) {
@@ -81,6 +84,7 @@ export default TableController.extend({
       status: this.get("status"),
       appID: this.get("appID"),
       callerID: this.get("callerID"),
+      queue: this.get("queue"),
 
       pageNum: this.get("pageNum"),
 
@@ -155,7 +159,18 @@ export default TableController.extend({
   },{
     id: 'queue',
     headerTitle: 'Queue',
-    contentPath: 'queue'
+    contentPath: 'queue',
+    observePath: true,
+    getCellContent: function (row) {
+      var queueName = row.get("queue");
+      if(!row.get("queueName") && row.get("app.queue")) {
+        return {
+          comment: "Queue name for this row was loaded separately, and will not be searchable!",
+          content: queueName
+        };
+      }
+      return queueName;
+    }
   },{
     id: 'callerID',
     headerTitle: 'Caller ID',

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/entities/entity.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/entities/entity.js b/tez-ui/src/main/webapp/app/entities/entity.js
index d982811..c8f602b 100644
--- a/tez-ui/src/main/webapp/app/entities/entity.js
+++ b/tez-ui/src/main/webapp/app/entities/entity.js
@@ -187,6 +187,10 @@ var Entity = Ember.Object.extend(NameMixin, {
       MoreObject.forEach(needs, function (name, needOptions) {
         needOptions = that.normalizeNeed(name, needOptions, parentModel, queryParams, urlParams);
 
+        if(MoreObject.isFunction(needOptions.loadType)) {
+          needOptions.loadType = needOptions.loadType.call(needOptions, parentModel);
+        }
+
         if(needOptions.loadType !== "demand") {
           let needLoader = that._loadNeed(loader, parentModel, needOptions, options);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/models/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/dag.js b/tez-ui/src/main/webapp/app/models/dag.js
index 5e011e2..7e57615 100644
--- a/tez-ui/src/main/webapp/app/models/dag.js
+++ b/tez-ui/src/main/webapp/app/models/dag.js
@@ -43,6 +43,11 @@ export default AMTimelineModel.extend({
     app: {
       type: ["AhsApp", "appRm"],
       idKey: "appID",
+      loadType: function (record) {
+        if(record.get("queueName")) {
+          return "demand";
+        }
+      },
       silent: true
     }
   },
@@ -58,8 +63,9 @@ export default AMTimelineModel.extend({
 
   domain: DS.attr("string"),
   containerLogs: DS.attr("object"),
-  queue: Ember.computed("app", function () {
-    return this.get("app.queue");
+  queueName: DS.attr("string"),
+  queue: Ember.computed("queueName", "app", function () {
+    return this.get("queueName") || this.get("app.queue");
   }),
 
   vertexIdNameMap: DS.attr("object"),

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/routes/home/index.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/routes/home/index.js b/tez-ui/src/main/webapp/app/routes/home/index.js
index 92a93d8..0cdcc7c 100644
--- a/tez-ui/src/main/webapp/app/routes/home/index.js
+++ b/tez-ui/src/main/webapp/app/routes/home/index.js
@@ -32,6 +32,7 @@ export default ServerSideOpsRoute.extend({
     status: REFRESH,
     appID: REFRESH,
     callerID: REFRESH,
+    queue: REFRESH,
 
     rowCount: REFRESH
   },
@@ -43,6 +44,7 @@ export default ServerSideOpsRoute.extend({
     status: "status",
     appID: "appID",
     callerID: "callerID",
+    queueName: "queue",
 
     limit: "rowCount",
   },
@@ -63,7 +65,8 @@ export default ServerSideOpsRoute.extend({
       submitter: query.submitter,
       status: query.status,
       appID: query.appID,
-      callerID: query.callerID
+      callerID: query.callerID,
+      queue: query.queueName
     };
 
     return records.filter(function (record) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/serializers/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/serializers/dag.js b/tez-ui/src/main/webapp/app/serializers/dag.js
index bf31174..04dbdd2 100644
--- a/tez-ui/src/main/webapp/app/serializers/dag.js
+++ b/tez-ui/src/main/webapp/app/serializers/dag.js
@@ -121,7 +121,9 @@ export default TimelineSerializer.extend({
 
     // appID
     domain: 'domain',
-    // queue
+
+    queueName: 'otherinfo.queueName',
+
     containerLogs: getContainerLogs,
 
     vertexIdNameMap: getIdNameMap,

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/styles/dags-page-search.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/dags-page-search.less b/tez-ui/src/main/webapp/app/styles/dags-page-search.less
index 5fc3cfb..fe37b84 100644
--- a/tez-ui/src/main/webapp/app/styles/dags-page-search.less
+++ b/tez-ui/src/main/webapp/app/styles/dags-page-search.less
@@ -41,7 +41,7 @@
 
   .search-element {
     display: inline-block;
-    width: 16.66%;
+    width: 14.25%;
 
     vertical-align: top;
 
@@ -67,7 +67,7 @@
 @media screen and (min-width: 1300px) {
   .dags-page-search{
     float: left;
-    width: 70%;
+    width: 75%;
 
     .form-group {
       margin-bottom: 0px;

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs b/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs
index 60a0958..7def955 100644
--- a/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/dags-page-search.hbs
@@ -63,6 +63,14 @@
       enter="search"
     }}
   </div><div class="search-element">
+    <label for="pwd">Queue:</label>
+    {{input value=queue
+    type="text"
+    class="form-control input-sm"
+    placeholder="Search..."
+    enter="search"
+    }}
+  </div><div class="search-element">
     <label for="pwd">Caller ID:</label>
     {{input value=callerID
       type="text"

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/templates/dag/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dag/index.hbs b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
index 4876b58..ad1e093 100644
--- a/tez-ui/src/main/webapp/app/templates/dag/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
@@ -70,6 +70,10 @@
         <td>{{txt model.duration type="duration"}}</td>
       </tr>
       <tr>
+        <td>Queue</td>
+        <td>{{txt model.queue}}</td>
+      </tr>
+      <tr>
         <td>Logs</td>
         <td>
           {{em-table-linked-cell content=model.containerLogs definition=logLinkDefinition}}

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/app/templates/home/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/home/index.hbs b/tez-ui/src/main/webapp/app/templates/home/index.hbs
index fe18266..e6f3aa0 100644
--- a/tez-ui/src/main/webapp/app/templates/home/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/home/index.hbs
@@ -36,6 +36,11 @@
     loadPage="loadPage"
     reload="reload"
   }}
+  {{#if queue}}
+    <div class="alert alert-info">
+      <strong>Info!</strong> Searching on Queue Name is supported only form <strong>Tez 0.9.0</strong>. So DAGs created by older versions of Tez will not be available!
+    </div>
+  {{/if}}
 {{else}}
   {{partial "loading"}}
 {{/if}}

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/package.json
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json
index 8985f82..d85766b 100644
--- a/tez-ui/src/main/webapp/package.json
+++ b/tez-ui/src/main/webapp/package.json
@@ -58,7 +58,7 @@
   },
   "dependencies": {
     "em-helpers": "0.8.0",
-    "em-table": "0.6.0",
+    "em-table": "0.7.2",
     "em-tgraph": "0.0.10"
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/integration/components/dags-page-search-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/integration/components/dags-page-search-test.js b/tez-ui/src/main/webapp/tests/integration/components/dags-page-search-test.js
index def7413..d9912eb 100644
--- a/tez-ui/src/main/webapp/tests/integration/components/dags-page-search-test.js
+++ b/tez-ui/src/main/webapp/tests/integration/components/dags-page-search-test.js
@@ -30,7 +30,7 @@ test('Basic creation test', function(assert) {
 
   this.render(hbs`{{dags-page-search}}`);
 
-  assert.equal(this.$("input").length, 5);
+  assert.equal(this.$("input").length, 6);
   assert.equal(this.$("select").length, 1);
 
   // Template block usage:" + EOL +
@@ -40,6 +40,6 @@ test('Basic creation test', function(assert) {
     {{/dags-page-search}}
   `);
 
-  assert.equal(this.$("input").length, 5);
+  assert.equal(this.$("input").length, 6);
   assert.equal(this.$("select").length, 1);
 });

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js b/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
index c7eab39..8f8fef6 100644
--- a/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
@@ -37,7 +37,7 @@ test('Basic creation test', function(assert) {
 
 test('filters test', function(assert) {
   let filters = this.subject().filters;
-  assert.equal(Object.keys(filters).length, 6 + 7 + 4);
+  assert.equal(Object.keys(filters).length, 6 + 8 + 4);
 });
 
 test('stringifyFilters test', function(assert) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
index 579cc76..a9e5ac1 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/index-test.js
@@ -56,6 +56,17 @@ test('Basic creation test', function(assert) {
   assert.ok(controller.actions.pageChanged);
 });
 
+test('queryParams test', function(assert) {
+  let controller = this.subject({
+        initVisibleColumns: Ember.K,
+        beforeSort: {bind: Ember.K},
+        send: Ember.K
+      });
+
+  // 11 New, 5 Inherited & 4 for backward compatibility
+  assert.equal(controller.get("queryParams.length"), 7 + 5 + 4);
+});
+
 test('definition test', function(assert) {
   let controller = this.subject({
         initVisibleColumns: Ember.K,
@@ -69,6 +80,7 @@ test('definition test', function(assert) {
       testStatus = "Status",
       testAppID = "AppID",
       testCallerID = "CallerID",
+      testQueue = "Queue",
       testPageNum = 10,
       testMoreAvailable = true,
       testLoadingMore = true;
@@ -79,6 +91,7 @@ test('definition test', function(assert) {
   assert.equal(definition.get("status"), "");
   assert.equal(definition.get("appID"), "");
   assert.equal(definition.get("callerID"), "");
+  assert.equal(definition.get("queue"), "");
 
   assert.equal(definition.get("pageNum"), 1);
 
@@ -104,6 +117,9 @@ test('definition test', function(assert) {
     controller.set("callerID", testCallerID);
     assert.equal(controller.get("definition.callerID"), testCallerID);
 
+    controller.set("queue", testQueue);
+    assert.equal(controller.get("definition.queue"), testQueue);
+
     controller.set("pageNum", testPageNum);
     assert.equal(controller.get("definition.pageNum"), testPageNum);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js b/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
index 1e353f0..57aad9f 100644
--- a/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
@@ -230,6 +230,31 @@ test('setNeed test', function(assert) {
   assert.equal(parentModel.get(testName), undefined);
 });
 
+test('loadAllNeeds loadType=function test', function(assert) {
+  var entity = this.subject(),
+      loader = {},
+      testRecord = Ember.Object.create({
+        refreshLoadTime: Ember.K,
+        needs: {
+          app: {
+            idKey: "appID",
+            loadType: function (record) {
+              assert.ok(testRecord === record);
+              return "demand";
+            }
+          },
+        },
+        appID: 1,
+      });
+
+  entity._loadNeed = function () {
+    assert.ok(true); // Shouldn't be called
+  };
+
+  assert.expect(1 + 1);
+  assert.equal(entity.loadAllNeeds(loader, testRecord), undefined);
+});
+
 test('_loadNeed single string type test', function(assert) {
   let entity = this.subject(),
       loader,

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
index 2c07f7b..a219cf8 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/dag-test.js
@@ -58,3 +58,21 @@ test('Basic creation test', function(assert) {
 
   assert.ok(model.amWsVersion);
 });
+
+test('queue test', function(assert) {
+  let model = this.subject(),
+      queueName = "queueName",
+      appQueueName = "AppQueueName";
+
+  assert.equal(model.get("queue"), undefined);
+
+  Ember.run(function () {
+    model.set("app", {
+      queue: appQueueName
+    });
+    assert.equal(model.get("queue"), appQueueName);
+
+    model.set("queueName", queueName);
+    assert.equal(model.get("queue"), queueName);
+  });
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/routes/home/index-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/routes/home/index-test.js b/tez-ui/src/main/webapp/tests/unit/routes/home/index-test.js
index 99d7cf6..d35a325 100644
--- a/tez-ui/src/main/webapp/tests/unit/routes/home/index-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/routes/home/index-test.js
@@ -56,7 +56,7 @@ test('refresh test', function(assert) {
 
 test('loaderQueryParams test', function(assert) {
   let route = this.subject();
-  assert.equal(Object.keys(route.get("loaderQueryParams")).length, 7);
+  assert.equal(Object.keys(route.get("loaderQueryParams")).length, 8);
 });
 
 test('filterRecords test', function(assert) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d40f3ad7/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js
index 7df4084..e782869 100644
--- a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js
@@ -35,6 +35,8 @@ test('Basic creation test', function(assert) {
   assert.ok(serializer.maps.endTime);
   assert.ok(serializer.maps.containerLogs);
   assert.ok(serializer.maps.vertexIdNameMap);
+
+  assert.equal(Object.keys(serializer.get("maps")).length, 18 + 5); //18 own & 7 inherited
 });
 
 test('atsStatus test', function(assert) {


[19/50] [abbrv] tez git commit: TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism (jeagles)

Posted by zh...@apache.org.
TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism (jeagles)


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

Branch: refs/heads/TEZ-1190
Commit: d5bf28bfaeda306936a9752dfd3f6821cbebf235
Parents: 2158b95
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Mon Feb 27 16:50:30 2017 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Mon Feb 27 16:50:30 2017 -0600

----------------------------------------------------------------------
 CHANGES.txt                                                        | 1 +
 .../src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java  | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d5bf28bf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 34f8624..c9ac898 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism
   TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.
   TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.
   TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes

http://git-wip-us.apache.org/repos/asf/tez/blob/d5bf28bf/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 4cda98d..1ab3da8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -2506,7 +2506,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       if (i <= newNumTasks) {
         continue;
       }
-      LOG.info("Removing task: " + entry.getKey());
+      LOG.debug("Removing task: {}", entry.getKey());
       iter.remove();
       this.numTasks--;
     }


[45/50] [abbrv] tez git commit: TEZ-3667. Stop using org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT (zhiyuany)

Posted by zh...@apache.org.
TEZ-3667. Stop using org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: e375b9d348f702093e9753f186dff5527c0f121c
Parents: cbcb3a7
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Wed Mar 22 21:19:29 2017 -0700
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Wed Mar 22 21:19:29 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/tez/http/SSLFactory.java              | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e375b9d3/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
index 9751463..e7a2dd0 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
@@ -37,7 +37,6 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.security.GeneralSecurityException;
 
-import static org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT;
 import static org.apache.hadoop.security.ssl.SSLFactory.KEYSTORES_FACTORY_CLASS_KEY;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_HOSTNAME_VERIFIER_KEY;
@@ -60,6 +59,7 @@ public class SSLFactory implements ConnectionConfigurator {
 
   public static final String SSL_ENABLED_PROTOCOLS = "hadoop.ssl.enabled.protocols";
   public static final String DEFAULT_SSL_ENABLED_PROTOCOLS = "TLSv1";
+  public static final boolean DEFAULT_SSL_REQUIRE_CLIENT_CERT = false;
 
   private Configuration conf;
   private Mode mode;


[11/50] [abbrv] tez git commit: TEZ-3625. Dag.getVertex should obtain a readlock. (sseth)

Posted by zh...@apache.org.
TEZ-3625. Dag.getVertex should obtain a readlock. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: dcae5e42e961a6b2fd0e82a2392d601ca935d0f1
Parents: 2268c72
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 15 20:30:06 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Feb 15 20:30:06 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                                   | 1 +
 .../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java    | 1 +
 .../main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java    | 7 ++++++-
 3 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/dcae5e42/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a3323b9..c505964 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3625. Dag.getVertex should obtain a readlock.
   TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.
   TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3267. Publish queue name to ATS as part of dag summary.

http://git-wip-us.apache.org/repos/asf/tez/blob/dcae5e42/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 7f27064..5a43358 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -2627,6 +2627,7 @@ public class DAGAppMaster extends AbstractService {
 
     // Send out an event to inform components that a new DAG has been submitted.
     // Information about this DAG is available via the context.
+    // This event may be processed after DAG_INIT, but will be processed before DAG_START
     sendEvent(new DAGAppMasterEvent(DAGAppMasterEventType.NEW_DAG_SUBMITTED));
     // create a job event for job initialization
     DAGEvent initDagEvent = new DAGEvent(currentDAG.getID(), DAGEventType.DAG_INIT);

http://git-wip-us.apache.org/repos/asf/tez/blob/dcae5e42/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 481353b..047fea6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -1810,7 +1810,12 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   @Override
   public Vertex getVertex(String vertexName) {
-    return vertexMap.get(vertexName);
+    this.readLock.lock();
+    try {
+      return vertexMap.get(vertexName);
+    } finally {
+      this.readLock.unlock();
+    }
   }
 
   private void mayBeConstructFinalFullCounters() {


[23/50] [abbrv] tez git commit: TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances. (sseth)

Posted by zh...@apache.org.
TEZ-3643. Long running AMs can go out of memory due to retained
AMContainer instances. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: 6051542030101e42738fa2c2da984bb2c744b9c5
Parents: ee4a9a9
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Mar 1 09:05:58 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Mar 1 09:05:58 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../tez/dag/app/rm/container/AMContainer.java   |   1 +
 .../dag/app/rm/container/AMContainerImpl.java   |   1 +
 .../dag/app/rm/container/AMContainerMap.java    |  40 +++++-
 .../dag/app/rm/container/TestAMContainer.java   |   8 +-
 .../app/rm/container/TestAMContainerMap.java    | 126 +++++++++++++++----
 6 files changed, 146 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3806e27..7538f3e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances.
   TEZ-3637. TezMerger logs too much at INFO level
   TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism
   TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.

http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
index 8f5034e..5f90a89 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainer.java
@@ -36,4 +36,5 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
   public int getTaskSchedulerIdentifier();
   public int getContainerLauncherIdentifier();
   public int getTaskCommunicatorIdentifier();
+  public boolean isInErrorState();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 5d73a7b..ac429c7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -401,6 +401,7 @@ public class AMContainerImpl implements AMContainer {
     return this.taskCommId;
   }
 
+  @Override
   public boolean isInErrorState() {
     return inError;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
index ab43db1..050ffb6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerMap.java
@@ -19,8 +19,11 @@
 package org.apache.tez.dag.app.rm.container;
 
 import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.common.ContainerSignatureMatcher;
 import org.slf4j.Logger;
@@ -41,7 +44,8 @@ public class AMContainerMap extends AbstractService implements EventHandler<AMCo
   private final TaskCommunicatorManagerInterface tal;
   private final AppContext context;
   private final ContainerSignatureMatcher containerSignatureMatcher;
-  private final ConcurrentHashMap<ContainerId, AMContainer> containerMap;
+  @VisibleForTesting
+  final ConcurrentHashMap<ContainerId, AMContainer> containerMap;
 
   public AMContainerMap(ContainerHeartbeatHandler chh, TaskCommunicatorManagerInterface tal,
       ContainerSignatureMatcher containerSignatureMatcher, AppContext context) {
@@ -64,11 +68,23 @@ public class AMContainerMap extends AbstractService implements EventHandler<AMCo
   }
 
   public boolean addContainerIfNew(Container container, int schedulerId, int launcherId, int taskCommId) {
-    AMContainer amc = new AMContainerImpl(container, chh, tal,
-      containerSignatureMatcher, context, schedulerId, launcherId, taskCommId);
+    AMContainer amc = createAmContainer(container, chh, tal,
+        containerSignatureMatcher, context, schedulerId, launcherId, taskCommId);
+
     return (containerMap.putIfAbsent(container.getId(), amc) == null);
   }
 
+  AMContainer createAmContainer(Container container,
+                                ContainerHeartbeatHandler chh,
+                                TaskCommunicatorManagerInterface tal,
+                                ContainerSignatureMatcher signatureMatcher,
+                                AppContext appContext, int schedulerId,
+                                int launcherId, int taskCommId) {
+    AMContainer amc = new AMContainerImpl(container, chh, tal,
+        signatureMatcher, appContext, schedulerId, launcherId, taskCommId);
+    return amc;
+  }
+
   public AMContainer get(ContainerId containerId) {
     return containerMap.get(containerId);
   }
@@ -79,6 +95,24 @@ public class AMContainerMap extends AbstractService implements EventHandler<AMCo
 
   public void dagComplete(DAG dag){
     AMContainerHelpers.dagComplete(dag.getID());
+    // Cleanup completed containers after a query completes.
+    cleanupCompletedContainers();
+  }
+
+  private void cleanupCompletedContainers() {
+    Iterator<Map.Entry<ContainerId, AMContainer>> iterator = containerMap.entrySet().iterator();
+    int count = 0;
+    while (iterator.hasNext()) {
+      Map.Entry<ContainerId, AMContainer> entry = iterator.next();
+      AMContainer amContainer = entry.getValue();
+      if (AMContainerState.COMPLETED.equals(amContainer.getState()) || amContainer.isInErrorState()) {
+        iterator.remove();
+        count++;
+      }
+    }
+    LOG.info(
+        "Cleaned up completed containers on dagComplete. Removed={}, Remaining={}",
+        count, containerMap.size());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index ed14871..4d1bbae 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -1183,7 +1183,7 @@ public class TestAMContainer {
 
   // TODO Verify diagnostics in most of the tests.
 
-  private static class WrappedContainer {
+  static class WrappedContainer {
 
     long rmIdentifier = 2000;
     static final int taskPriority = 10;
@@ -1215,10 +1215,10 @@ public class TestAMContainer {
     public AMContainerImpl amContainer;
 
     @SuppressWarnings("deprecation") // ContainerId
-    public WrappedContainer(boolean shouldProfile, String profileString) {
+    public WrappedContainer(boolean shouldProfile, String profileString, int cIdInt) {
       applicationID = ApplicationId.newInstance(rmIdentifier, 1);
       appAttemptID = ApplicationAttemptId.newInstance(applicationID, 1);
-      containerID = ContainerId.newInstance(appAttemptID, 1);
+      containerID = ContainerId.newInstance(appAttemptID, cIdInt);
       nodeID = NodeId.newInstance("host", 12500);
       nodeHttpAddress = "host:12501";
       resource = Resource.newInstance(1024, 1);
@@ -1265,7 +1265,7 @@ public class TestAMContainer {
     }
 
     public WrappedContainer() {
-      this(false, null);
+      this(false, null, 1);
     }
 
     protected void mockDAGID() {

http://git-wip-us.apache.org/repos/asf/tez/blob/60515420/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
index 2fcd0c8..efea327 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainerMap.java
@@ -18,11 +18,15 @@
 
 package org.apache.tez.dag.app.rm.container;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
 import java.net.InetSocketAddress;
+import java.util.Map;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -31,43 +35,117 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.tez.common.ContainerSignatureMatcher;
+import org.apache.tez.dag.app.TaskCommunicatorWrapper;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.rm.container.TestAMContainer.WrappedContainer;
 import org.apache.tez.serviceplugins.api.TaskCommunicator;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
 import org.apache.tez.serviceplugins.api.ServicePluginException;
+import org.junit.Test;
 
 public class TestAMContainerMap {
 
-  private ContainerHeartbeatHandler mockContainerHeartBeatHandler() {
-    return mock(ContainerHeartbeatHandler.class);
-  }
 
-  private TaskCommunicatorManagerInterface mockTaskAttemptListener() throws ServicePluginException {
-    TaskCommunicatorManagerInterface tal = mock(TaskCommunicatorManagerInterface.class);
-    TaskCommunicator taskComm = mock(TaskCommunicator.class);
-    doReturn(new InetSocketAddress("localhost", 21000)).when(taskComm).getAddress();
-    doReturn(taskComm).when(tal).getTaskCommunicator(0);
-    return tal;
-  }
+  @Test (timeout = 10000)
+  public void testCleanupOnDagComplete() {
 
-  private AppContext mockAppContext() {
+    ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class);
+    TaskCommunicatorManagerInterface tal = mock(TaskCommunicatorManagerInterface.class);
     AppContext appContext = mock(AppContext.class);
-    return appContext;
-  }
 
-  @SuppressWarnings("deprecation")
-  private ContainerId mockContainerId(int cId) {
-    ApplicationId appId = ApplicationId.newInstance(1000, 1);
-    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, cId);
-    return containerId;
+
+
+    int numContainers = 7;
+    WrappedContainer[] wContainers = new WrappedContainer[numContainers];
+    for (int i = 0 ; i < numContainers ; i++) {
+      WrappedContainer wc =
+          new WrappedContainer(false, null, i);
+      wContainers[i] = wc;
+    }
+
+    AMContainerMap amContainerMap = new AMContainerMapForTest(chh, tal, mock(
+        ContainerSignatureMatcher.class), appContext, wContainers);
+
+    for (int i = 0 ; i < numContainers ; i++) {
+      amContainerMap.addContainerIfNew(wContainers[i].container, 0, 0, 0);
+    }
+
+
+    // Container 1 in LAUNCHING state
+    wContainers[0].launchContainer();
+    wContainers[0].verifyState(AMContainerState.LAUNCHING);
+
+    // Container 2 in IDLE state
+    wContainers[1].launchContainer();
+    wContainers[1].containerLaunched();
+    wContainers[1].verifyState(AMContainerState.IDLE);
+
+    // Container 3 RUNNING state
+    wContainers[2].launchContainer();
+    wContainers[2].containerLaunched();
+    wContainers[2].assignTaskAttempt(wContainers[2].taskAttemptID);
+    wContainers[2].verifyState(AMContainerState.RUNNING);
+
+    // Cointainer 4 STOP_REQUESTED
+    wContainers[3].launchContainer();
+    wContainers[3].containerLaunched();
+    wContainers[3].stopRequest();
+    wContainers[3].verifyState(AMContainerState.STOP_REQUESTED);
+
+    // Container 5 STOPPING
+    wContainers[4].launchContainer();
+    wContainers[4].containerLaunched();
+    wContainers[4].stopRequest();
+    wContainers[4].nmStopSent();
+    wContainers[4].verifyState(AMContainerState.STOPPING);
+
+    // Container 6 COMPLETED
+    wContainers[5].launchContainer();
+    wContainers[5].containerLaunched();
+    wContainers[5].stopRequest();
+    wContainers[5].nmStopSent();
+    wContainers[5].containerCompleted();
+    wContainers[5].verifyState(AMContainerState.COMPLETED);
+
+    // Container 7 STOP_REQUESTED + ERROR
+    wContainers[6].launchContainer();
+    wContainers[6].containerLaunched();
+    wContainers[6].containerLaunched();
+    assertTrue(wContainers[6].amContainer.isInErrorState());
+    wContainers[6].verifyState(AMContainerState.STOP_REQUESTED);
+
+    // 7 containers present, and registered with AMContainerMap at this point.
+
+    assertEquals(7, amContainerMap.containerMap.size());
+    amContainerMap.dagComplete(mock(DAG.class));
+    assertEquals(5, amContainerMap.containerMap.size());
   }
 
-  private Container mockContainer(ContainerId containerId) {
-    NodeId nodeId = NodeId.newInstance("localhost", 43255);
-    Container container = Container.newInstance(containerId, nodeId, "localhost:33333",
-        Resource.newInstance(1024, 1), Priority.newInstance(1), mock(Token.class));
-    return container;
+  private static class AMContainerMapForTest extends AMContainerMap {
+
+
+    private WrappedContainer[] wrappedContainers;
+
+    public AMContainerMapForTest(ContainerHeartbeatHandler chh,
+                                 TaskCommunicatorManagerInterface tal,
+                                 ContainerSignatureMatcher containerSignatureMatcher,
+                                 AppContext context, WrappedContainer[] wrappedContainers) {
+      super(chh, tal, containerSignatureMatcher, context);
+      this.wrappedContainers = wrappedContainers;
+    }
+
+    @Override
+    AMContainer createAmContainer(Container container,
+                                  ContainerHeartbeatHandler chh,
+                                  TaskCommunicatorManagerInterface tal,
+                                  ContainerSignatureMatcher signatureMatcher,
+                                  AppContext appContext, int schedulerId,
+                                  int launcherId, int taskCommId) {
+      return wrappedContainers[container.getId().getId()].amContainer;
+    }
+
   }
 }


[48/50] [abbrv] tez git commit: TEZ-3672. Remove duplicate Apache license headers. Contributed by Saijin Huang

Posted by zh...@apache.org.
TEZ-3672. Remove duplicate Apache license headers. Contributed by Saijin Huang


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

Branch: refs/heads/TEZ-1190
Commit: 618c8118239b72e6b9af61fe9ff4717d5233b980
Parents: f7f6038
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Tue Mar 28 10:07:21 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Tue Mar 28 10:07:21 2017 -0500

----------------------------------------------------------------------
 .../tez/serviceplugins/api/TaskCommunicator.java      | 14 --------------
 .../serviceplugins/api/TaskCommunicatorContext.java   | 14 --------------
 .../tez/serviceplugins/api/TaskHeartbeatRequest.java  | 14 --------------
 .../tez/serviceplugins/api/TaskHeartbeatResponse.java | 14 --------------
 4 files changed, 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/618c8118/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java
index 34debd4..fceddf2 100644
--- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java
+++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java
@@ -12,20 +12,6 @@
  * limitations under the License.
  */
 
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 package org.apache.tez.serviceplugins.api;
 
 import javax.annotation.Nullable;

http://git-wip-us.apache.org/repos/asf/tez/blob/618c8118/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java
index 00a830b..4c6e846 100644
--- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicatorContext.java
@@ -12,20 +12,6 @@
  * limitations under the License.
  */
 
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 package org.apache.tez.serviceplugins.api;
 
 import javax.annotation.Nullable;

http://git-wip-us.apache.org/repos/asf/tez/blob/618c8118/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatRequest.java
index 40b006f..2f9fb7f 100644
--- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatRequest.java
@@ -12,20 +12,6 @@
  * limitations under the License.
  */
 
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 package org.apache.tez.serviceplugins.api;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/tez/blob/618c8118/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatResponse.java
index 9145004..d7a353b 100644
--- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatResponse.java
+++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskHeartbeatResponse.java
@@ -12,20 +12,6 @@
  * limitations under the License.
  */
 
-/*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 package org.apache.tez.serviceplugins.api;
 
 import java.util.List;


[24/50] [abbrv] tez git commit: TEZ-3646. IFile.Writer has an extra output stream flush call (jeagles)

Posted by zh...@apache.org.
TEZ-3646. IFile.Writer has an extra output stream flush call (jeagles)


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

Branch: refs/heads/TEZ-1190
Commit: 1f2a93563ac936b607028dbf671471686eb94829
Parents: 6051542
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Thu Mar 2 10:16:29 2017 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Thu Mar 2 10:16:29 2017 -0600

----------------------------------------------------------------------
 CHANGES.txt                                                         | 1 +
 .../java/org/apache/tez/runtime/library/common/sort/impl/IFile.java | 1 -
 2 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/1f2a9356/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7538f3e..b8465de 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3646. IFile.Writer has an extra output stream flush call
   TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances.
   TEZ-3637. TezMerger logs too much at INFO level
   TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism

http://git-wip-us.apache.org/repos/asf/tez/blob/1f2a9356/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
index f49bc35..bcf6ca6 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
@@ -187,7 +187,6 @@ public class IFile {
       if (!headerWritten) {
         outputStream.write(HEADER, 0, HEADER.length - 1);
         outputStream.write((compressOutput) ? (byte) 1 : (byte) 0);
-        outputStream.flush();
         headerWritten = true;
       }
     }


[02/50] [abbrv] tez git commit: TEZ-3581. Add different logger to enable suppressing logs for specific lines. Contributed by Harish Jaiprakash.

Posted by zh...@apache.org.
TEZ-3581. Add different logger to enable suppressing logs for specific
lines. Contributed by Harish Jaiprakash.


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

Branch: refs/heads/TEZ-1190
Commit: d415197c67562ee2859b240e8a7e316067c4ed6b
Parents: c0270cb
Author: Siddharth Seth <ss...@apache.org>
Authored: Sun Feb 5 19:08:08 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Sun Feb 5 19:08:08 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../tez/dag/history/HistoryEventHandler.java    | 23 +++--
 .../resources/tez-container-log4j.properties    |  8 ++
 .../org/apache/tez/http/HttpConnection.java     | 18 +++-
 .../library/common/shuffle/ShuffleUtils.java    | 92 ++++++++++++++------
 .../common/shuffle/impl/ShuffleManager.java     |  7 +-
 .../orderedgrouped/ShuffleScheduler.java        |  8 +-
 .../common/shuffle/TestShuffleUtils.java        | 34 ++++++--
 8 files changed, 144 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 37438d9..c4a1d72 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3581. Add different logger to enable suppressing logs for specific lines.
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3600. Fix flaky test: TestTokenCache
   TEZ-3589. add a unit test for amKeepAlive not being shutdown if an app takes a long time to launch.

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
index 79d1fc3..4fa1926 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,6 +45,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 public class HistoryEventHandler extends CompositeService {
 
   private static Logger LOG = LoggerFactory.getLogger(HistoryEventHandler.class);
+  private static Logger LOG_CRITICAL_EVENTS =
+      LoggerFactory.getLogger(LOG.getName() + ".criticalEvents");
 
   private final AppContext context;
   private RecoveryService recoveryService;
@@ -59,6 +62,8 @@ public class HistoryEventHandler extends CompositeService {
   private final ConcurrentHashMap<TezTaskAttemptID, DAGHistoryEvent> suppressedEvents =
       new ConcurrentHashMap<>();
 
+  private final AtomicLong criticalEventCount = new AtomicLong();
+
   public HistoryEventHandler(AppContext context) {
     super(HistoryEventHandler.class.getName());
     this.context = context;
@@ -141,12 +146,18 @@ public class HistoryEventHandler extends CompositeService {
       historyLoggingService.handle(event);
     }
 
-    // TODO at some point we should look at removing this once
-    // there is a UI in place
-    LOG.info("[HISTORY]"
-        + "[DAG:" + dagIdStr + "]"
-        + "[Event:" + event.getHistoryEvent().getEventType().name() + "]"
-        + ": " + event.getHistoryEvent().toString());
+    if (LOG_CRITICAL_EVENTS.isInfoEnabled()) {
+      // TODO at some point we should look at removing this once
+      // there is a UI in place
+      LOG_CRITICAL_EVENTS.info("[HISTORY]"
+          + "[DAG:" + dagIdStr + "]"
+          + "[Event:" + event.getHistoryEvent().getEventType().name() + "]"
+          + ": " + event.getHistoryEvent().toString());
+    } else {
+      if (criticalEventCount.incrementAndGet() % 1000 == 0) {
+        LOG.info("Got {} critical events", criticalEventCount);
+      }
+    }
   }
 
   private boolean shouldLogEvent(DAGHistoryEvent event) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-dag/src/main/resources/tez-container-log4j.properties
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/resources/tez-container-log4j.properties b/tez-dag/src/main/resources/tez-container-log4j.properties
index 4620a78..721cd67 100644
--- a/tez-dag/src/main/resources/tez-container-log4j.properties
+++ b/tez-dag/src/main/resources/tez-container-log4j.properties
@@ -36,3 +36,11 @@ log4j.appender.CLA.layout.ConversionPattern=%d{ISO8601} [%p] [%t] |%c{2}|: %m%n
 #
 log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
 
+# Disable loggers which log a lot, use this if you want to reduce the log sizes. This will affect
+# the analyzer since it relies on these log lines.
+# log4j.logger.org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager.fetch=WARN
+# log4j.logger.org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleScheduler=WARN
+# log4j.logger.org.apache.tez.http.HttpConnection.url=WARN
+
+# This should be part of the AM log4j.properties file, it will not work from this file.
+# log4j.logger.org.apache.tez.dag.history.HistoryEventHandler.criticalEvents=WARN

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
index d781e64..9bfe4e7 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
@@ -36,10 +36,12 @@ import java.io.InputStream;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 public class HttpConnection extends BaseHttpConnection {
 
   private static final Logger LOG = LoggerFactory.getLogger(HttpConnection.class);
+  private static final Logger URL_LOG = LoggerFactory.getLogger(LOG.getName() + ".url");
 
   private URL url;
   private final String logIdentifier;
@@ -56,6 +58,7 @@ public class HttpConnection extends BaseHttpConnection {
 
   private final HttpConnectionParams httpConnParams;
   private final StopWatch stopWatch;
+  private final AtomicLong urlLogCount;
 
   /**
    * HttpConnection
@@ -73,6 +76,7 @@ public class HttpConnection extends BaseHttpConnection {
     this.httpConnParams = connParams;
     this.url = url;
     this.stopWatch = new StopWatch();
+    this.urlLogCount = new AtomicLong();
     if (LOG.isDebugEnabled()) {
       LOG.debug("MapOutput URL :" + url.toString());
     }
@@ -229,9 +233,17 @@ public class HttpConnection extends BaseHttpConnection {
 
     // verify that replyHash is HMac of encHash
     SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecretMgr);
-    //Following log statement will be used by tez-tool perf-analyzer for mapping attempt to NM host
-    LOG.info("for url=" + url +
-        " sent hash and receievd reply " + stopWatch.now(TimeUnit.MILLISECONDS) + " ms");
+    if (URL_LOG.isInfoEnabled()) {
+      // Following log statement will be used by tez-tool perf-analyzer for mapping attempt to NM
+      // host
+      URL_LOG.info("for url=" + url + " sent hash and receievd reply " +
+          stopWatch.now(TimeUnit.MILLISECONDS) + " ms");
+    } else {
+      // Log summary.
+      if (urlLogCount.incrementAndGet() % 1000 == 0) {
+        LOG.info("Sent hash and recieved reply for {} urls", urlLogCount);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index aa07233..82e844d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -29,6 +29,7 @@ import java.text.DecimalFormat;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.zip.Deflater;
 
 import javax.annotation.Nullable;
@@ -516,35 +517,70 @@ public class ShuffleUtils {
     return builder.build();
   }
 
-  /**
-   * Log individual fetch complete event.
-   * This log information would be used by tez-tool/perf-analzyer/shuffle tools for mining
-   * - amount of data transferred between source to destination machine
-   * - time taken to transfer data between source to destination machine
-   * - details on DISK/DISK_DIRECT/MEMORY based shuffles
-   *
-   * @param log
-   * @param millis
-   * @param bytesCompressed
-   * @param bytesDecompressed
-   * @param outputType
-   * @param srcAttemptIdentifier
-   */
-  public static void logIndividualFetchComplete(Logger log, long millis, long
-      bytesCompressed,
-      long bytesDecompressed, String outputType, InputAttemptIdentifier srcAttemptIdentifier) {
-    double rate = 0;
-    if (millis != 0) {
-      rate = bytesCompressed / ((double) millis / 1000);
-      rate = rate / (1024 * 1024);
+  public static class FetchStatsLogger {
+    private final Logger activeLogger;
+    private final Logger aggregateLogger;
+    private final AtomicLong logCount = new AtomicLong();
+    private final AtomicLong compressedSize = new AtomicLong();
+    private final AtomicLong decompressedSize = new AtomicLong();
+    private final AtomicLong totalTime = new AtomicLong();
+
+    public FetchStatsLogger(Logger activeLogger, Logger aggregateLogger) {
+      this.activeLogger = activeLogger;
+      this.aggregateLogger = aggregateLogger;
+    }
+
+    /**
+     * Log individual fetch complete event.
+     * This log information would be used by tez-tool/perf-analzyer/shuffle tools for mining
+     * - amount of data transferred between source to destination machine
+     * - time taken to transfer data between source to destination machine
+     * - details on DISK/DISK_DIRECT/MEMORY based shuffles
+     *
+     * @param millis
+     * @param bytesCompressed
+     * @param bytesDecompressed
+     * @param outputType
+     * @param srcAttemptIdentifier
+     */
+    public void logIndividualFetchComplete(long millis, long bytesCompressed,
+        long bytesDecompressed, String outputType, InputAttemptIdentifier srcAttemptIdentifier) {
+      double rate = 0;
+      if (millis != 0) {
+        rate = bytesCompressed / ((double) millis / 1000);
+        rate = rate / (1024 * 1024);
+      }
+      if (activeLogger.isInfoEnabled()) {
+        activeLogger.info(
+            "Completed fetch for attempt: "
+                + toShortString(srcAttemptIdentifier)
+                +" to " + outputType +
+                ", csize=" + bytesCompressed + ", dsize=" + bytesDecompressed +
+                ", EndTime=" + System.currentTimeMillis() + ", TimeTaken=" + millis + ", Rate=" +
+                MBPS_FORMAT.get().format(rate) + " MB/s");
+      } else {
+        long currentCount, currentCompressedSize, currentDecompressedSize, currentTotalTime;
+        synchronized (this) {
+          currentCount = logCount.incrementAndGet();
+          currentCompressedSize = compressedSize.addAndGet(bytesCompressed);
+          currentDecompressedSize = decompressedSize.addAndGet(bytesDecompressed);
+          currentTotalTime = totalTime.addAndGet(millis);
+          if (currentCount % 1000 == 0) {
+            compressedSize.set(0);
+            decompressedSize.set(0);
+            totalTime.set(0);
+          }
+        }
+        if (currentCount % 1000 == 0) {
+          double avgRate = currentTotalTime == 0 ? 0
+              : currentCompressedSize / (double)currentTotalTime / 1000 / 1024 / 1024;
+          aggregateLogger.info("Completed {} fetches, stats for last 1000 fetches: "
+              + "avg csize: {}, avg dsize: {}, avgTime: {}, avgRate: {}", currentCount,
+              currentCompressedSize / 1000, currentDecompressedSize / 1000, currentTotalTime / 1000,
+              MBPS_FORMAT.get().format(avgRate));
+        }
+      }
     }
-    log.info(
-        "Completed fetch for attempt: "
-            + toShortString(srcAttemptIdentifier)
-            +" to " + outputType +
-            ", csize=" + bytesCompressed + ", dsize=" + bytesDecompressed +
-            ", EndTime=" + System.currentTimeMillis() + ", TimeTaken=" + millis + ", Rate=" +
-            MBPS_FORMAT.get().format(rate) + " MB/s");
   }
 
   private static String toShortString(InputAttemptIdentifier inputAttemptIdentifier) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
index d034b2e..b2ff51d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleManager.java
@@ -80,6 +80,7 @@ import org.apache.tez.runtime.library.common.shuffle.HostPort;
 import org.apache.tez.runtime.library.common.shuffle.InputHost;
 import org.apache.tez.runtime.library.common.shuffle.InputHost.PartitionToInputs;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger;
 
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
@@ -98,6 +99,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 public class ShuffleManager implements FetcherCallback {
 
   private static final Logger LOG = LoggerFactory.getLogger(ShuffleManager.class);
+  private static final Logger LOG_FETCH = LoggerFactory.getLogger(LOG.getName() + ".fetch");
+  private static final FetchStatsLogger fetchStatsLogger = new FetchStatsLogger(LOG_FETCH, LOG);
 
   private final InputContext inputContext;
   private final int numInputs;
@@ -628,8 +631,8 @@ public class ShuffleManager implements FetcherCallback {
         if (!completedInputSet.contains(inputIdentifier)) {
           fetchedInput.commit();
           committed = true;
-          ShuffleUtils.logIndividualFetchComplete(LOG, copyDuration,
-              fetchedBytes, decompressedLength, fetchedInput.getType().toString(), srcAttemptIdentifier);
+          fetchStatsLogger.logIndividualFetchComplete(copyDuration, fetchedBytes,
+              decompressedLength, fetchedInput.getType().toString(), srcAttemptIdentifier);
 
           // Processing counters for completed and commit fetches only. Need
           // additional counters for excessive fetches - which primarily comes

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
index 3d2c1ad..cce486c 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
@@ -73,6 +73,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 import org.apache.tez.runtime.library.common.TezRuntimeUtils;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger;
 import org.apache.tez.runtime.library.common.shuffle.HostPort;
 import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapHost.HostPortPartition;
 import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Type;
@@ -140,6 +141,9 @@ class ShuffleScheduler {
   private final AtomicLong shuffleStart = new AtomicLong(0);
 
   private static final Logger LOG = LoggerFactory.getLogger(ShuffleScheduler.class);
+  private static final Logger LOG_FETCH = LoggerFactory.getLogger(LOG.getName() + ".fetch");
+  private static final FetchStatsLogger fetchStatsLogger = new FetchStatsLogger(LOG_FETCH, LOG);
+
   static final long INITIAL_PENALTY = 2000L; // 2 seconds
   private static final float PENALTY_GROWTH_RATE = 1.3f;
 
@@ -576,8 +580,8 @@ class ShuffleScheduler {
         }
 
         output.commit();
-        ShuffleUtils.logIndividualFetchComplete(LOG, millis, bytesCompressed,
-            bytesDecompressed, output.getType().toString(), srcAttemptIdentifier);
+        fetchStatsLogger.logIndividualFetchComplete(millis, bytesCompressed, bytesDecompressed,
+            output.getType().toString(), srcAttemptIdentifier);
         if (output.getType() == Type.DISK) {
           bytesShuffledToDisk.increment(bytesCompressed);
         } else if (output.getType() == Type.DISK_DIRECT) {

http://git-wip-us.apache.org/repos/asf/tez/blob/d415197c/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
index 496468b..f21da7c 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
@@ -4,7 +4,6 @@ import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -24,7 +23,8 @@ import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
 import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
-import org.apache.tez.runtime.library.common.sort.impl.IFileOutputStream;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger;
 import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
 import org.apache.tez.runtime.library.partitioner.HashPartitioner;
@@ -32,6 +32,7 @@ import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Matchers;
 import org.slf4j.Logger;
 
 import java.io.ByteArrayInputStream;
@@ -47,8 +48,11 @@ import java.util.Random;
 
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.anyInt;
+import static org.mockito.Mockito.anyString;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 /**
@@ -70,10 +74,6 @@ import static org.mockito.Mockito.when;
  */
 public class TestShuffleUtils {
 
-  private static final String HOST = "localhost";
-  private static final int PORT = 8080;
-  private static final String PATH_COMPONENT = "attempt";
-
   private OutputContext outputContext;
   private Configuration conf;
   private FileSystem localFs;
@@ -313,4 +313,26 @@ public class TestShuffleUtils {
     } catch (IOException e) {
     }
   }
+
+  @Test
+  public void testFetchStatsLogger() throws Exception {
+    Logger activeLogger = mock(Logger.class);
+    Logger aggregateLogger = mock(Logger.class);
+    FetchStatsLogger logger = new FetchStatsLogger(activeLogger, aggregateLogger);
+
+    InputAttemptIdentifier ident = new InputAttemptIdentifier(1, 1);
+    when(activeLogger.isInfoEnabled()).thenReturn(false);
+    for (int i = 0; i < 1000; i++) {
+      logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident);
+    }
+    verify(activeLogger, times(0)).info(anyString());
+    verify(aggregateLogger, times(1)).info(anyString(), Matchers.<Object[]>anyVararg());
+
+    when(activeLogger.isInfoEnabled()).thenReturn(true);
+    for (int i = 0; i < 1000; i++) {
+      logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident);
+    }
+    verify(activeLogger, times(1000)).info(anyString());
+    verify(aggregateLogger, times(1)).info(anyString(), Matchers.<Object[]>anyVararg());
+  }
 }


[22/50] [abbrv] tez git commit: TEZ-3640. Tez UI: Add associated llap application id to queries page (sree)

Posted by zh...@apache.org.
TEZ-3640. Tez UI: Add associated llap application id to queries page (sree)


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

Branch: refs/heads/TEZ-1190
Commit: ee4a9a90856a229f5aaebf2027838660f1efae38
Parents: 57119a5
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Tue Feb 28 14:55:06 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Tue Feb 28 14:55:06 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                    | 1 +
 tez-ui/src/main/webapp/app/controllers/home/queries.js         | 5 +++++
 tez-ui/src/main/webapp/app/models/hive-query.js                | 1 +
 tez-ui/src/main/webapp/app/serializers/hive-query.js           | 1 +
 tez-ui/src/main/webapp/app/templates/query/index.hbs           | 6 ++++++
 .../main/webapp/tests/unit/controllers/home/queries-test.js    | 2 +-
 tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js    | 1 +
 .../src/main/webapp/tests/unit/serializers/hive-query-test.js  | 2 +-
 8 files changed, 17 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8bb3749..3806e27 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -206,6 +206,7 @@ ALL CHANGES:
   TEZ-3626. Tez UI: First Task Start Time & Last Task Finish Time values are showing up incorrectly
   TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling
   TEZ-3639. Tez UI: Footer pagination is improper in landing page
+  TEZ-3640. Tez UI: Add associated llap application id to queries page
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/app/controllers/home/queries.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js
index 57b3964..9c0b564 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/queries.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js
@@ -131,6 +131,11 @@ export default TableController.extend({
     headerTitle: 'Tables Written',
     contentPath: 'tablesWritten',
   },{
+    id: 'llapAppID',
+    headerTitle: 'LLAP App ID',
+    contentPath: 'llapAppID',
+    minWidth: "250px",
+  },{
     id: 'clientAddress',
     headerTitle: 'Client Address',
     contentPath: 'clientAddress',

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/app/models/hive-query.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/hive-query.js b/tez-ui/src/main/webapp/app/models/hive-query.js
index 2f2f129..38b4785 100644
--- a/tez-ui/src/main/webapp/app/models/hive-query.js
+++ b/tez-ui/src/main/webapp/app/models/hive-query.js
@@ -41,6 +41,7 @@ export default AbstractModel.extend({
 
   sessionID: DS.attr('string'),
   operationID: DS.attr('string'),
+  llapAppID: DS.attr('string'),
 
   instanceType: DS.attr('string'),
   executionMode: DS.attr('string'), // Would be ideally TEZ

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/app/serializers/hive-query.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/serializers/hive-query.js b/tez-ui/src/main/webapp/app/serializers/hive-query.js
index f4dd964..787bfea 100644
--- a/tez-ui/src/main/webapp/app/serializers/hive-query.js
+++ b/tez-ui/src/main/webapp/app/serializers/hive-query.js
@@ -53,6 +53,7 @@ export default TimelineSerializer.extend({
 
     sessionID: 'otherinfo.INVOKER_INFO',
     operationID: 'primaryfilters.operationid.0',
+    llapAppID: 'otherinfo.LLAP_APP_ID',
 
     instanceType: 'otherinfo.HIVE_INSTANCE_TYPE',
     executionMode: 'primaryfilters.executionmode.0',

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/app/templates/query/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/query/index.hbs b/tez-ui/src/main/webapp/app/templates/query/index.hbs
index ddb5d72..b90c627 100644
--- a/tez-ui/src/main/webapp/app/templates/query/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/query/index.hbs
@@ -69,6 +69,12 @@
       <td>Session ID</td>
       <td>{{txt model.sessionID}}</td>
     </tr>
+    {{#if model.llapAppID}}
+      <tr>
+        <td>LLAP App ID</td>
+        <td>{{txt model.llapAppID}}</td>
+      </tr>
+    {{/if}}
     <tr>
       <td>Thread Name</td>
       <td>{{txt model.threadName}}</td>

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
index 22461d2..34b05d0 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
@@ -42,7 +42,7 @@ test('Basic creation test', function(assert) {
 
   assert.ok(controller.definition);
   assert.ok(controller.columns);
-  assert.equal(controller.columns.length, 16);
+  assert.equal(controller.columns.length, 17);
 
   assert.ok(controller.getCounterColumns);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js b/tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js
index e44592c..42803fd 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/hive-query-test.js
@@ -36,6 +36,7 @@ test('Basic creation test', function(assert) {
 
   assert.ok(model.version);
 
+  assert.ok(model.llapAppID);
   assert.ok(model.sessionID);
   assert.ok(model.threadName);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/ee4a9a90/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
index fc0272b..4544c72 100644
--- a/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
@@ -26,7 +26,7 @@ moduleFor('serializer:hive-query', 'Unit | Serializer | hive query', {
 
 test('Basic creation test', function(assert) {
   let serializer = this.subject();
-  assert.equal(Object.keys(serializer.get("maps")).length, 6 + 20);
+  assert.equal(Object.keys(serializer.get("maps")).length, 6 + 21);
   assert.ok(serializer.get("extractAttributes"));
 });
 


[20/50] [abbrv] tez git commit: TEZ-3637. TezMerger logs too much at INFO level. (sseth)

Posted by zh...@apache.org.
TEZ-3637. TezMerger logs too much at INFO level. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: bdc0ee9c9ffcc9c199e0ca4245d7084f6df943c4
Parents: d5bf28b
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 27 19:26:31 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Feb 27 19:26:31 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../shuffle/orderedgrouped/MergeManager.java    | 213 +++++++++++++++----
 .../library/common/sort/impl/TezMerger.java     |  56 ++---
 3 files changed, 197 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/bdc0ee9c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c9ac898..88b0b98 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3637. TezMerger logs too much at INFO level
   TEZ-3638. VertexImpl logs too much at info when removing tasks after auto-reduce parallelism
   TEZ-3634. reduce the default buffer sizes in PipelinedSorter by a small amount.
   TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.

http://git-wip-us.apache.org/repos/asf/tez/blob/bdc0ee9c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
index 26bdca7..9f0e73c 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.io.FileChunk;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
@@ -146,6 +147,12 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
   private final int ifileReadAheadLength;
   private final int ifileBufferSize;
 
+  // Variables for stats and logging
+  private long lastInMemSegmentLogTime = -1L;
+  private final SegmentStatsTracker statsInMemTotal = new SegmentStatsTracker();
+  private final SegmentStatsTracker statsInMemLastLog = new SegmentStatsTracker();
+
+
   private AtomicInteger mergeFileSequenceId = new AtomicInteger(0);
 
   private final boolean cleanup;
@@ -465,13 +472,11 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     unreserve(size);
   }
 
+
   @Override
   public synchronized void closeInMemoryFile(MapOutput mapOutput) { 
     inMemoryMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
-          + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
-          + ", commitMemory -> " + commitMemory + ", usedMemory ->" + usedMemory + ", mapOutput=" +
-          mapOutput);
+    trackAndLogCloseInMemoryFile(mapOutput);
 
     commitMemory+= mapOutput.getSize();
 
@@ -490,6 +495,44 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     }
   }
 
+  private void trackAndLogCloseInMemoryFile(MapOutput mapOutput) {
+    statsInMemTotal.updateStats(mapOutput.getSize());
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
+          + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
+          + ", commitMemory -> " + commitMemory + ", usedMemory ->" +
+          usedMemory + ", mapOutput=" +
+          mapOutput);
+    } else {
+      statsInMemLastLog.updateStats(mapOutput.getSize());
+      long now = Time.monotonicNow();
+      if (now > lastInMemSegmentLogTime + 30 * 1000L) {
+        LOG.info(
+            "CloseInMemoryFile. Current state: inMemoryMapOutputs.size={}," +
+                " commitMemory={}," +
+                " usedMemory={}. Since last log:" +
+                " count={}," +
+                " min={}," +
+                " max={}," +
+                " total={}," +
+                " avg={}",
+            inMemoryMapOutputs.size(),
+            commitMemory,
+            usedMemory,
+            statsInMemLastLog.count,
+            statsInMemLastLog.minSize,
+            statsInMemLastLog.maxSize,
+            statsInMemLastLog.size,
+            (statsInMemLastLog.count == 0 ? "nan" :
+                (statsInMemLastLog.size / (double) statsInMemLastLog.count))
+        );
+        statsInMemLastLog.reset();
+        lastInMemSegmentLogTime = now;
+      }
+    }
+  }
+
   private void startMemToDiskMerge() {
     synchronized (inMemoryMerger) {
       if (!inMemoryMerger.isInProgress()) {
@@ -505,9 +548,13 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
   
   public synchronized void closeInMemoryMergedFile(MapOutput mapOutput) {
     inMemoryMergedMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() +
-             ", inMemoryMergedMapOutputs.size() -> " + 
-             inMemoryMergedMapOutputs.size());
+    if (LOG.isDebugEnabled()) {
+      // This log could be moved to INFO level for a while, after mem-to-mem
+      // merge is production ready.
+      LOG.debug("closeInMemoryMergedFile -> size: " + mapOutput.getSize() +
+          ", inMemoryMergedMapOutputs.size() -> " +
+          inMemoryMergedMapOutputs.size());
+    }
 
     commitMemory += mapOutput.getSize();
 
@@ -535,9 +582,7 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     }
 
     onDiskMapOutputs.add(file);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("close onDiskFile=" + file.getPath() + ", len=" + file.getLength());
-    }
+    logCloseOnDiskFile(file);
 
     synchronized (onDiskMerger) {
       if (!onDiskMerger.isInProgress() &&
@@ -547,6 +592,23 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     }
   }
 
+  private long lastOnDiskSegmentLogTime = -1L;
+  private void logCloseOnDiskFile(FileChunk file) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "close onDiskFile=" + file.getPath() + ", len=" + file.getLength() +
+              ", onDisMapOutputs=" + onDiskMapOutputs.size());
+    } else {
+      long now = Time.monotonicNow();
+      if (now > lastOnDiskSegmentLogTime + 30 * 1000L) {
+        LOG.info(
+            "close onDiskFile. State: NumOnDiskFiles={}. Current: path={}, len={}",
+            onDiskMapOutputs.size(), file.getPath(), file.getLength());
+        lastOnDiskSegmentLogTime = now;
+      }
+    }
+  }
+
   /**
    * Should <b>only</b> be used after the Shuffle phaze is complete, otherwise can
    * return an invalid state since a merge may not be in progress dur to
@@ -576,6 +638,14 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
       List<FileChunk> disk = new ArrayList<FileChunk>(onDiskMapOutputs);
       onDiskMapOutputs.clear();
 
+      if (statsInMemTotal.count > 0) {
+        LOG.info(
+            "TotalInMemFetchStats: count={}, totalSize={}, min={}, max={}, avg={}",
+            statsInMemTotal.count, statsInMemTotal.size,
+            statsInMemTotal.minSize, statsInMemTotal.maxSize,
+            (statsInMemTotal.size / (float) statsInMemTotal.size));
+      }
+
       // Don't attempt a final merge if close is invoked as a result of a previous
       // shuffle exception / error.
       if (tryFinalMerge) {
@@ -1069,21 +1139,9 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
                                        List<MapOutput> inMemoryMapOutputs,
                                        List<FileChunk> onDiskMapOutputs
                                        ) throws IOException, InterruptedException {
-    LOG.info("finalMerge called with " + 
-             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
-             onDiskMapOutputs.size() + " on-disk map-outputs");
 
-    if (LOG.isDebugEnabled()) {
-      for (MapOutput inMemoryMapOutput : inMemoryMapOutputs) {
-        LOG.debug("inMemoryOutput=" + inMemoryMapOutput + ", size=" + inMemoryMapOutput
-            .getSize());
-      }
-
-      for (FileChunk onDiskMapOutput : onDiskMapOutputs) {
-        LOG.debug("onDiskMapOutput=" + onDiskMapOutput.getPath() + ", size=" + onDiskMapOutput
-                .getLength());
-      }
-    }
+    logFinalMergeStart(inMemoryMapOutputs, onDiskMapOutputs);
+    StringBuilder finalMergeLog = new StringBuilder();
     
     inputContext.notifyProgress();
 
@@ -1148,15 +1206,25 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
         // add to list of final disk outputs.
         onDiskMapOutputs.add(new FileChunk(outputPath, 0, fStatus.getLen()));
 
-        LOG.info("Merged " + numMemDiskSegments + " segments, " +
-                 inMemToDiskBytes + " bytes to disk to satisfy " +
-                 "reduce memory limit. outputPath=" + outputPath);
+        if (LOG.isInfoEnabled()) {
+          finalMergeLog.append("MemMerged: " + numMemDiskSegments + ", " + inMemToDiskBytes);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Merged " + numMemDiskSegments + "segments, size=" +
+                inMemToDiskBytes + " to " + outputPath);
+          }
+        }
+
         inMemToDiskBytes = 0;
         memDiskSegments.clear();
       } else if (inMemToDiskBytes != 0) {
-        LOG.info("Keeping " + numMemDiskSegments + " segments, " +
-            inMemToDiskBytes + " bytes in memory for " +
-                 "intermediate, on-disk merge");
+        if (LOG.isInfoEnabled()) {
+          finalMergeLog.append("DelayedMemMerge: " + numMemDiskSegments + ", " + inMemToDiskBytes);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Keeping " + numMemDiskSegments + " segments, " +
+                inMemToDiskBytes + " bytes in memory for " +
+                "intermediate, on-disk merge");
+          }
+        }
       }
     }
 
@@ -1167,8 +1235,11 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     for (FileChunk fileChunk : onDisk) {
       final long fileLength = fileChunk.getLength();
       onDiskBytes += fileLength;
-      LOG.info("Disk file=" + fileChunk.getPath() + ", len=" + fileLength + ", isLocal=" +
-          fileChunk.isLocalFile());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Disk file=" + fileChunk.getPath() + ", len=" + fileLength +
+            ", isLocal=" +
+            fileChunk.isLocalFile());
+      }
 
       final Path file = fileChunk.getPath();
       TezCounter counter =
@@ -1179,8 +1250,13 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
       diskSegments.add(new DiskSegment(fs, file, fileOffset, fileLength, codec, ifileReadAhead,
                                    ifileReadAheadLength, ifileBufferSize, preserve, counter));
     }
-    LOG.info("Merging " + onDisk.length + " files, " +
-             onDiskBytes + " bytes from disk");
+    if (LOG.isInfoEnabled()) {
+      finalMergeLog.append(". DiskSeg: " + onDisk.length + ", " + onDiskBytes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merging " + onDisk.length + " files, " +
+            onDiskBytes + " bytes from disk");
+      }
+    }
     Collections.sort(diskSegments, new Comparator<Segment>() {
       public int compare(Segment o1, Segment o2) {
         if (o1.getLength() == o2.getLength()) {
@@ -1194,8 +1270,14 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
     List<Segment> finalSegments = new ArrayList<Segment>();
     long inMemBytes = createInMemorySegments(inMemoryMapOutputs, 
                                              finalSegments, 0);
-    LOG.info("Merging " + finalSegments.size() + " segments, " +
-             inMemBytes + " bytes from memory into reduce");
+    if (LOG.isInfoEnabled()) {
+      finalMergeLog.append(". MemSeg: " + finalSegments.size() + ", " + inMemBytes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merging " + finalSegments.size() + " segments, " +
+            inMemBytes + " bytes from memory into reduce");
+      }
+    }
+
     if (0 != onDiskBytes) {
       final int numInMemSegments = memDiskSegments.size();
       diskSegments.addAll(0, memDiskSegments);
@@ -1211,6 +1293,9 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
       finalSegments.add(new Segment(
             new RawKVIteratorReader(diskMerge, onDiskBytes), null));
     }
+    if (LOG.isInfoEnabled()) {
+      LOG.info(finalMergeLog.toString());
+    }
     // This is doing nothing but creating an iterator over the segments.
     return TezMerger.merge(job, fs, keyClass, valueClass,
                  finalSegments, finalSegments.size(), tmpDir,
@@ -1218,6 +1303,35 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
                  additionalBytesRead, null);
   }
 
+
+  private void logFinalMergeStart(List<MapOutput> inMemoryMapOutputs,
+                                  List<FileChunk> onDiskMapOutputs) {
+    long inMemSegmentSize = 0;
+    for (MapOutput inMemoryMapOutput : inMemoryMapOutputs) {
+      inMemSegmentSize += inMemoryMapOutput.getSize();
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("finalMerge: inMemoryOutput=" + inMemoryMapOutput + ", size=" +
+            inMemoryMapOutput.getSize());
+      }
+    }
+    long onDiskSegmentSize = 0;
+    for (FileChunk onDiskMapOutput : onDiskMapOutputs) {
+      onDiskSegmentSize += onDiskMapOutput.getLength();
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("finalMerge: onDiskMapOutput=" + onDiskMapOutput.getPath() +
+            ", size=" + onDiskMapOutput.getLength());
+      }
+    }
+
+    LOG.info(
+        "finalMerge with #inMemoryOutputs={}, size={} and #onDiskOutputs={}, size={}",
+        inMemoryMapOutputs.size(), inMemSegmentSize, onDiskMapOutputs.size(),
+        onDiskSegmentSize);
+
+  }
+
   @VisibleForTesting
   long getCommitMemory() {
     return commitMemory;
@@ -1232,4 +1346,31 @@ public class MergeManager implements FetchedInputAllocatorOrderedGrouped {
   void waitForMemToMemMerge() throws InterruptedException {
     memToMemMerger.waitForMerge();
   }
+
+
+
+  private static class SegmentStatsTracker {
+    private long size;
+    private int count;
+    private long minSize;
+    private long maxSize;
+
+    SegmentStatsTracker() {
+      reset();
+    }
+
+    void updateStats(long segSize) {
+      size += segSize;
+      count++;
+      minSize = (segSize < minSize ? segSize : minSize);
+      maxSize = (segSize > maxSize ? segSize : maxSize);
+    }
+
+    void reset() {
+      size = 0L;
+      count = 0;
+      minSize = Long.MAX_VALUE;
+      maxSize = Long.MIN_VALUE;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/bdc0ee9c/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
index 17e0fe2..8f3e84a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
@@ -90,32 +90,6 @@ public class TezMerger {
                                            mergePhase);
   }
 
-  public static 
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass, 
-                            CompressionCodec codec, boolean ifileReadAhead,
-                            int ifileReadAheadLength, int ifileBufferSize,
-                            Path[] inputs, boolean deleteInputs, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator,
-                            Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            TezCounter mergedMapOutputsCounter,
-                            TezCounter bytesReadCounter,
-                            Progress mergePhase)
-      throws IOException, InterruptedException {
-    return 
-      new MergeQueue(conf, fs, inputs, deleteInputs, codec, ifileReadAhead,
-                           ifileReadAheadLength, ifileBufferSize, false, comparator, 
-                           reporter, mergedMapOutputsCounter).merge(
-                                           keyClass, valueClass,
-                                           mergeFactor, tmpDir,
-                                           readsCounter, writesCounter,
-                                           bytesReadCounter,
-                                           mergePhase);
-  }
-  
   // Used by the in-memory merger.
   public static
   TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, 
@@ -225,8 +199,8 @@ public class TezMerger {
         }
       }
     }
-    if ((count > 0) && LOG.isDebugEnabled()) {
-      LOG.debug("writeFile SAME_KEY count=" + count);
+    if ((count > 0) && LOG.isTraceEnabled()) {
+      LOG.trace("writeFile SAME_KEY count=" + count);
     }
   }
 
@@ -510,7 +484,9 @@ public class TezMerger {
       this.considerFinalMergeForProgress = considerFinalMergeForProgress;
       
       for (Path file : inputs) {
-        LOG.debug("MergeQ: adding: " + file);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("MergeQ: adding: " + file);
+        }
         segments.add(new DiskSegment(fs, file, codec, ifileReadAhead,
                                       ifileReadAheadLength, ifileBufferSize,
                                       !deleteInputs, 
@@ -702,11 +678,13 @@ public class TezMerger {
                                      TezCounter bytesReadCounter,
                                      Progress mergePhase)
         throws IOException, InterruptedException {
-      LOG.info("Merging " + segments.size() + " sorted segments");
       if (segments.size() == 0) {
         LOG.info("Nothing to merge. Returning an empty iterator");
         return new EmptyIterator();
       }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merging " + segments.size() + " sorted segments");
+      }
 
       /*
        * If there are inMemory segments, then they come first in the segments
@@ -806,19 +784,23 @@ public class TezMerger {
             mergeProgress.set(totalBytesProcessed * progPerByte);
           else
             mergeProgress.set(1.0f); // Last pass and no segments left - we're done
-          
-          LOG.info("Down to the last merge-pass, with " + numSegments + 
-                   " segments left of total size: " +
-                   (totalBytes - totalBytesProcessed) + " bytes");
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Down to the last merge-pass, with " + numSegments +
+                " segments left of total size: " +
+                (totalBytes - totalBytesProcessed) + " bytes");
+          }
           // At this point, Factor Segments have not been physically
           // materialized. The merge will be done dynamically. Some of them may
           // be in-memory segments, other on-disk semgnets. Decision to be made
           // by a finalMerge is that is required.
           return this;
         } else {
-          LOG.info("Merging " + segmentsToMerge.size() + 
-                   " intermediate segments out of a total of " + 
-                   (segments.size()+segmentsToMerge.size()));
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Merging " + segmentsToMerge.size() +
+                " intermediate segments out of a total of " +
+                (segments.size() + segmentsToMerge.size()));
+          }
           
           long bytesProcessedInPrevMerges = totalBytesProcessed;
           totalBytesProcessed += startBytes;


[08/50] [abbrv] tez git commit: TEZ-3550. Provide access to sessionId/dagId via DagClient. (sseth)

Posted by zh...@apache.org.
TEZ-3550. Provide access to sessionId/dagId via DagClient. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: cd44c906f762c24a13ddabe691e55c879c223eac
Parents: e1b0b28
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 14 10:46:18 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 14 10:46:18 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                           |  2 ++
 .../java/org/apache/tez/dag/api/client/DAGClient.java | 13 +++++++++++++
 .../org/apache/tez/dag/api/client/DAGClientImpl.java  | 10 ++++++++++
 .../apache/tez/dag/api/client/DAGClientInternal.java  | 14 +++++++++++++-
 .../tez/dag/api/client/DAGClientTimelineImpl.java     | 11 ++++++++++-
 .../tez/dag/api/client/rpc/DAGClientRPCImpl.java      | 10 +++++++++-
 .../java/org/apache/tez/client/TestTezClient.java     |  5 ++++-
 .../apache/tez/dag/api/client/rpc/TestDAGClient.java  |  2 ++
 .../org/apache/tez/dag/api/client/MRDAGClient.java    | 10 ++++++++++
 9 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6991f05..337b394 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3267. Publish queue name to ATS as part of dag summary.
   TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent
@@ -201,6 +202,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3550. Provide access to sessionId/dagId via DagClient.
   TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
index 9b11b96..c70da75 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
@@ -84,6 +84,19 @@ public abstract class DAGClient implements Closeable {
     throws IOException, TezException;
 
   /**
+   * Get the dag identifier for the currently executing dag. This is a string
+   * which represents this dag
+   * @return the dag identifier
+   */
+  public abstract String getDagIdentifierString();
+
+  /**
+   * Get the session identifier for the session in which this dag is running
+   * @return the session identifier
+   */
+  public abstract String getSessionIdentifierString();
+
+  /**
    * Kill a running DAG
    *
    */

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
index 4820b6e..1cf0bfc 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
@@ -319,6 +319,16 @@ public class DAGClientImpl extends DAGClient {
   }
 
   @Override
+  public String getDagIdentifierString() {
+    return realClient.getDagIdentifierString();
+  }
+
+  @Override
+  public String getSessionIdentifierString() {
+    return realClient.getSessionIdentifierString();
+  }
+
+  @Override
   public void tryKillDAG() throws IOException, TezException {
     if (!dagCompleted) {
       realClient.tryKillDAG();

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java
index bb236a3..a3c898a 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java
@@ -25,7 +25,6 @@ import java.util.Set;
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.tez.dag.api.TezException;
@@ -85,6 +84,19 @@ public abstract class DAGClientInternal implements Closeable {
     throws IOException, TezException, ApplicationNotFoundException;
 
   /**
+   * Get the dag identifier for the currently executing dag. This is a string
+   * which represents this dag
+   * @return the dag identifier
+   */
+  public abstract String getDagIdentifierString();
+
+  /**
+   * Get the session identifier for the session in which this dag is running
+   * @return the session identifier
+   */
+  public abstract String getSessionIdentifierString();
+
+  /**
    * Kill a running DAG
    *
    */

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
index ffd91b7..d34dbf0 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
@@ -38,7 +38,6 @@ import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
 
-import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -182,6 +181,16 @@ public class DAGClientTimelineImpl extends DAGClientInternal {
   }
 
   @Override
+  public String getDagIdentifierString() {
+    return dagId;
+  }
+
+  @Override
+  public String getSessionIdentifierString() {
+    return appId.toString();
+  }
+
+  @Override
   public void tryKillDAG() throws IOException, TezException {
     throw new TezException("tryKillDAG is unsupported for DAGClientTimelineImpl");
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
index 9eb9807..02935df 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
@@ -41,7 +41,6 @@ import org.apache.tez.dag.api.DAGNotRunningException;
 import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.DagStatusSource;
 import org.apache.tez.dag.api.client.StatusGetOpts;
@@ -130,6 +129,15 @@ public class DAGClientRPCImpl extends DAGClientInternal {
     return null;
   }
 
+  @Override
+  public String getDagIdentifierString() {
+    return dagId.toString();
+  }
+
+  @Override
+  public String getSessionIdentifierString() {
+    return appId.toString();
+  }
 
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index c9415a4..89310df 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -277,7 +277,8 @@ public class TestTezClient {
     DAGClient dagClient = client.submitDAG(dag);
         
     assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
-    
+    assertEquals(dagClient.getSessionIdentifierString(), client.mockAppId.toString());
+
     if (isSession) {
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
       verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any());
@@ -316,6 +317,7 @@ public class TestTezClient {
       // same app master
       verify(client.mockYarnClient, times(1)).submitApplication(captor.capture());
       assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
+      assertEquals(dagClient.getSessionIdentifierString(), client.mockAppId.toString());
       // additional resource is sent
       ArgumentCaptor<SubmitDAGRequestProto> captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class);
       verify(client.sessionAmProxy, times(2)).submitDAG((RpcController)any(), captor1.capture());
@@ -325,6 +327,7 @@ public class TestTezClient {
     } else {
       // new app master
       assertTrue(dagClient.getExecutionContext().contains(appId2.toString()));
+      assertEquals(dagClient.getSessionIdentifierString(), appId2.toString());
       verify(client.mockYarnClient, times(2)).submitApplication(captor.capture());
       // additional resource is added
       ApplicationSubmissionContext context = captor.getValue();

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
index e979f8b..70ee1d4 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
@@ -213,6 +213,8 @@ public class TestDAGClient {
   @Test(timeout = 5000)
   public void testApp() throws IOException, TezException, ServiceException{
     assertTrue(dagClient.getExecutionContext().contains(mockAppId.toString()));
+    assertEquals(mockAppId.toString(), dagClient.getSessionIdentifierString());
+    assertEquals(dagIdStr, dagClient.getDagIdentifierString());
     DAGClientRPCImpl realClient = (DAGClientRPCImpl)((DAGClientImpl)dagClient).getRealClient();
     assertEquals(mockAppReport, realClient.getApplicationReportInternal());
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/cd44c906/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java
index d743feb..42b52e0 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java
@@ -66,6 +66,16 @@ public class MRDAGClient extends DAGClient {
   }
 
   @Override
+  public String getDagIdentifierString() {
+    return realClient.getDagIdentifierString();
+  }
+
+  @Override
+  public String getSessionIdentifierString() {
+    return realClient.getSessionIdentifierString();
+  }
+
+  @Override
   public void tryKillDAG() throws IOException, TezException {
     realClient.tryKillDAG();
   }


[49/50] [abbrv] tez git commit: TEZ-3671. TestCompositeDataMovementEvent has a misplaced Apache license header. Contributed by Saijin Huang

Posted by zh...@apache.org.
TEZ-3671. TestCompositeDataMovementEvent has a misplaced Apache license header. Contributed by Saijin Huang


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

Branch: refs/heads/TEZ-1190
Commit: 906c2a8e8d7c8edd631dbfa444624752c5d4034c
Parents: 618c811
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Tue Mar 28 10:16:59 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Tue Mar 28 10:16:59 2017 -0500

----------------------------------------------------------------------
 .../tez/runtime/api/event/TestCompositeDataMovementEvent.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/906c2a8e/tez-api/src/test/java/org/apache/tez/runtime/api/event/TestCompositeDataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/runtime/api/event/TestCompositeDataMovementEvent.java b/tez-api/src/test/java/org/apache/tez/runtime/api/event/TestCompositeDataMovementEvent.java
index eefbde7..7dce699 100644
--- a/tez-api/src/test/java/org/apache/tez/runtime/api/event/TestCompositeDataMovementEvent.java
+++ b/tez-api/src/test/java/org/apache/tez/runtime/api/event/TestCompositeDataMovementEvent.java
@@ -1,5 +1,3 @@
-package org.apache.tez.runtime.api.event;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -17,6 +15,9 @@ package org.apache.tez.runtime.api.event;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+package org.apache.tez.runtime.api.event;
+
 import java.nio.ByteBuffer;
 
 import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;


[03/50] [abbrv] tez git commit: TEZ-3602. Tez UI: Query Name field is not required (sree)

Posted by zh...@apache.org.
TEZ-3602. Tez UI: Query Name field is not required (sree)


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

Branch: refs/heads/TEZ-1190
Commit: b3a3af3f476e7c93d46158a3d82285596be5462d
Parents: d415197
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Mon Feb 6 14:49:00 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Mon Feb 6 14:49:00 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 tez-ui/src/main/webapp/app/adapters/timeline.js           |  1 -
 .../src/main/webapp/app/components/queries-page-search.js |  2 --
 tez-ui/src/main/webapp/app/controllers/home/queries.js    | 10 ++--------
 tez-ui/src/main/webapp/app/models/hive-query.js           |  2 --
 tez-ui/src/main/webapp/app/routes/home/queries.js         |  2 --
 tez-ui/src/main/webapp/app/serializers/hive-query.js      |  2 --
 tez-ui/src/main/webapp/app/templates/query/index.hbs      |  4 ----
 .../src/main/webapp/tests/unit/adapters/timeline-test.js  |  2 +-
 .../webapp/tests/unit/controllers/home/queries-test.js    |  7 +------
 .../main/webapp/tests/unit/routes/home/queries-test.js    |  3 +--
 .../main/webapp/tests/unit/serializers/hive-query-test.js |  2 +-
 12 files changed, 7 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c4a1d72..16e239f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -189,6 +189,7 @@ ALL CHANGES:
   TEZ-3593. Tez UI: Issues in timeline page
   TEZ-3594. Tez UI: Graphical view tooltip issues
   TEZ-3598. Tez UI: Text formatting changes
+  TEZ-3602. Tez UI: Query Name field is not required
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/adapters/timeline.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/adapters/timeline.js b/tez-ui/src/main/webapp/app/adapters/timeline.js
index 0029329..41cbd1c 100644
--- a/tez-ui/src/main/webapp/app/adapters/timeline.js
+++ b/tez-ui/src/main/webapp/app/adapters/timeline.js
@@ -42,7 +42,6 @@ export default AbstractAdapter.extend({
     executionMode: "executionmode",
     callerId: "callerId",
 
-    queryName: "queryname",
     tablesRead: "tablesread",
     tablesWritten: "tableswritten",
     operationID: "operationid",

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/components/queries-page-search.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/components/queries-page-search.js b/tez-ui/src/main/webapp/app/components/queries-page-search.js
index 0daf988..f5f2711 100644
--- a/tez-ui/src/main/webapp/app/components/queries-page-search.js
+++ b/tez-ui/src/main/webapp/app/components/queries-page-search.js
@@ -21,7 +21,6 @@ import Ember from 'ember';
 export default Ember.Component.extend({
   classNames: ['queries-page-search'],
 
-  queryName: Ember.computed.oneWay("tableDefinition.queryName"),
   queryID: Ember.computed.oneWay("tableDefinition.queryID"),
   dagID: Ember.computed.oneWay("tableDefinition.dagID"),
   appID: Ember.computed.oneWay("tableDefinition.appID"),
@@ -35,7 +34,6 @@ export default Ember.Component.extend({
 
   sendSearch: function () {
     this.get('parentView').sendAction('search', {
-      queryName: this.get("queryName"),
       queryID: this.get("queryID"),
       dagID: this.get("dagID"),
       appID: this.get("appID"),

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/controllers/home/queries.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js
index 4b7b9ce..648f98e 100644
--- a/tez-ui/src/main/webapp/app/controllers/home/queries.js
+++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js
@@ -24,9 +24,8 @@ import TableDefinition from 'em-table/utils/table-definition';
 
 export default TableController.extend({
 
-  queryParams: ["queryName", "queryID", "dagID", "appID", "user", "requestUser",
+  queryParams: ["queryID", "dagID", "appID", "user", "requestUser",
       "tablesRead", "tablesWritten", "operationID", "queue"],
-  queryName: "",
   queryID: "",
   dagID: "",
   appID: "",
@@ -54,14 +53,13 @@ export default TableController.extend({
 
   _definition: TableDefinition.create(),
   // Using computed, as observer won't fire if the property is not used
-  definition: Ember.computed("queryID", "queryName", "dagID", "appID", "user", "requestUser",
+  definition: Ember.computed("queryID", "dagID", "appID", "user", "requestUser",
       "executionMode", "tablesRead", "tablesWritten", "operationID", "queue",
       "pageNum", "moreAvailable", "loadingMore", function () {
 
     var definition = this.get("_definition");
 
     definition.setProperties({
-      queryName: this.get("queryName"),
       queryID: this.get("queryID"),
       dagID: this.get("dagID"),
       appID: this.get("appID"),
@@ -173,10 +171,6 @@ export default TableController.extend({
     headerTitle: 'Hive Server 2 Address',
     contentPath: 'hiveAddress'
   },{
-    id: 'queryName',
-    headerTitle: 'Query Name',
-    contentPath: 'queryName'
-  },{
     id: 'instanceType',
     headerTitle: 'Client Type',
     contentPath: 'instanceType'

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/models/hive-query.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/hive-query.js b/tez-ui/src/main/webapp/app/models/hive-query.js
index a640050..2f2f129 100644
--- a/tez-ui/src/main/webapp/app/models/hive-query.js
+++ b/tez-ui/src/main/webapp/app/models/hive-query.js
@@ -35,8 +35,6 @@ export default AbstractModel.extend({
     }
   },
 
-  queryName: DS.attr('string'),
-
   queryText: DS.attr("string"),
 
   dag: DS.attr('object'),

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/routes/home/queries.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/routes/home/queries.js b/tez-ui/src/main/webapp/app/routes/home/queries.js
index 50d9f54..7d09a2e 100644
--- a/tez-ui/src/main/webapp/app/routes/home/queries.js
+++ b/tez-ui/src/main/webapp/app/routes/home/queries.js
@@ -26,7 +26,6 @@ export default ServerSideOpsRoute.extend({
   title: "Hive Queries",
 
   queryParams: {
-    queryName: REFRESH,
     queryID: REFRESH,
     dagID: REFRESH,
     appID: REFRESH,
@@ -42,7 +41,6 @@ export default ServerSideOpsRoute.extend({
   },
 
   loaderQueryParams: {
-    queryName: "queryName",
     id: "queryID",
     dagID: "dagID",
     appID: "appID",

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/serializers/hive-query.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/serializers/hive-query.js b/tez-ui/src/main/webapp/app/serializers/hive-query.js
index dd5c1b8..f4dd964 100644
--- a/tez-ui/src/main/webapp/app/serializers/hive-query.js
+++ b/tez-ui/src/main/webapp/app/serializers/hive-query.js
@@ -49,8 +49,6 @@ function getStatus(source) {
 
 export default TimelineSerializer.extend({
   maps: {
-    queryName: 'primaryfilters.queryname.0',
-
     queryText: 'otherinfo.QUERY.queryText',
 
     sessionID: 'otherinfo.INVOKER_INFO',

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/app/templates/query/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/query/index.hbs b/tez-ui/src/main/webapp/app/templates/query/index.hbs
index 1cf2e36..ddb5d72 100644
--- a/tez-ui/src/main/webapp/app/templates/query/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/query/index.hbs
@@ -70,10 +70,6 @@
       <td>{{txt model.sessionID}}</td>
     </tr>
     <tr>
-      <td>Query Name</td>
-      <td>{{txt model.queryName}}</td>
-    </tr>
-    <tr>
       <td>Thread Name</td>
       <td>{{txt model.threadName}}</td>
     </tr>

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js b/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
index d189b5d..c7eab39 100644
--- a/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/adapters/timeline-test.js
@@ -37,7 +37,7 @@ test('Basic creation test', function(assert) {
 
 test('filters test', function(assert) {
   let filters = this.subject().filters;
-  assert.equal(Object.keys(filters).length, 6 + 7 + 5);
+  assert.equal(Object.keys(filters).length, 6 + 7 + 4);
 });
 
 test('stringifyFilters test', function(assert) {

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
index 753015d..cf36c7a 100644
--- a/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/controllers/home/queries-test.js
@@ -33,7 +33,7 @@ test('Basic creation test', function(assert) {
   assert.ok(controller);
 
   assert.ok(controller.queryParams);
-  assert.equal(controller.queryParams.length, 10 + 5);
+  assert.equal(controller.queryParams.length, 9 + 5);
 
   assert.ok(controller.breadcrumbs);
   assert.ok(controller.headerComponentNames);
@@ -59,7 +59,6 @@ test('definition test', function(assert) {
       }),
       definition = controller.get("definition"),
 
-      testQueryName = "QueryName",
       testQueryID = "QueryID",
       testDagID = "DagID",
       testAppID = "AppID",
@@ -74,7 +73,6 @@ test('definition test', function(assert) {
       testMoreAvailable = true,
       testLoadingMore = true;
 
-  assert.equal(definition.get("queryName"), "");
   assert.equal(definition.get("queryID"), "");
   assert.equal(definition.get("dagID"), "");
   assert.equal(definition.get("appID"), "");
@@ -91,9 +89,6 @@ test('definition test', function(assert) {
   assert.equal(definition.get("loadingMore"), false);
 
   Ember.run(function () {
-    controller.set("queryName", testQueryName);
-    assert.equal(controller.get("definition.queryName"), testQueryName);
-
     controller.set("queryID", testQueryID);
     assert.equal(controller.get("definition.queryID"), testQueryID);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/tests/unit/routes/home/queries-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/routes/home/queries-test.js b/tez-ui/src/main/webapp/tests/unit/routes/home/queries-test.js
index 2e39928..5d5df52 100644
--- a/tez-ui/src/main/webapp/tests/unit/routes/home/queries-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/routes/home/queries-test.js
@@ -43,7 +43,6 @@ test('it exists', function(assert) {
 test('refresh test', function(assert) {
   let route = this.subject();
 
-  assert.equal(route.get("queryParams.queryName.refreshModel"), true);
   assert.equal(route.get("queryParams.queryID.refreshModel"), true);
   assert.equal(route.get("queryParams.dagID.refreshModel"), true);
   assert.equal(route.get("queryParams.appID.refreshModel"), true);
@@ -60,7 +59,7 @@ test('refresh test', function(assert) {
 
 test('loaderQueryParams test', function(assert) {
   let route = this.subject();
-  assert.equal(Object.keys(route.get("loaderQueryParams")).length, 11 + 1);
+  assert.equal(Object.keys(route.get("loaderQueryParams")).length, 10 + 1);
 });
 
 test('load - query test', function(assert) {

http://git-wip-us.apache.org/repos/asf/tez/blob/b3a3af3f/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
index 4544c72..fc0272b 100644
--- a/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/serializers/hive-query-test.js
@@ -26,7 +26,7 @@ moduleFor('serializer:hive-query', 'Unit | Serializer | hive query', {
 
 test('Basic creation test', function(assert) {
   let serializer = this.subject();
-  assert.equal(Object.keys(serializer.get("maps")).length, 6 + 21);
+  assert.equal(Object.keys(serializer.get("maps")).length, 6 + 20);
   assert.ok(serializer.get("extractAttributes"));
 });
 


[13/50] [abbrv] tez git commit: TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS. Contributed by Harish Jaiprakash.

Posted by zh...@apache.org.
TEZ-3627. Use queue name available in RegisterApplicationMasterResponse
for publishing to ATS. Contributed by Harish Jaiprakash.


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

Branch: refs/heads/TEZ-1190
Commit: 8c311e4127f18e6d468e361a7805f87e84544c25
Parents: 10ded7c
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 21 21:16:51 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 21 21:16:51 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/tez/common/ATSConstants.java     |  2 +-
 .../api/TaskSchedulerContext.java               |  3 ++-
 .../java/org/apache/tez/dag/app/AppContext.java |  5 ++++
 .../org/apache/tez/dag/app/DAGAppMaster.java    | 24 +++++++++++---------
 .../dag/app/rm/TaskSchedulerContextImpl.java    |  7 +++---
 .../app/rm/TaskSchedulerContextImplWrapper.java | 11 +++++----
 .../tez/dag/app/rm/TaskSchedulerManager.java    |  4 +++-
 .../dag/app/rm/YarnTaskSchedulerService.java    |  3 ++-
 .../impl/HistoryEventJsonConversion.java        |  5 +++-
 .../tez/dag/app/rm/TestTaskScheduler.java       |  3 ++-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |  4 ++--
 .../impl/TestHistoryEventJsonConversion.java    |  4 +++-
 .../ats/HistoryEventTimelineConversion.java     |  5 +++-
 .../ats/TestHistoryEventTimelineConversion.java |  7 ++++--
 15 files changed, 58 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f34252b..a5c59ca 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3627. Use queue name available in RegisterApplicationMasterResponse for publishing to ATS.
   TEZ-3610. TEZ UI 0.7 0.9 compatibility for url query params and tez-app sub-routes
   TEZ-3625. Dag.getVertex should obtain a readlock.
   TEZ-3624. Split multiple calls on the same line in TaskCommunicatorContextImpl.

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
index 03c9fa1..25c802e 100644
--- a/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/common/ATSConstants.java
@@ -55,7 +55,7 @@ public class ATSConstants {
   public static final String DAG_PLAN = "dagPlan";
   public static final String DAG_NAME = "dagName";
   public static final String DAG_STATE = "dagState";
-  public static final String DAG_SUBMITTED_QUEUE_NAME = "submittedQueueName";
+  public static final String DAG_QUEUE_NAME = "queueName";
   public static final String DAG_AM_WEB_SERVICE_VERSION = "amWebServiceVersion";
   public static final String RECOVERY_FAILURE_REASON = "recoveryFailureReason";
   public static final String VERTEX_NAME = "vertexName";

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
index d30ada3..97fe7ae 100644
--- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/TaskSchedulerContext.java
@@ -131,7 +131,8 @@ public interface TaskSchedulerContext extends ServicePluginContextBase {
   void setApplicationRegistrationData(
       Resource maxContainerCapability,
       Map<ApplicationAccessType, String> appAcls,
-      ByteBuffer clientAMSecretKey
+      ByteBuffer clientAMSecretKey,
+      String queueName
   );
 
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 45ce8c1..b3d561a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -116,6 +117,10 @@ public interface AppContext {
 
   String getAMUser();
 
+  String getQueueName();
+
+  void setQueueName(String queueName);
+
   /** Whether the AM is in the process of shutting down/completing */
   boolean isAMInCompletionState();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 5a43358..2085789 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -1528,7 +1528,6 @@ public class DAGAppMaster extends AbstractService {
   }
 
   private class RunningAppContext implements AppContext {
-
     private DAG dag;
     private DAGRecoveryData dagRecoveryData;
     private final Configuration conf;
@@ -1537,6 +1536,8 @@ public class DAGAppMaster extends AbstractService {
     private final Lock rLock = rwLock.readLock();
     private final Lock wLock = rwLock.writeLock();
     private final EventHandler eventHandler;
+    private volatile String queueName;
+
     public RunningAppContext(Configuration config) {
       checkNotNull(config, "config is null");
       this.conf = config;
@@ -1793,6 +1794,16 @@ public class DAGAppMaster extends AbstractService {
     public DAGRecoveryData getDAGRecoveryData() {
       return dagRecoveryData;
     }
+
+    @Override
+    public String getQueueName() {
+      return queueName;
+    }
+
+    @Override
+    public void setQueueName(String queueName) {
+      this.queueName = queueName;
+    }
   }
 
   private static class ServiceWithDependency implements ServiceStateChangeListener {
@@ -2578,7 +2589,7 @@ public class DAGAppMaster extends AbstractService {
     // for an app later
     final DAGSubmittedEvent submittedEvent = new DAGSubmittedEvent(newDAG.getID(),
         submitTime, dagPlan, this.appAttemptID, cumulativeAdditionalResources,
-        newDAG.getUserName(), newDAG.getConf(), containerLogs, getSubmittedQueueName());
+        newDAG.getUserName(), newDAG.getConf(), containerLogs, getContext().getQueueName());
     boolean dagLoggingEnabled = newDAG.getConf().getBoolean(
         TezConfiguration.TEZ_DAG_HISTORY_LOGGING_ENABLED,
         TezConfiguration.TEZ_DAG_HISTORY_LOGGING_ENABLED_DEFAULT);
@@ -2672,15 +2683,6 @@ public class DAGAppMaster extends AbstractService {
     });
   }
 
-  private String getSubmittedQueueName() {
-    // TODO: Replace this with constant once the yarn patch is backported. (JIRA: TEZ-3279)
-    String submittedQueueName = System.getenv("YARN_RESOURCEMANAGER_APPLICATION_QUEUE");
-    if (submittedQueueName == null) {
-      submittedQueueName = amConf.get(TezConfiguration.TEZ_QUEUE_NAME);
-    }
-    return submittedQueueName;
-  }
-
   @SuppressWarnings("unchecked")
   private void sendEvent(Event<?> event) {
     dispatcher.getEventHandler().handle(event);

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
index fb4198b..39000d6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImpl.java
@@ -92,9 +92,10 @@ public class TaskSchedulerContextImpl implements TaskSchedulerContext {
   @Override
   public void setApplicationRegistrationData(Resource maxContainerCapability,
                                              Map<ApplicationAccessType, String> appAcls,
-                                             ByteBuffer clientAMSecretKey) {
-    taskSchedulerManager.setApplicationRegistrationData(schedulerId, maxContainerCapability, appAcls,
-        clientAMSecretKey);
+                                             ByteBuffer clientAMSecretKey,
+                                             String queueName) {
+    taskSchedulerManager.setApplicationRegistrationData(schedulerId, maxContainerCapability,
+        appAcls, clientAMSecretKey, queueName);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
index 7e1988b..49ab77d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerContextImplWrapper.java
@@ -95,9 +95,9 @@ class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
 
   @Override
   public void setApplicationRegistrationData(Resource maxContainerCapability,
-      Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
+      Map<ApplicationAccessType, String> appAcls, ByteBuffer key, String queueName) {
     executorService.submit(new SetApplicationRegistrationDataCallable(real,
-        maxContainerCapability, appAcls, key));
+        maxContainerCapability, appAcls, key, queueName));
   }
 
   @Override
@@ -295,20 +295,23 @@ class TaskSchedulerContextImplWrapper implements TaskSchedulerContext {
     private final Resource maxContainerCapability;
     private final Map<ApplicationAccessType, String> appAcls;
     private final ByteBuffer key;
+    private final String queueName;
 
     public SetApplicationRegistrationDataCallable(TaskSchedulerContext app,
         Resource maxContainerCapability,
         Map<ApplicationAccessType, String> appAcls,
-        ByteBuffer key) {
+        ByteBuffer key,
+        String queueName) {
       super(app);
       this.maxContainerCapability = maxContainerCapability;
       this.appAcls = appAcls;
       this.key = key;
+      this.queueName = queueName;
     }
 
     @Override
     public Void call() throws Exception {
-      app.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
+      app.setApplicationRegistrationData(maxContainerCapability, appAcls, key, queueName);
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java
index 7989e5f..d32261f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java
@@ -768,9 +768,11 @@ public class TaskSchedulerManager extends AbstractService implements
       int schedulerId,
       Resource maxContainerCapability,
       Map<ApplicationAccessType, String> appAcls, 
-      ByteBuffer clientAMSecretKey) {
+      ByteBuffer clientAMSecretKey,
+      String queueName) {
     this.appContext.getClusterInfo().setMaxContainerCapability(
         maxContainerCapability);
+    this.appContext.setQueueName(queueName);
     this.appAcls = appAcls;
     this.clientService.setClientAMSecretKey(clientAMSecretKey);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
index 41d380a..95cd85b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java
@@ -371,7 +371,8 @@ public class YarnTaskSchedulerService extends TaskScheduler
       getContext().setApplicationRegistrationData(
           response.getMaximumResourceCapability(),
           response.getApplicationACLs(),
-          response.getClientToAMTokenMasterKey());
+          response.getClientToAMTokenMasterKey(),
+          response.getQueue());
 
       delayedContainerManager.start();
     } catch (YarnException e) {

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
index 69c40e4..e60575f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
@@ -490,6 +490,9 @@ public class HistoryEventJsonConversion {
       primaryFilters.put(ATSConstants.CALLER_CONTEXT_TYPE,
           event.getDAGPlan().getCallerContext().getCallerType());
     }
+    if (event.getQueueName() != null) {
+      primaryFilters.put(ATSConstants.DAG_QUEUE_NAME, event.getQueueName());
+    }
 
     jsonObject.put(ATSConstants.PRIMARY_FILTERS, primaryFilters);
 
@@ -516,7 +519,7 @@ public class HistoryEventJsonConversion {
           event.getDAGPlan().getCallerContext().getCallerType());
     }
     if (event.getQueueName() != null) {
-      otherInfo.put(ATSConstants.DAG_SUBMITTED_QUEUE_NAME, event.getQueueName());
+      otherInfo.put(ATSConstants.DAG_QUEUE_NAME, event.getQueueName());
     }
     jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
index b3511e8..16c560e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskScheduler.java
@@ -150,7 +150,8 @@ public class TestTaskScheduler {
     RegisterApplicationMasterResponse regResponse = mockRMClient.getRegistrationResponse();
     verify(mockApp).setApplicationRegistrationData(regResponse.getMaximumResourceCapability(),
                                                    regResponse.getApplicationACLs(),
-                                                   regResponse.getClientToAMTokenMasterKey());
+                                                   regResponse.getClientToAMTokenMasterKey(),
+                                                   regResponse.getQueue());
 
     Assert.assertEquals(scheduler.getClusterNodeCount(), mockRMClient.getClusterNodeCount());
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
index 9a845a1..35ab30b 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java
@@ -287,9 +287,9 @@ class TestTaskSchedulerHelpers {
 
     @Override
     public void setApplicationRegistrationData(Resource maxContainerCapability,
-        Map<ApplicationAccessType, String> appAcls, ByteBuffer key) {
+        Map<ApplicationAccessType, String> appAcls, ByteBuffer key, String queueName) {
       invocations++;
-      real.setApplicationRegistrationData(maxContainerCapability, appAcls, key);
+      real.setApplicationRegistrationData(maxContainerCapability, appAcls, key, queueName);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
index 1bbecd3..081e47f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
@@ -204,7 +204,9 @@ public class TestHistoryEventJsonConversion {
       if (eventType == HistoryEventType.DAG_SUBMITTED) {
         try {
           Assert.assertEquals("Q_" + eventType.name(), json.getJSONObject(ATSConstants.OTHER_INFO)
-              .getString(ATSConstants.DAG_SUBMITTED_QUEUE_NAME));
+              .getString(ATSConstants.DAG_QUEUE_NAME));
+          Assert.assertEquals("Q_" + eventType.name(), json
+              .getJSONObject(ATSConstants.PRIMARY_FILTERS).getString(ATSConstants.DAG_QUEUE_NAME));
         } catch (JSONException ex) {
           Assert.fail("Exception: " + ex.getMessage() + " for type: " + eventType);
         }

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
index 8d0c547..faccc98 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
@@ -400,6 +400,9 @@ public class HistoryEventTimelineConversion {
       atsEntity.addPrimaryFilter(ATSConstants.CALLER_CONTEXT_ID,
           event.getDAGPlan().getCallerContext().getCallerId());
     }
+    if (event.getQueueName() != null) {
+      atsEntity.addPrimaryFilter(ATSConstants.DAG_QUEUE_NAME, event.getQueueName());
+    }
 
     try {
       atsEntity.addOtherInfo(ATSConstants.DAG_PLAN,
@@ -424,7 +427,7 @@ public class HistoryEventTimelineConversion {
           event.getDAGPlan().getCallerContext().getCallerType());
     }
     if (event.getQueueName() != null) {
-      atsEntity.addOtherInfo(ATSConstants.DAG_SUBMITTED_QUEUE_NAME, event.getQueueName());
+      atsEntity.addOtherInfo(ATSConstants.DAG_QUEUE_NAME, event.getQueueName());
     }
 
     return atsEntity;

http://git-wip-us.apache.org/repos/asf/tez/blob/8c311e41/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index bb189d3..28fd5da 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -496,7 +496,7 @@ public class TestHistoryEventTimelineConversion {
 
     Assert.assertEquals(submitTime, timelineEntity.getStartTime().longValue());
 
-    Assert.assertEquals(4, timelineEntity.getPrimaryFilters().size());
+    Assert.assertEquals(5, timelineEntity.getPrimaryFilters().size());
 
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(ATSConstants.DAG_NAME).contains(
@@ -509,6 +509,9 @@ public class TestHistoryEventTimelineConversion {
             applicationAttemptId.getApplicationId().toString()));
     Assert.assertTrue(
         timelineEntity.getPrimaryFilters().get(ATSConstants.USER).contains(user));
+    Assert.assertTrue(
+        timelineEntity.getPrimaryFilters().get(ATSConstants.DAG_QUEUE_NAME)
+            .contains(queueName));
 
     Assert.assertEquals(9, timelineEntity.getOtherInfo().size());
     Assert.assertTrue(timelineEntity.getOtherInfo().containsKey(ATSConstants.DAG_PLAN));
@@ -532,7 +535,7 @@ public class TestHistoryEventTimelineConversion {
         timelineEntity.getOtherInfo().get(ATSConstants.CALLER_CONTEXT_TYPE),
             dagPlan.getCallerContext().getCallerType());
     Assert.assertEquals(
-        queueName, timelineEntity.getOtherInfo().get(ATSConstants.DAG_SUBMITTED_QUEUE_NAME));
+        queueName, timelineEntity.getOtherInfo().get(ATSConstants.DAG_QUEUE_NAME));
 
   }
 


[04/50] [abbrv] tez git commit: TEZ-3244. Allow overlap of input and output memory when they are not concurrent. (jlowe)

Posted by zh...@apache.org.
TEZ-3244. Allow overlap of input and output memory when they are not concurrent. (jlowe)


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

Branch: refs/heads/TEZ-1190
Commit: 63ae97d5f3fe6e30e3c5f7c9a892ef9902e83b39
Parents: b3a3af3
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Tue Feb 7 13:32:37 2017 -0600
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Tue Feb 7 13:32:37 2017 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../apache/tez/dag/api/TezConfiguration.java    |  30 ++++
 .../common/resources/MemoryDistributor.java     |  12 +-
 .../WeightedScalingMemoryDistributor.java       |  62 ++++++-
 .../TestWeightedScalingMemoryDistributor.java   | 165 +++++++++++++++++++
 5 files changed, 264 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/63ae97d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 16e239f..a7cc0ce 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3581. Add different logger to enable suppressing logs for specific lines.
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3600. Fix flaky test: TestTokenCache
@@ -197,6 +198,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3462. Task attempt failure during container shutdown loses useful container diagnostics
   TEZ-3574. Container reuse won't pickup extra dag level local resource.

http://git-wip-us.apache.org/repos/asf/tez/blob/63ae97d5/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index fd71b35..94f40bb 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -875,6 +875,36 @@ public class TezConfiguration extends Configuration {
   public static final String TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS =
       TEZ_TASK_PREFIX + "scale.memory.ratios";
 
+  /**
+   * Concurrent input/output memory allocation control. When enabled memory
+   * distributions assume that inputs and outputs will use their memory
+   * simultaneously. When disabled the distributions assume that outputs are not
+   * initialized until inputs release memory buffers, allowing inputs to
+   * leverage memory normally set aside for outputs and vice-versa.
+   * NOTE: This property currently is not supported by the ScalingAllocator
+   *       memory distributor.
+   */
+  @Private
+  @Unstable
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT =
+      TEZ_TASK_PREFIX + "scale.memory.input-output-concurrent";
+  public static final boolean TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT_DEFAULT = true;
+
+  /**
+   * Controls distributing output memory to inputs when non-concurrent I/O
+   * memory allocation is being used.  When enabled inputs will receive the
+   * same memory allocation as if concurrent I/O memory allocation were used.
+   * NOTE: This property currently is not supported by the ScalingAllocator
+   *       memory distributor.
+   */
+  @Private
+  @Unstable
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED =
+      TEZ_TASK_PREFIX + "scale.memory.non-concurrent-inputs.enabled";
+  public static final boolean TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED_DEFAULT = false;
+
   @Private
   @Unstable
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/63ae97d5/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java
index c822357..e63a414 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/MemoryDistributor.java
@@ -61,6 +61,7 @@ public class MemoryDistributor {
 
   private long totalJvmMemory;
   private final boolean isEnabled;
+  private final boolean isInputOutputConcurrent;
   private final String allocatorClassName;
   private final Set<TaskContext> dupSet = Collections
       .newSetFromMap(new ConcurrentHashMap<TaskContext, Boolean>());
@@ -78,6 +79,9 @@ public class MemoryDistributor {
     this.conf = conf;
     isEnabled = conf.getBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ENABLED,
         TezConfiguration.TEZ_TASK_SCALE_MEMORY_ENABLED_DEFAULT);
+    isInputOutputConcurrent = conf.getBoolean(
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT,
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT_DEFAULT);
 
     if (isEnabled) {
       allocatorClassName = conf.get(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS,
@@ -213,9 +217,11 @@ public class MemoryDistributor {
     Preconditions.checkState(numAllocations == numRequestors,
         "Number of allocations must match number of requestors. Allocated=" + numAllocations
             + ", Requests: " + numRequestors);
-    Preconditions.checkState(totalAllocated <= totalJvmMemory,
-        "Total allocation should be <= availableMem. TotalAllocated: " + totalAllocated
-            + ", totalJvmMemory: " + totalJvmMemory);
+    if (isInputOutputConcurrent) {
+      Preconditions.checkState(totalAllocated <= totalJvmMemory,
+          "Total allocation should be <= availableMem. TotalAllocated: " + totalAllocated
+              + ", totalJvmMemory: " + totalJvmMemory);
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/63ae97d5/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java
index 8477300..c5b4fb0 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.runtime.common.resources.InitialMemoryAllocator;
 import org.apache.tez.runtime.common.resources.InitialMemoryRequestContext;
+import org.apache.tez.runtime.common.resources.InitialMemoryRequestContext.ComponentType;
 import org.apache.tez.runtime.library.input.OrderedGroupedKVInput;
 import org.apache.tez.runtime.library.input.OrderedGroupedInputLegacy;
 import org.apache.tez.runtime.library.input.UnorderedKVInput;
@@ -129,9 +130,15 @@ public class WeightedScalingMemoryDistributor implements InitialMemoryAllocator
         + availableForAllocation + ", TotalRequested/TotalJVMHeap:"
         + new DecimalFormat("0.00").format(ratio));
 
+    int numInputRequestsScaled = 0;
+    int numOutputRequestsScaled = 0;
+    long totalInputAllocated = 0;
+    long totalOutputAllocated = 0;
+
     // Actual scaling
     List<Long> allocations = Lists.newArrayListWithCapacity(numRequests);
     for (Request request : requests) {
+      long allocated = 0;
       if (request.requestSize == 0) {
         allocations.add(0l);
         if (LOG.isDebugEnabled()) {
@@ -141,7 +148,7 @@ public class WeightedScalingMemoryDistributor implements InitialMemoryAllocator
       } else {
         double requestFactor = request.requestWeight / (double) numRequestsScaled;
         double scaledRequest = requestFactor * request.requestSize;
-        long allocated = Math.min(
+        allocated = Math.min(
             (long) ((scaledRequest / totalScaledRequest) * availableForAllocation),
             request.requestSize);
         // TODO Later - If requestedSize is used, the difference (allocated -
@@ -152,9 +159,52 @@ public class WeightedScalingMemoryDistributor implements InitialMemoryAllocator
               + request.requestType + " " + request.requestSize + "  to allocated: " + allocated);
         }
       }
+
+      if (request.componentType == ComponentType.INPUT) {
+        numInputRequestsScaled += request.requestWeight;
+        totalInputAllocated += allocated;
+      } else if (request.componentType == ComponentType.OUTPUT) {
+        numOutputRequestsScaled += request.requestWeight;
+        totalOutputAllocated += allocated;
+      }
+    }
+
+    if (!conf.getBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT,
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT_DEFAULT)) {
+      adjustAllocationsForNonConcurrent(allocations, requests,
+          numInputRequestsScaled, totalInputAllocated,
+          numOutputRequestsScaled, totalOutputAllocated);
     }
+
     return allocations;
+  }
 
+  private void adjustAllocationsForNonConcurrent(List<Long> allocations,
+      List<Request> requests, int numInputsScaled, long totalInputAllocated,
+      int numOutputsScaled, long totalOutputAllocated) {
+    boolean inputsEnabled = conf.getBoolean(
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED,
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED_DEFAULT);
+    LOG.info("Adjusting scaled allocations for I/O non-concurrent."
+        + " numInputsScaled: {} InputAllocated: {} numOutputsScaled: {} outputAllocated: {} inputsEnabled: {}",
+        numInputsScaled, totalInputAllocated, numOutputsScaled, totalOutputAllocated, inputsEnabled);
+    for (int i = 0; i < requests.size(); i++) {
+      Request request = requests.get(i);
+      long additional = 0;
+      if (request.componentType == ComponentType.INPUT && inputsEnabled) {
+        double share = request.requestWeight / (double)numInputsScaled;
+        additional = (long) (totalOutputAllocated * share);
+      } else if (request.componentType == ComponentType.OUTPUT) {
+        double share = request.requestWeight / (double)numOutputsScaled;
+        additional = (long) (totalInputAllocated * share);
+      }
+      if (additional > 0) {
+        long newTotal = Math.min(allocations.get(i) + additional, request.requestSize);
+        // TODO Later - If requestedSize is used, the difference could be allocated to others.
+        allocations.set(i, newTotal);
+        LOG.debug("Adding {} to {} total={}", additional, request.componentClassname, newTotal);
+      }
+    }
   }
 
   private void initialProcessMemoryRequestContext(InitialMemoryRequestContext context) {
@@ -164,9 +214,10 @@ public class WeightedScalingMemoryDistributor implements InitialMemoryAllocator
     String className = context.getComponentClassName();
     requestType = getRequestTypeForClass(className);
     Integer typeScaleFactor = getScaleFactorForType(requestType);
+    ComponentType componentType = context.getComponentType();
 
-    Request request = new Request(context.getComponentClassName(), context.getRequestedSize(),
-        requestType, typeScaleFactor);
+    Request request = new Request(context.getComponentClassName(), componentType,
+        context.getRequestedSize(), requestType, typeScaleFactor);
     requests.add(request);
     numRequestsScaled += typeScaleFactor;
   }
@@ -293,14 +344,17 @@ public class WeightedScalingMemoryDistributor implements InitialMemoryAllocator
   }
 
   private static class Request {
-    Request(String componentClassname, long requestSize, RequestType requestType, int requestWeight) {
+    Request(String componentClassname, ComponentType componentType, long requestSize,
+        RequestType requestType, int requestWeight) {
       this.componentClassname = componentClassname;
+      this.componentType = componentType;
       this.requestSize = requestSize;
       this.requestType = requestType;
       this.requestWeight = requestWeight;
     }
 
     String componentClassname;
+    ComponentType componentType;
     long requestSize;
     private RequestType requestType;
     private int requestWeight;

http://git-wip-us.apache.org/repos/asf/tez/blob/63ae97d5/tez-runtime-library/src/test/java/org/apache/tez/runtime/common/resources/TestWeightedScalingMemoryDistributor.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/common/resources/TestWeightedScalingMemoryDistributor.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/common/resources/TestWeightedScalingMemoryDistributor.java
index a38497c..2fbe264 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/common/resources/TestWeightedScalingMemoryDistributor.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/common/resources/TestWeightedScalingMemoryDistributor.java
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.runtime.api.LogicalInput;
@@ -32,6 +33,7 @@ import org.apache.tez.runtime.api.LogicalOutput;
 import org.apache.tez.runtime.api.MemoryUpdateCallback;
 import org.apache.tez.runtime.api.InputContext;
 import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.ProcessorContext;
 import org.apache.tez.runtime.library.input.OrderedGroupedKVInput;
 import org.apache.tez.runtime.library.input.UnorderedKVInput;
 import org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput;
@@ -145,6 +147,169 @@ public class TestWeightedScalingMemoryDistributor extends TestMemoryDistributor
     assertEquals(1500, e4Callback.assigned);
   }
   
+  @Test(timeout = 5000)
+  public void testWeightedScalingNonConcurrent() throws TezException {
+    Configuration conf = new Configuration(this.conf);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT, false);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED, true);
+    conf.setDouble(TezConfiguration.TEZ_TASK_SCALE_MEMORY_RESERVE_FRACTION, 0.2);
+    conf.setStrings(TezConfiguration.TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS,
+        WeightedScalingMemoryDistributor.generateWeightStrings(0, 0, 1, 2, 3, 1, 1));
+    System.err.println(Joiner.on(",").join(conf.getStringCollection(
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS)));
+
+    MemoryDistributor dist = new MemoryDistributor(2, 2, conf);
+
+    dist.setJvmMemory(10000l);
+
+    // First request - ScatterGatherShuffleInput
+    MemoryUpdateCallbackForTest e1Callback = new MemoryUpdateCallbackForTest();
+    InputContext e1InputContext1 = createTestInputContext();
+    InputDescriptor e1InDesc1 = createTestInputDescriptor(OrderedGroupedKVInput.class);
+    dist.requestMemory(10000, e1Callback, e1InputContext1, e1InDesc1);
+
+    // Second request - BroadcastInput
+    MemoryUpdateCallbackForTest e2Callback = new MemoryUpdateCallbackForTest();
+    InputContext e2InputContext2 = createTestInputContext();
+    InputDescriptor e2InDesc2 = createTestInputDescriptor(UnorderedKVInput.class);
+    dist.requestMemory(10000, e2Callback, e2InputContext2, e2InDesc2);
+
+    // Third request - randomOutput (simulates MROutput)
+    MemoryUpdateCallbackForTest e3Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e3OutputContext1 = createTestOutputContext();
+    OutputDescriptor e3OutDesc1 = createTestOutputDescriptor();
+    dist.requestMemory(10000, e3Callback, e3OutputContext1, e3OutDesc1);
+
+    // Fourth request - OnFileSortedOutput
+    MemoryUpdateCallbackForTest e4Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e4OutputContext2 = createTestOutputContext();
+    OutputDescriptor e4OutDesc2 = createTestOutputDescriptor(OrderedPartitionedKVOutput.class);
+    dist.requestMemory(10000, e4Callback, e4OutputContext2, e4OutDesc2);
+
+    // Fifth request - Processor
+    MemoryUpdateCallbackForTest e5Callback = new MemoryUpdateCallbackForTest();
+    ProcessorContext e5ProcContext = createTestProcessortContext();
+    ProcessorDescriptor e5ProcDesc = createTestProcessorDescriptor();
+    dist.requestMemory(10000, e5Callback, e5ProcContext, e5ProcDesc);
+
+    dist.makeInitialAllocations();
+
+    // Total available: 80% of 10K = 8000
+    // 5 requests (weight) - 10K (3), 10K(1), 10K(1), 10K(2), 10K(1)
+    // Overlap input and output memory
+    assertEquals(5250, e1Callback.assigned);
+    assertEquals(1750, e2Callback.assigned);
+    assertEquals(2333, e3Callback.assigned);
+    assertEquals(4666, e4Callback.assigned);
+    assertEquals(1000, e5Callback.assigned);
+  }
+
+  @Test(timeout = 5000)
+  public void testAdditionalReserveFractionWeightedScalingNonConcurrent() throws TezException {
+    Configuration conf = new Configuration(this.conf);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT, false);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED, true);
+    conf.setStrings(TezConfiguration.TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS,
+        WeightedScalingMemoryDistributor.generateWeightStrings(0, 0, 2, 3, 6, 1, 1));
+    conf.setDouble(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ADDITIONAL_RESERVATION_FRACTION_PER_IO, 0.025d);
+    conf.setDouble(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ADDITIONAL_RESERVATION_FRACTION_MAX, 0.2d);
+
+    MemoryDistributor dist = new MemoryDistributor(2, 2, conf);
+
+    dist.setJvmMemory(10000l);
+
+    // First request - ScatterGatherShuffleInput [weight 6]
+    MemoryUpdateCallbackForTest e1Callback = new MemoryUpdateCallbackForTest();
+    InputContext e1InputContext1 = createTestInputContext();
+    InputDescriptor e1InDesc1 = createTestInputDescriptor(OrderedGroupedKVInput.class);
+    dist.requestMemory(10000, e1Callback, e1InputContext1, e1InDesc1);
+
+    // Second request - BroadcastInput [weight 2]
+    MemoryUpdateCallbackForTest e2Callback = new MemoryUpdateCallbackForTest();
+    InputContext e2InputContext2 = createTestInputContext();
+    InputDescriptor e2InDesc2 = createTestInputDescriptor(UnorderedKVInput.class);
+    dist.requestMemory(10000, e2Callback, e2InputContext2, e2InDesc2);
+
+    // Third request - randomOutput (simulates MROutput) [weight 1]
+    MemoryUpdateCallbackForTest e3Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e3OutputContext1 = createTestOutputContext();
+    OutputDescriptor e3OutDesc1 = createTestOutputDescriptor();
+    dist.requestMemory(10000, e3Callback, e3OutputContext1, e3OutDesc1);
+
+    // Fourth request - OnFileSortedOutput [weight 3]
+    MemoryUpdateCallbackForTest e4Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e4OutputContext2 = createTestOutputContext();
+    OutputDescriptor e4OutDesc2 = createTestOutputDescriptor(OrderedPartitionedKVOutput.class);
+    dist.requestMemory(10000, e4Callback, e4OutputContext2, e4OutDesc2);
+
+    dist.makeInitialAllocations();
+
+    // Total available: 60% of 10K = 6000
+    // 4 requests (weight) - 10K (6), 10K(2), 10K(1), 10K(3)
+    // Overlap input and output memory
+    assertEquals(4500, e1Callback.assigned);
+    assertEquals(1500, e2Callback.assigned);
+    assertEquals(1500, e3Callback.assigned);
+    assertEquals(4500, e4Callback.assigned);
+  }
+
+  @Test(timeout = 5000)
+  public void testWeightedScalingNonConcurrentInputsDisabled() throws TezException {
+    Configuration conf = new Configuration(this.conf);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_INPUT_OUTPUT_CONCURRENT, false);
+    conf.setBoolean(TezConfiguration.TEZ_TASK_SCALE_MEMORY_NON_CONCURRENT_INPUTS_ENABLED, false);
+    conf.setDouble(TezConfiguration.TEZ_TASK_SCALE_MEMORY_RESERVE_FRACTION, 0.2);
+    conf.setStrings(TezConfiguration.TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS,
+        WeightedScalingMemoryDistributor.generateWeightStrings(0, 0, 1, 2, 3, 1, 1));
+    System.err.println(Joiner.on(",").join(conf.getStringCollection(
+        TezConfiguration.TEZ_TASK_SCALE_MEMORY_WEIGHTED_RATIOS)));
+
+    MemoryDistributor dist = new MemoryDistributor(2, 2, conf);
+
+    dist.setJvmMemory(10000l);
+
+    // First request - ScatterGatherShuffleInput
+    MemoryUpdateCallbackForTest e1Callback = new MemoryUpdateCallbackForTest();
+    InputContext e1InputContext1 = createTestInputContext();
+    InputDescriptor e1InDesc1 = createTestInputDescriptor(OrderedGroupedKVInput.class);
+    dist.requestMemory(10000, e1Callback, e1InputContext1, e1InDesc1);
+
+    // Second request - BroadcastInput
+    MemoryUpdateCallbackForTest e2Callback = new MemoryUpdateCallbackForTest();
+    InputContext e2InputContext2 = createTestInputContext();
+    InputDescriptor e2InDesc2 = createTestInputDescriptor(UnorderedKVInput.class);
+    dist.requestMemory(10000, e2Callback, e2InputContext2, e2InDesc2);
+
+    // Third request - randomOutput (simulates MROutput)
+    MemoryUpdateCallbackForTest e3Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e3OutputContext1 = createTestOutputContext();
+    OutputDescriptor e3OutDesc1 = createTestOutputDescriptor();
+    dist.requestMemory(10000, e3Callback, e3OutputContext1, e3OutDesc1);
+
+    // Fourth request - OnFileSortedOutput
+    MemoryUpdateCallbackForTest e4Callback = new MemoryUpdateCallbackForTest();
+    OutputContext e4OutputContext2 = createTestOutputContext();
+    OutputDescriptor e4OutDesc2 = createTestOutputDescriptor(OrderedPartitionedKVOutput.class);
+    dist.requestMemory(10000, e4Callback, e4OutputContext2, e4OutDesc2);
+
+    // Fifth request - Processor
+    MemoryUpdateCallbackForTest e5Callback = new MemoryUpdateCallbackForTest();
+    ProcessorContext e5ProcContext = createTestProcessortContext();
+    ProcessorDescriptor e5ProcDesc = createTestProcessorDescriptor();
+    dist.requestMemory(10000, e5Callback, e5ProcContext, e5ProcDesc);
+
+    dist.makeInitialAllocations();
+
+    // Total available: 80% of 10K = 8000
+    // 5 requests (weight) - 10K (3), 10K(1), 10K(1), 10K(2), 10K(1)
+    // Overlap input and output memory
+    assertEquals(3000, e1Callback.assigned);
+    assertEquals(1000, e2Callback.assigned);
+    assertEquals(2333, e3Callback.assigned);
+    assertEquals(4666, e4Callback.assigned);
+    assertEquals(1000, e5Callback.assigned);
+  }
+
   private static class MemoryUpdateCallbackForTest extends MemoryUpdateCallback {
 
     long assigned = -1000;


[28/50] [abbrv] tez git commit: TEZ-3648. IFile.Write#close has an extra output stream flush

Posted by zh...@apache.org.
TEZ-3648. IFile.Write#close has an extra output stream flush


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

Branch: refs/heads/TEZ-1190
Commit: a5ffdea62cd6afc4f4a4e54dc20ff93df10b641f
Parents: 518deb6
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Mon Mar 6 09:42:30 2017 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Mon Mar 6 09:42:42 2017 -0600

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/tez/runtime/library/common/sort/impl/IFile.java  | 5 +----
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a5ffdea6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f90a29e..4c28405 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3648. IFile.Write#close has an extra output stream flush
   TEZ-3649. AsyncHttpConnection should add StopWatch start.
   TEZ-3647. Add a setting which lets Tez determine Xmx.
   TEZ-3644. Cleanup container list stored in AMNode.

http://git-wip-us.apache.org/repos/asf/tez/blob/a5ffdea6/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
index bcf6ca6..8e3661b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
@@ -214,9 +214,6 @@ public class IFile {
       //account for header bytes
       decompressedBytesWritten += HEADER.length;
 
-      //Flush the stream
-      out.flush();
-
       // Close the underlying stream iff we own it...
       if (ownOutputStream) {
         out.close();
@@ -226,7 +223,7 @@ public class IFile {
           compressedOut.finish();
           compressedOut.resetState();
         }
-        // Write the checksum
+        // Write the checksum and flush the buffer
         checksumOut.finish();
       }
       //header bytes are already included in rawOut


[34/50] [abbrv] tez git commit: TEZ-3650. Improve performance of FetchStatsLogger#logIndividualFetchComplete (jeagles)

Posted by zh...@apache.org.
TEZ-3650. Improve performance of FetchStatsLogger#logIndividualFetchComplete (jeagles)


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

Branch: refs/heads/TEZ-1190
Commit: a9af6cfccb8069afb01cbac3d373ffa0e2ecba93
Parents: c118104
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Sun Mar 12 22:56:02 2017 -0500
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Sun Mar 12 22:56:02 2017 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/tez/util/FastNumberFormat.java   | 55 +++++++++++
 .../org/apache/tez/util/TestNumberFormat.java   | 39 ++++++++
 .../runtime/library/common/shuffle/Fetcher.java |  4 +-
 .../library/common/shuffle/ShuffleUtils.java    | 85 +++++++++++------
 .../orderedgrouped/FetcherOrderedGrouped.java   |  4 +-
 .../common/shuffle/TestShuffleUtils.java        |  6 +-
 .../orderedgrouped/TestMergeManager.java        | 98 ++++++++++----------
 8 files changed, 209 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 48ccb54..54b17b1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3650. Improve performance of FetchStatsLogger#logIndividualFetchComplete
   TEZ-3655. Specify netty version instead of inheriting from hadoop dependency.
   TEZ-3253. Remove special handling for last app attempt.
   TEZ-3648. IFile.Write#close has an extra output stream flush

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-common/src/main/java/org/apache/tez/util/FastNumberFormat.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/util/FastNumberFormat.java b/tez-common/src/main/java/org/apache/tez/util/FastNumberFormat.java
new file mode 100644
index 0000000..f22fc64
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/util/FastNumberFormat.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.util;
+
+public class FastNumberFormat {
+
+  public static final int MAX_COUNT = 19;
+  private final char[] digits = new char[MAX_COUNT];
+  private int minimumIntegerDigits;
+
+  public static FastNumberFormat getInstance() {
+    return new FastNumberFormat();
+  }
+
+  public void setMinimumIntegerDigits(int minimumIntegerDigits) {
+    this.minimumIntegerDigits = minimumIntegerDigits;
+  }
+
+  public StringBuilder format(long source, StringBuilder sb) {
+    int left = MAX_COUNT;
+    if (source < 0) {
+      sb.append('-');
+      source = - source;
+    }
+    while (source > 0) {
+      digits[--left] = (char)('0' + (source % 10));
+      source /= 10;
+    }
+    while (MAX_COUNT - left < minimumIntegerDigits) {
+      digits[--left] = '0';
+    }
+    sb.append(digits, left, MAX_COUNT - left);
+    return sb;
+  }
+
+  public String format(long source) {
+    return format(source, new StringBuilder()).toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-common/src/test/java/org/apache/tez/util/TestNumberFormat.java
----------------------------------------------------------------------
diff --git a/tez-common/src/test/java/org/apache/tez/util/TestNumberFormat.java b/tez-common/src/test/java/org/apache/tez/util/TestNumberFormat.java
new file mode 100644
index 0000000..c2f1185
--- /dev/null
+++ b/tez-common/src/test/java/org/apache/tez/util/TestNumberFormat.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.text.NumberFormat;
+
+public class TestNumberFormat {
+
+  @Test(timeout = 1000)
+  public void testLongWithPadding() throws Exception {
+    FastNumberFormat fastNumberFormat = FastNumberFormat.getInstance();
+    fastNumberFormat.setMinimumIntegerDigits(6);
+    NumberFormat numberFormat = NumberFormat.getInstance();
+    numberFormat.setGroupingUsed(false);
+    numberFormat.setMinimumIntegerDigits(6);
+    long[] testLongs = {1, 23, 456, 7890, 12345, 678901, 2345689, 0, -0, -1, -23, -456, -7890, -12345, -678901, -2345689};
+    for (long l: testLongs) {
+      Assert.assertEquals("Number formats should be equal", numberFormat.format(l), fastNumberFormat.format(l));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
index 6cbff94..9d1f42a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
@@ -814,11 +814,11 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
         ShuffleUtils.shuffleToMemory(((MemoryFetchedInput) fetchedInput).getBytes(),
           input, (int) decompressedLength, (int) compressedLength, codec,
           ifileReadAhead, ifileReadAheadLength, LOG,
-          fetchedInput.getInputAttemptIdentifier().toString());
+          fetchedInput.getInputAttemptIdentifier());
       } else if (fetchedInput.getType() == Type.DISK) {
         ShuffleUtils.shuffleToDisk(((DiskFetchedInput) fetchedInput).getOutputStream(),
           (host +":" +port), input, compressedLength, decompressedLength, LOG,
-          fetchedInput.getInputAttemptIdentifier().toString(),
+          fetchedInput.getInputAttemptIdentifier(),
           ifileReadAhead, ifileReadAheadLength, verifyDiskChecksum);
       } else {
         throw new TezUncheckedException("Bad fetchedInput type while fetching shuffle data " +

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index 82e844d..caddbc8 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -48,6 +48,7 @@ import org.apache.tez.http.SSLFactory;
 import org.apache.tez.http.async.netty.AsyncHttpConnection;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB;
+import org.apache.tez.util.FastNumberFormat;
 import org.roaringbitmap.RoaringBitmap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -88,6 +89,15 @@ public class ShuffleUtils {
           return new DecimalFormat("0.00");
         }
       };
+  static final ThreadLocal<FastNumberFormat> MBPS_FAST_FORMAT =
+      new ThreadLocal<FastNumberFormat>() {
+        @Override
+        protected FastNumberFormat initialValue() {
+          FastNumberFormat fmt = FastNumberFormat.getInstance();
+          fmt.setMinimumIntegerDigits(2);
+          return fmt;
+        }
+      };
 
   public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta)
       throws IOException {
@@ -119,7 +129,7 @@ public class ShuffleUtils {
   public static void shuffleToMemory(byte[] shuffleData,
       InputStream input, int decompressedLength, int compressedLength,
       CompressionCodec codec, boolean ifileReadAhead, int ifileReadAheadLength,
-      Logger LOG, String identifier) throws IOException {
+      Logger LOG, InputAttemptIdentifier identifier) throws IOException {
     try {
       IFile.Reader.readToMemory(shuffleData, input, compressedLength, codec,
           ifileReadAhead, ifileReadAheadLength);
@@ -145,7 +155,7 @@ public class ShuffleUtils {
   }
   
   public static void shuffleToDisk(OutputStream output, String hostIdentifier,
-      InputStream input, long compressedLength, long decompressedLength, Logger LOG, String identifier,
+      InputStream input, long compressedLength, long decompressedLength, Logger LOG, InputAttemptIdentifier identifier,
       boolean ifileReadAhead, int ifileReadAheadLength, boolean verifyChecksum) throws IOException {
     // Copy data to local-disk
     long bytesLeft = compressedLength;
@@ -530,6 +540,20 @@ public class ShuffleUtils {
       this.aggregateLogger = aggregateLogger;
     }
 
+
+    private static StringBuilder toShortString(InputAttemptIdentifier inputAttemptIdentifier, StringBuilder sb) {
+      sb.append("{");
+      sb.append(inputAttemptIdentifier.getInputIdentifier());
+      sb.append(", ").append(inputAttemptIdentifier.getAttemptNumber());
+      sb.append(", ").append(inputAttemptIdentifier.getPathComponent());
+      if (inputAttemptIdentifier.getFetchTypeInfo()
+          != InputAttemptIdentifier.SPILL_INFO.FINAL_MERGE_ENABLED) {
+        sb.append(", ").append(inputAttemptIdentifier.getFetchTypeInfo().ordinal());
+        sb.append(", ").append(inputAttemptIdentifier.getSpillEventId());
+      }
+      sb.append("}");
+      return sb;
+    }
     /**
      * Log individual fetch complete event.
      * This log information would be used by tez-tool/perf-analzyer/shuffle tools for mining
@@ -545,19 +569,37 @@ public class ShuffleUtils {
      */
     public void logIndividualFetchComplete(long millis, long bytesCompressed,
         long bytesDecompressed, String outputType, InputAttemptIdentifier srcAttemptIdentifier) {
-      double rate = 0;
-      if (millis != 0) {
-        rate = bytesCompressed / ((double) millis / 1000);
-        rate = rate / (1024 * 1024);
-      }
+
       if (activeLogger.isInfoEnabled()) {
-        activeLogger.info(
-            "Completed fetch for attempt: "
-                + toShortString(srcAttemptIdentifier)
-                +" to " + outputType +
-                ", csize=" + bytesCompressed + ", dsize=" + bytesDecompressed +
-                ", EndTime=" + System.currentTimeMillis() + ", TimeTaken=" + millis + ", Rate=" +
-                MBPS_FORMAT.get().format(rate) + " MB/s");
+        long wholeMBs = 0;
+        long partialMBs = 0;
+        if (millis != 0) {
+          // fast math is done using integer math to avoid double to string conversion
+          // calculate B/s * 100 to preserve MBs precision to two decimal places
+          // multiply numerator by 100000 (2^5 * 5^5) and divide denominator by MB (2^20)
+          // simply fraction to protect ourselves from overflow by factoring out 2^5
+          wholeMBs = (bytesCompressed * 3125) / (millis * 32768);
+          partialMBs = wholeMBs % 100;
+          wholeMBs /= 100;
+        }
+        StringBuilder sb = new StringBuilder("Completed fetch for attempt: ");
+        toShortString(srcAttemptIdentifier, sb);
+        sb.append(" to ");
+        sb.append(outputType);
+        sb.append(", csize=");
+        sb.append(bytesCompressed);
+        sb.append(", dsize=");
+        sb.append(bytesDecompressed);
+        sb.append(", EndTime=");
+        sb.append(System.currentTimeMillis());
+        sb.append(", TimeTaken=");
+        sb.append(millis);
+        sb.append(", Rate=");
+        sb.append(wholeMBs);
+        sb.append(".");
+        MBPS_FAST_FORMAT.get().format(partialMBs, sb);
+        sb.append(" MB/s");
+        activeLogger.info(sb.toString());
       } else {
         long currentCount, currentCompressedSize, currentDecompressedSize, currentTotalTime;
         synchronized (this) {
@@ -583,21 +625,6 @@ public class ShuffleUtils {
     }
   }
 
-  private static String toShortString(InputAttemptIdentifier inputAttemptIdentifier) {
-    StringBuilder sb = new StringBuilder();
-    sb.append("{");
-    sb.append(inputAttemptIdentifier.getInputIdentifier());
-    sb.append(", ").append(inputAttemptIdentifier.getAttemptNumber());
-    sb.append(", ").append(inputAttemptIdentifier.getPathComponent());
-    if (inputAttemptIdentifier.getFetchTypeInfo()
-        != InputAttemptIdentifier.SPILL_INFO.FINAL_MERGE_ENABLED) {
-      sb.append(", ").append(inputAttemptIdentifier.getFetchTypeInfo().ordinal());
-      sb.append(", ").append(inputAttemptIdentifier.getSpillEventId());
-    }
-    sb.append("}");
-    return sb.toString();
-  }
-
   /**
    * Build {@link org.apache.tez.http.HttpConnectionParams} from configuration
    *

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
index bcb75d2..58ca1e2 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
@@ -504,11 +504,11 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
       if (mapOutput.getType() == Type.MEMORY) {
         ShuffleUtils.shuffleToMemory(mapOutput.getMemory(), input,
           (int) decompressedLength, (int) compressedLength, codec, ifileReadAhead,
-          ifileReadAheadLength, LOG, mapOutput.getAttemptIdentifier().toString());
+          ifileReadAheadLength, LOG, mapOutput.getAttemptIdentifier());
       } else if (mapOutput.getType() == Type.DISK) {
         ShuffleUtils.shuffleToDisk(mapOutput.getDisk(), host.getHostIdentifier(),
           input, compressedLength, decompressedLength, LOG,
-          mapOutput.getAttemptIdentifier().toString(),
+          mapOutput.getAttemptIdentifier(),
           ifileReadAhead, ifileReadAheadLength, verifyDiskChecksum);
       } else {
         throw new IOException("Unknown mapOutput type while fetching shuffle data:" +

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
index f21da7c..b1ce716 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
@@ -284,7 +284,7 @@ public class TestShuffleUtils {
     byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1};
     try {
       ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header),
-          1024, 128, mockCodec, false, 0, mock(Logger.class), "identifier");
+          1024, 128, mockCodec, false, 0, mock(Logger.class), null);
       Assert.fail("shuffle was supposed to throw!");
     } catch (IOException e) {
       Assert.assertTrue(e.getCause() instanceof InternalError);
@@ -301,14 +301,14 @@ public class TestShuffleUtils {
     ByteArrayInputStream in = new ByteArrayInputStream(bogusData);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     ShuffleUtils.shuffleToDisk(baos, "somehost", in,
-        bogusData.length, 2000, mock(Logger.class), "identifier", false, 0, false);
+        bogusData.length, 2000, mock(Logger.class), null, false, 0, false);
     Assert.assertArrayEquals(bogusData, baos.toByteArray());
 
     // verify sending same stream of zeroes with validation generates an exception
     in.reset();
     try {
       ShuffleUtils.shuffleToDisk(mock(OutputStream.class), "somehost", in,
-          bogusData.length, 2000, mock(Logger.class), "identifier", false, 0, true);
+          bogusData.length, 2000, mock(Logger.class), null, false, 0, true);
       Assert.fail("shuffle was supposed to throw!");
     } catch (IOException e) {
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/a9af6cfc/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
index 9209ff4..a812728 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java
@@ -235,8 +235,8 @@ public class TestMergeManager {
     assertEquals(0, mergeManager.getUsedMemory());
     assertEquals(0, mergeManager.getCommitMemory());
 
-    byte[] data1 = generateData(conf, 10);
-    byte[] data2 = generateData(conf, 20);
+    byte[] data1 = generateData(conf, 10, null);
+    byte[] data2 = generateData(conf, 20, null);
     MapOutput firstMapOutput = mergeManager.reserve(null, data1.length, data1.length, 0);
     MapOutput secondMapOutput = mergeManager.reserve(null, data2.length, data2.length, 0);
     assertEquals(MapOutput.Type.MEMORY, firstMapOutput.getType());
@@ -294,15 +294,19 @@ public class TestMergeManager {
      * - After 3 segment commits, it would trigger mem-to-mem merge.
      * - All of them can be merged in memory.
      */
-    byte[] data1 = generateDataBySize(conf, 10);
-    byte[] data2 = generateDataBySize(conf, 20);
-    byte[] data3 = generateDataBySize(conf, 200);
-    byte[] data4 = generateDataBySize(conf, 20000);
-
-    MapOutput mo1 = mergeManager.reserve(new InputAttemptIdentifier(0,0), data1.length, data1.length, 0);
-    MapOutput mo2 = mergeManager.reserve(new InputAttemptIdentifier(1,0), data2.length, data2.length, 0);
-    MapOutput mo3 = mergeManager.reserve(new InputAttemptIdentifier(2,0), data3.length, data3.length, 0);
-    MapOutput mo4 = mergeManager.reserve(new InputAttemptIdentifier(3,0), data4.length, data4.length, 0);
+    InputAttemptIdentifier inputAttemptIdentifier1 = new InputAttemptIdentifier(0,0);
+    InputAttemptIdentifier inputAttemptIdentifier2 = new InputAttemptIdentifier(1,0);
+    InputAttemptIdentifier inputAttemptIdentifier3 = new InputAttemptIdentifier(2,0);
+    InputAttemptIdentifier inputAttemptIdentifier4 = new InputAttemptIdentifier(3,0);
+    byte[] data1 = generateDataBySize(conf, 10, inputAttemptIdentifier1);
+    byte[] data2 = generateDataBySize(conf, 20, inputAttemptIdentifier2);
+    byte[] data3 = generateDataBySize(conf, 200, inputAttemptIdentifier3);
+    byte[] data4 = generateDataBySize(conf, 20000, inputAttemptIdentifier4);
+
+    MapOutput mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0);
+    MapOutput mo2 = mergeManager.reserve(inputAttemptIdentifier1, data2.length, data2.length, 0);
+    MapOutput mo3 = mergeManager.reserve(inputAttemptIdentifier1, data3.length, data3.length, 0);
+    MapOutput mo4 = mergeManager.reserve(inputAttemptIdentifier1, data4.length, data4.length, 0);
 
     assertEquals(MapOutput.Type.MEMORY, mo1.getType());
     assertEquals(MapOutput.Type.MEMORY, mo2.getType());
@@ -351,15 +355,15 @@ public class TestMergeManager {
     mergeManager.configureAndStart();
 
     //Single shuffle limit is 25% of 2000000
-    data1 = generateDataBySize(conf, 10);
-    data2 = generateDataBySize(conf, 400000);
-    data3 = generateDataBySize(conf, 400000);
-    data4 = generateDataBySize(conf, 400000);
+    data1 = generateDataBySize(conf, 10, inputAttemptIdentifier1);
+    data2 = generateDataBySize(conf, 400000, inputAttemptIdentifier2);
+    data3 = generateDataBySize(conf, 400000, inputAttemptIdentifier3);
+    data4 = generateDataBySize(conf, 400000, inputAttemptIdentifier4);
 
-    mo1 = mergeManager.reserve(new InputAttemptIdentifier(0,0), data1.length, data1.length, 0);
-    mo2 = mergeManager.reserve(new InputAttemptIdentifier(1,0), data2.length, data2.length, 0);
-    mo3 = mergeManager.reserve(new InputAttemptIdentifier(2,0), data3.length, data3.length, 0);
-    mo4 = mergeManager.reserve(new InputAttemptIdentifier(3,0), data4.length, data4.length, 0);
+    mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0);
+    mo2 = mergeManager.reserve(inputAttemptIdentifier2, data2.length, data2.length, 0);
+    mo3 = mergeManager.reserve(inputAttemptIdentifier3, data3.length, data3.length, 0);
+    mo4 = mergeManager.reserve(inputAttemptIdentifier4, data4.length, data4.length, 0);
 
     assertEquals(MapOutput.Type.MEMORY, mo1.getType());
     assertEquals(MapOutput.Type.MEMORY, mo2.getType());
@@ -409,15 +413,15 @@ public class TestMergeManager {
     mergeManager.configureAndStart();
 
     //Single shuffle limit is 25% of 2000000
-    data1 = generateDataBySize(conf, 400000);
-    data2 = generateDataBySize(conf, 400000);
-    data3 = generateDataBySize(conf, 400000);
-    data4 = generateDataBySize(conf, 400000);
+    data1 = generateDataBySize(conf, 400000, inputAttemptIdentifier1);
+    data2 = generateDataBySize(conf, 400000, inputAttemptIdentifier2);
+    data3 = generateDataBySize(conf, 400000, inputAttemptIdentifier3);
+    data4 = generateDataBySize(conf, 400000, inputAttemptIdentifier4);
 
-    mo1 = mergeManager.reserve(new InputAttemptIdentifier(0,0), data1.length, data1.length, 0);
-    mo2 = mergeManager.reserve(new InputAttemptIdentifier(1,0), data2.length, data2.length, 0);
-    mo3 = mergeManager.reserve(new InputAttemptIdentifier(2,0), data3.length, data3.length, 0);
-    mo4 = mergeManager.reserve(new InputAttemptIdentifier(3,0), data4.length, data4.length, 0);
+    mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0);
+    mo2 = mergeManager.reserve(inputAttemptIdentifier2, data2.length, data2.length, 0);
+    mo3 = mergeManager.reserve(inputAttemptIdentifier3, data3.length, data3.length, 0);
+    mo4 = mergeManager.reserve(inputAttemptIdentifier4, data4.length, data4.length, 0);
 
     assertEquals(MapOutput.Type.MEMORY, mo1.getType());
     assertEquals(MapOutput.Type.MEMORY, mo2.getType());
@@ -465,15 +469,15 @@ public class TestMergeManager {
     mergeManager.configureAndStart();
 
     //Single shuffle limit is 25% of 2000000
-    data1 = generateDataBySize(conf, 490000);
-    data2 = generateDataBySize(conf, 490000);
-    data3 = generateDataBySize(conf, 490000);
-    data4 = generateDataBySize(conf, 230000);
+    data1 = generateDataBySize(conf, 490000, inputAttemptIdentifier1);
+    data2 = generateDataBySize(conf, 490000, inputAttemptIdentifier2);
+    data3 = generateDataBySize(conf, 490000, inputAttemptIdentifier3);
+    data4 = generateDataBySize(conf, 230000, inputAttemptIdentifier4);
 
-    mo1 = mergeManager.reserve(new InputAttemptIdentifier(0,0), data1.length, data1.length, 0);
-    mo2 = mergeManager.reserve(new InputAttemptIdentifier(1,0), data2.length, data2.length, 0);
-    mo3 = mergeManager.reserve(new InputAttemptIdentifier(2,0), data3.length, data3.length, 0);
-    mo4 = mergeManager.reserve(new InputAttemptIdentifier(3,0), data4.length, data4.length, 0);
+    mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0);
+    mo2 = mergeManager.reserve(inputAttemptIdentifier2, data2.length, data2.length, 0);
+    mo3 = mergeManager.reserve(inputAttemptIdentifier3, data3.length, data3.length, 0);
+    mo4 = mergeManager.reserve(inputAttemptIdentifier4, data4.length, data4.length, 0);
 
     assertTrue(mergeManager.getUsedMemory() >= (490000 + 490000 + 490000 + 23000));
 
@@ -520,15 +524,15 @@ public class TestMergeManager {
     mergeManager.configureAndStart();
 
     //Single shuffle limit is 25% of 2000000
-    data1 = generateDataBySize(conf, 490000);
-    data2 = generateDataBySize(conf, 490000);
-    data3 = generateDataBySize(conf, 490000);
-    data4 = generateDataBySize(conf, 230000);
+    data1 = generateDataBySize(conf, 490000, inputAttemptIdentifier1);
+    data2 = generateDataBySize(conf, 490000, inputAttemptIdentifier2);
+    data3 = generateDataBySize(conf, 490000, inputAttemptIdentifier3);
+    data4 = generateDataBySize(conf, 230000, inputAttemptIdentifier4);
 
-    mo1 = mergeManager.reserve(new InputAttemptIdentifier(0,0), data1.length, data1.length, 0);
-    mo2 = mergeManager.reserve(new InputAttemptIdentifier(1,0), data2.length, data2.length, 0);
-    mo3 = mergeManager.reserve(new InputAttemptIdentifier(2,0), data3.length, data3.length, 0);
-    mo4 = mergeManager.reserve(new InputAttemptIdentifier(3,0), data4.length, data4.length, 0);
+    mo1 = mergeManager.reserve(inputAttemptIdentifier1, data1.length, data1.length, 0);
+    mo2 = mergeManager.reserve(inputAttemptIdentifier2, data2.length, data2.length, 0);
+    mo3 = mergeManager.reserve(inputAttemptIdentifier3, data3.length, data3.length, 0);
+    mo4 = mergeManager.reserve(inputAttemptIdentifier4, data4.length, data4.length, 0);
 
     assertTrue(mergeManager.getUsedMemory() >= (490000 + 490000 + 490000 + 23000));
 
@@ -566,7 +570,7 @@ public class TestMergeManager {
     Assert.assertFalse(mergeManager.isMergeComplete());
   }
 
-  private byte[] generateDataBySize(Configuration conf, int rawLen) throws IOException {
+  private byte[] generateDataBySize(Configuration conf, int rawLen, InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     FSDataOutputStream fsdos = new FSDataOutputStream(baos, null);
     IFile.Writer writer =
@@ -584,11 +588,11 @@ public class TestMergeManager {
     int rawLength = (int)writer.getRawLength();
     byte[] data = new byte[rawLength];
     ShuffleUtils.shuffleToMemory(data, new ByteArrayInputStream(baos.toByteArray()),
-        rawLength, compressedLength, null, false, 0, LOG, "sometask");
+        rawLength, compressedLength, null, false, 0, LOG, inputAttemptIdentifier);
     return data;
   }
 
-  private byte[] generateData(Configuration conf, int numEntries) throws IOException {
+  private byte[] generateData(Configuration conf, int numEntries, InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     FSDataOutputStream fsdos = new FSDataOutputStream(baos, null);
     IFile.Writer writer =
@@ -601,7 +605,7 @@ public class TestMergeManager {
     int rawLength = (int)writer.getRawLength();
     byte[] data = new byte[rawLength];
     ShuffleUtils.shuffleToMemory(data, new ByteArrayInputStream(baos.toByteArray()),
-        rawLength, compressedLength, null, false, 0, LOG, "sometask");
+        rawLength, compressedLength, null, false, 0, LOG, inputAttemptIdentifier);
     return data;
   }
 


[05/50] [abbrv] tez git commit: TEZ-3609. Improve ATSv15 performance for DAG entities read calls. Contributed by Harish Jaiprakash.

Posted by zh...@apache.org.
TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
Contributed by Harish Jaiprakash.


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

Branch: refs/heads/TEZ-1190
Commit: 11815a7ccd80ac3cb052f34a420d44de9f083542
Parents: 63ae97d
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 7 22:54:08 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 7 22:54:08 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../logging/ats/TimelineCachePluginImpl.java    |  1 -
 .../ats/TestTimelineCachePluginImpl.java        | 41 ++++++++++----------
 3 files changed, 23 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/11815a7c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a7cc0ce..0949339 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3581. Add different logger to enable suppressing logs for specific lines.
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
@@ -198,6 +199,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3609. Improve ATSv15 performance for DAG entities read calls.
   TEZ-3244. Allow overlap of input and output memory when they are not concurrent
   TEZ-3601. Add another HistoryLogLevel to suppress TaskAttempts at specific levels
   TEZ-3462. Task attempt failure during container shutdown loses useful container diagnostics

http://git-wip-us.apache.org/repos/asf/tez/blob/11815a7c/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java
index b4217a1..8269714 100644
--- a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java
+++ b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java
@@ -72,7 +72,6 @@ public class TimelineCachePluginImpl extends TimelineEntityGroupPlugin implement
   private Set<TimelineEntityGroupId> createTimelineEntityGroupIds(TezDAGID dagId) {
     ApplicationId appId = dagId.getApplicationId();
     HashSet<TimelineEntityGroupId> groupIds = Sets.newHashSet(
-        TimelineEntityGroupId.newInstance(appId, appId.toString()),
         TimelineEntityGroupId.newInstance(appId, dagId.toString()));
     for (int numGroupsPerDag : allNumGroupsPerDag) {
       groupIds.add(TimelineEntityGroupId.newInstance(appId, dagId.getGroupId(numGroupsPerDag)));

http://git-wip-us.apache.org/repos/asf/tez/blob/11815a7c/tez-plugins/tez-yarn-timeline-cache-plugin/src/test/java/org/apache/tez/dag/history/logging/ats/TestTimelineCachePluginImpl.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/src/test/java/org/apache/tez/dag/history/logging/ats/TestTimelineCachePluginImpl.java b/tez-plugins/tez-yarn-timeline-cache-plugin/src/test/java/org/apache/tez/dag/history/logging/ats/TestTimelineCachePluginImpl.java
index 6f819ba..3d1af63 100644
--- a/tez-plugins/tez-yarn-timeline-cache-plugin/src/test/java/org/apache/tez/dag/history/logging/ats/TestTimelineCachePluginImpl.java
+++ b/tez-plugins/tez-yarn-timeline-cache-plugin/src/test/java/org/apache/tez/dag/history/logging/ats/TestTimelineCachePluginImpl.java
@@ -124,12 +124,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(3, groupIds.size());
+      Assert.assertEquals(2, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId1, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID1, appId1, 100).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID1, 100).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -143,12 +143,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(2, groupIds.size());
+      Assert.assertEquals(1, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId1, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID1, appId1).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID1).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -162,12 +162,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(2, groupIds.size());
+      Assert.assertEquals(1, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId1, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID1, appId1).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID1).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -181,12 +181,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(3, groupIds.size());
+      Assert.assertEquals(2, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId1, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID1, appId1, 100).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID1, 100).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -200,12 +200,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(4, groupIds.size());
+      Assert.assertEquals(3, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId2, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID2, appId2, 100, 50).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID2, 100, 50).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -219,12 +219,13 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(5, groupIds.size());
+      Assert.assertEquals(4, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId2, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID2, appId2, 100, 25, 50).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(
+            getGroupIds(dagID2, 100, 25, 50).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -238,12 +239,12 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(3, groupIds.size());
+      Assert.assertEquals(2, groupIds.size());
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         Assert.assertEquals(appId2, groupId.getApplicationId());
-        Assert.assertTrue(getGroupIds(dagID2, appId2, 100).contains(groupId.getTimelineEntityGroupId()));
+        Assert.assertTrue(getGroupIds(dagID2, 100).contains(groupId.getTimelineEntityGroupId()));
       }
     }
   }
@@ -261,21 +262,21 @@ public class TestTimelineCachePluginImpl {
         Assert.assertNull(groupIds);
         continue;
       }
-      Assert.assertEquals(6, groupIds.size());
+      Assert.assertEquals(4, groupIds.size());
       int found = 0;
       Iterator<TimelineEntityGroupId> iter = groupIds.iterator();
       while (iter.hasNext()) {
         TimelineEntityGroupId groupId = iter.next();
         if (groupId.getApplicationId().equals(appId1)) {
           String entityGroupId = groupId.getTimelineEntityGroupId();
-          if (getGroupIds(dagID1, appId1, 100).contains(entityGroupId)) {
+          if (getGroupIds(dagID1, 100).contains(entityGroupId)) {
             ++found;
           } else {
             Assert.fail("Unexpected group id: " + entityGroupId);
           }
         } else if (groupId.getApplicationId().equals(appId2)) {
           String entityGroupId = groupId.getTimelineEntityGroupId();
-          if (getGroupIds(dagID2, appId2, 100).contains(entityGroupId)) {
+          if (getGroupIds(dagID2, 100).contains(entityGroupId)) {
             ++found;
           } else {
             Assert.fail("Unexpected group id: " + entityGroupId);
@@ -284,7 +285,7 @@ public class TestTimelineCachePluginImpl {
           Assert.fail("Unexpected appId: " + groupId.getApplicationId());
         }
       }
-      Assert.assertEquals("All groupIds not returned", 6, found);
+      Assert.assertEquals("All groupIds not returned", 4, found);
     }
   }
 
@@ -374,8 +375,8 @@ public class TestTimelineCachePluginImpl {
     Assert.assertEquals("All groupIds not returned", 1, found);
   }
 
-  private Set<String> getGroupIds(TezDAGID dagId, ApplicationId appId, int ... allNumDagsPerGroup) {
-    HashSet<String> groupIds = Sets.newHashSet(dagId.toString(), appId.toString());
+  private Set<String> getGroupIds(TezDAGID dagId, int ... allNumDagsPerGroup) {
+    HashSet<String> groupIds = Sets.newHashSet(dagId.toString());
     for (int numDagsPerGroup : allNumDagsPerGroup) {
       groupIds.add(dagId.getGroupId(numDagsPerGroup));
     }


[46/50] [abbrv] tez git commit: TEZ-3668. Explicitly include hadoop-mapreduce-client-shuffle for test in root pom (zhiyuany)

Posted by zh...@apache.org.
TEZ-3668. Explicitly include hadoop-mapreduce-client-shuffle for test in root pom (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: 2bdf58aa92bdf00408383d2f0787bb1d26081954
Parents: e375b9d
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Mon Mar 27 18:49:34 2017 -0700
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Mon Mar 27 18:49:34 2017 -0700

----------------------------------------------------------------------
 pom.xml                                       | 6 ++++++
 tez-plugins/tez-yarn-timeline-history/pom.xml | 4 ++++
 tez-tests/pom.xml                             | 4 ++++
 3 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/2bdf58aa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e4d2dd3..2bbac69 100644
--- a/pom.xml
+++ b/pom.xml
@@ -580,6 +580,12 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
+        <scope>test</scope>
+        <version>${hadoop.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <scope>test</scope>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/tez/blob/2bdf58aa/tez-plugins/tez-yarn-timeline-history/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml
index 2c0ad4d..e37883c 100644
--- a/tez-plugins/tez-yarn-timeline-history/pom.xml
+++ b/tez-plugins/tez-yarn-timeline-history/pom.xml
@@ -103,6 +103,10 @@
       <type>test-jar</type>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.codehaus.jettison</groupId>
       <artifactId>jettison</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/tez/blob/2bdf58aa/tez-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml
index d314f6d..20fd747 100644
--- a/tez-tests/pom.xml
+++ b/tez-tests/pom.xml
@@ -104,6 +104,10 @@
       <type>test-jar</type>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-shuffle</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>


[33/50] [abbrv] tez git commit: TEZ-3655. Specify netty version instead of inheriting from hadoop dependency. (zhiyuany)

Posted by zh...@apache.org.
TEZ-3655. Specify netty version instead of inheriting from hadoop dependency. (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: c118104402b15ab8d5375708f9f471dd6f3b94ed
Parents: c6d4908
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Fri Mar 10 15:07:36 2017 -0800
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Fri Mar 10 15:15:30 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                   | 1 +
 tez-ext-service-tests/pom.xml | 5 +++++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/c1181044/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0065a83..48ccb54 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3655. Specify netty version instead of inheriting from hadoop dependency.
   TEZ-3253. Remove special handling for last app attempt.
   TEZ-3648. IFile.Write#close has an extra output stream flush
   TEZ-3649. AsyncHttpConnection should add StopWatch start.

http://git-wip-us.apache.org/repos/asf/tez/blob/c1181044/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index 70b8e00..fd5255e 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -27,6 +27,11 @@
 
   <dependencies>
     <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty</artifactId>
+      <version>3.6.2.Final</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
     </dependency>


[39/50] [abbrv] tez git commit: TEZ-3660. Remove CHANGES.txt. (sseth)

Posted by zh...@apache.org.
TEZ-3660. Remove CHANGES.txt. (sseth)


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

Branch: refs/heads/TEZ-1190
Commit: 5f953bfd9d6c4c4c77842c1de29e2868c8ae2e5e
Parents: 9081420
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Mar 13 20:57:11 2017 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Mar 13 20:57:11 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt | 2294 ------------------------------------------------------
 1 file changed, 2294 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] tez git commit: TEZ-3655. Add committer zhiyuany to the Tez Team List. (zhiyuany)

Posted by zh...@apache.org.
TEZ-3655. Add committer zhiyuany to the Tez Team List. (zhiyuany)


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

Branch: refs/heads/TEZ-1190
Commit: 98bf198aa0a5e852b2ea9a3588c4c2d089ea2152
Parents: 57c857d
Author: Zhiyuan Yang <zh...@apache.org>
Authored: Thu Mar 16 02:09:21 2017 -0700
Committer: Zhiyuan Yang <zh...@apache.org>
Committed: Thu Mar 16 02:09:21 2017 -0700

----------------------------------------------------------------------
 docs/pom.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/98bf198a/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index db45da3..97ae60f 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -383,6 +383,15 @@
         </roles>
       </developer>
       <developer>
+        <id>zhiyuany</id>
+        <name>Zhiyuan Yang</name>
+        <email>zhiyuany@apache.org</email>
+        <timezone>-8</timezone>
+        <roles>
+          <role>Committer</role>
+        </roles>
+      </developer>
+      <developer>
         <id>zjffdu</id>
         <name>Jeff Zhang</name>
         <email>zjffdu@apache.org</email>


[30/50] [abbrv] tez git commit: TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity (sree)

Posted by zh...@apache.org.
TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity (sree)


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

Branch: refs/heads/TEZ-1190
Commit: af0d4d7c81f407898057056b786b38d6679e90c7
Parents: e84231e
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Tue Mar 7 16:38:54 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Tue Mar 7 16:38:54 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 tez-ui/src/main/webapp/app/entities/entity.js   | 33 +++++++++----
 .../webapp/tests/unit/entities/entity-test.js   | 49 ++++++++++++++++++++
 3 files changed, 75 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/af0d4d7c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a3a74bc..1d2af07 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -214,6 +214,7 @@ ALL CHANGES:
   TEZ-3630. Tez UI: Use DAG status for controlling auto-refresh polling
   TEZ-3639. Tez UI: Footer pagination is improper in landing page
   TEZ-3640. Tez UI: Add associated llap application id to queries page
+  TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
 
 Release 0.8.5: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/af0d4d7c/tez-ui/src/main/webapp/app/entities/entity.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/entities/entity.js b/tez-ui/src/main/webapp/app/entities/entity.js
index 80a836e..d982811 100644
--- a/tez-ui/src/main/webapp/app/entities/entity.js
+++ b/tez-ui/src/main/webapp/app/entities/entity.js
@@ -32,6 +32,7 @@ var Entity = Ember.Object.extend(NameMixin, {
       params: query,
       urlParams: urlParams
     }).then(function (record) {
+      that.resetAllNeeds(loader, record, options);
       return that._loadAllNeeds(loader, record, options, urlParams);
     });
   },
@@ -44,6 +45,7 @@ var Entity = Ember.Object.extend(NameMixin, {
       urlParams: urlParams
     }).then(function (records) {
       return Ember.RSVP.all(records.map(function (record) {
+        that.resetAllNeeds(loader, record, options);
         return that._loadAllNeeds(loader, record, options, urlParams);
       })).then(function () {
        return records;
@@ -96,6 +98,14 @@ var Entity = Ember.Object.extend(NameMixin, {
     return Ember.Object.create(need, overrides);
   },
 
+  setNeed: function (parentModel, name, model) {
+    if(!parentModel.get("isDeleted")) {
+      parentModel.set(name, model);
+      parentModel.refreshLoadTime();
+    }
+    return parentModel;
+  },
+
   _loadNeed: function (loader, parentModel, needOptions, options, index) {
     var needLoader,
         that = this,
@@ -128,10 +138,7 @@ var Entity = Ember.Object.extend(NameMixin, {
     }
 
     needLoader = needLoader.then(function (model) {
-      if(!parentModel.get("isDeleted")) {
-        parentModel.set(needOptions.name, model);
-        parentModel.refreshLoadTime();
-      }
+      that.setNeed(parentModel, needOptions.name, model);
       return model;
     });
 
@@ -141,10 +148,7 @@ var Entity = Ember.Object.extend(NameMixin, {
       }
 
       if(needOptions.silent) {
-        if(!parentModel.get("isDeleted")) {
-          parentModel.set(needOptions.name, null);
-          parentModel.refreshLoadTime();
-        }
+        that.setNeed(parentModel, needOptions.name, null);
       }
       else {
         throw(err);
@@ -198,6 +202,19 @@ var Entity = Ember.Object.extend(NameMixin, {
     }
   },
 
+  resetAllNeeds: function (loader, parentModel/*, options*/) {
+    var needs = parentModel.get("needs"),
+        that = this;
+
+    if(needs) {
+      MoreObject.forEach(needs, function (name, needOptions) {
+        needOptions = that.normalizeNeed(name, needOptions, parentModel);
+        that.setNeed(parentModel, needOptions.name, null);
+      });
+    }
+
+    return parentModel;
+  },
 });
 
 export default Entity;

http://git-wip-us.apache.org/repos/asf/tez/blob/af0d4d7c/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js b/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
index a7677a7..1e353f0 100644
--- a/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/entities/entity-test.js
@@ -34,11 +34,14 @@ test('Basic creation test', function(assert) {
   assert.ok(entity.query);
 
   assert.ok(entity.normalizeNeed);
+  assert.ok(entity.setNeed);
   assert.ok(entity._loadNeed);
   assert.ok(entity.loadNeed);
 
   assert.ok(entity._loadAllNeeds);
   assert.ok(entity.loadAllNeeds);
+
+  assert.ok(entity.resetAllNeeds);
 });
 
 test('normalizeNeed test', function(assert) {
@@ -206,6 +209,27 @@ test('loadAllNeeds silent=true test', function(assert) {
   });
 });
 
+test('setNeed test', function(assert) {
+  let entity = this.subject(),
+      parentModel = Ember.Object.create({
+        refreshLoadTime: function () {
+          assert.ok(true);
+        }
+      }),
+      testModel = {},
+      testName = "name";
+
+  assert.expect(1 + 2);
+
+  entity.setNeed(parentModel, testName, testModel);
+  assert.equal(parentModel.get(testName), testModel);
+
+  parentModel.set("isDeleted", true);
+  parentModel.set(testName, undefined);
+  entity.setNeed(parentModel, testName, testModel);
+  assert.equal(parentModel.get(testName), undefined);
+});
+
 test('_loadNeed single string type test', function(assert) {
   let entity = this.subject(),
       loader,
@@ -300,3 +324,28 @@ test('_loadNeed test with silent false', function(assert) {
     assert.equal(err, testErr);
   });
 });
+
+test('resetAllNeeds test', function(assert) {
+  let entity = this.subject(),
+      parentModel = Ember.Object.create({
+        needs : {
+          foo: {},
+          bar: {}
+        },
+        foo: 1,
+        bar: 2,
+        refreshLoadTime: function () {
+          assert.ok(true);
+        }
+      });
+
+  assert.expect(2 + 2 + 2);
+
+  assert.equal(parentModel.get("foo"), 1);
+  assert.equal(parentModel.get("bar"), 2);
+
+  entity.resetAllNeeds({}, parentModel);
+
+  assert.equal(parentModel.get("foo"), null);
+  assert.equal(parentModel.get("bar"), null);
+});
\ No newline at end of file


[36/50] [abbrv] tez git commit: TEZ-3653. Tez UI: Swimlane tooltip is not proper for running DAGs (sree)

Posted by zh...@apache.org.
TEZ-3653. Tez UI: Swimlane tooltip is not proper for running DAGs (sree)


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

Branch: refs/heads/TEZ-1190
Commit: b2be0d553c54ce4eb9f1315e17da47eb760d9aa5
Parents: 852e80d
Author: Sreenath Somarajapuram <sr...@apache.org>
Authored: Mon Mar 13 18:46:30 2017 +0530
Committer: Sreenath Somarajapuram <sr...@apache.org>
Committed: Mon Mar 13 18:46:30 2017 +0530

----------------------------------------------------------------------
 CHANGES.txt                                       |  1 +
 .../main/webapp/app/controllers/dag/swimlane.js   | 12 +++---------
 tez-ui/src/main/webapp/app/models/timed.js        | 10 ++--------
 .../app/templates/components/em-tooltip.hbs       | 18 +++++++++++-------
 .../src/main/webapp/app/utils/vertex-process.js   |  5 +++--
 .../main/webapp/tests/unit/models/timed-test.js   |  8 ++++----
 .../tests/unit/utils/vertex-process-test.js       |  7 ++++++-
 7 files changed, 30 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 87dda41..df0caa1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -219,6 +219,7 @@ ALL CHANGES:
   TEZ-3642. Tez UI: Auto-refresh is not stopping when DAG is the main entity
   TEZ-3503. Tez UI: Support search by queue name
   TEZ-3656. Tez UI: Status correction is not working as expected
+  TEZ-3653. Tez UI: Swimlane tooltip is not proper for running DAGs
 
 Release 0.8.6: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js
index 8793992..bbac40b 100644
--- a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js
+++ b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js
@@ -55,16 +55,12 @@ export default MultiTableController.extend({
     id: 'startTime',
     headerTitle: 'Start Time',
     contentPath: 'startTime',
-    cellDefinition: {
-      type: 'date'
-    }
+    cellComponentName: 'date-formatter',
   },{
     id: 'endTime',
     headerTitle: 'End Time',
     contentPath: 'endTime',
-    cellDefinition: {
-      type: 'date'
-    }
+    cellComponentName: 'date-formatter',
   },{
     id: 'duration',
     headerTitle: 'Duration',
@@ -80,9 +76,7 @@ export default MultiTableController.extend({
     id: 'firstTaskStartTime',
     headerTitle: 'First Task Start Time',
     contentPath: 'firstTaskStartTime',
-    cellDefinition: {
-      type: 'date'
-    }
+    cellComponentName: 'date-formatter',
   },{
     id: 'totalTasks',
     headerTitle: 'Tasks',

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/app/models/timed.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/models/timed.js b/tez-ui/src/main/webapp/app/models/timed.js
index 8892814..0be71f9 100644
--- a/tez-ui/src/main/webapp/app/models/timed.js
+++ b/tez-ui/src/main/webapp/app/models/timed.js
@@ -30,14 +30,8 @@ export default AbstractModel.extend({
     var startTime = this.get("startTime"),
         endTime = this.get("endTime");
 
-    if(startTime !== undefined && endTime !== undefined && startTime !== null && endTime !== null) {
-      if(startTime <= 0) {
-        return new Error("Start time is less than or equal to zero!");
-      }
-      else if(endTime <= 0) {
-        return new Error("End time is less than or equal to zero!");
-      }
-      else if(startTime > endTime) {
+    if(startTime > 0 && endTime > 0) {
+      if(startTime > endTime) {
         let delta = startTime - endTime;
         return new Error(`Start time is greater than end time by ${delta} msecs!`);
       }

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/app/templates/components/em-tooltip.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-tooltip.hbs b/tez-ui/src/main/webapp/app/templates/components/em-tooltip.hbs
index 65f33db..385baa1 100644
--- a/tez-ui/src/main/webapp/app/templates/components/em-tooltip.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/em-tooltip.hbs
@@ -35,13 +35,17 @@
                   {{prop.name}}
                 </td>
                 <td>
-                  {{txt prop.value
-                  type=prop.type
-                  format=prop.format
-                  timeZone=prop.timeZone
-                  valueFormat=prop.valueFormat
-                  valueTimeZone=prop.valueTimeZone
-                  valueUnit=prop.valueUnit}}
+                  {{#if prop.componentName}}
+                    {{component prop.componentName content=prop.value}}
+                  {{else}}
+                    {{txt prop.value
+                    type=prop.type
+                    format=prop.format
+                    timeZone=prop.timeZone
+                    valueFormat=prop.valueFormat
+                    valueTimeZone=prop.valueTimeZone
+                    valueUnit=prop.valueUnit}}
+                  {{/if}}
                 </td>
               </tr>
             {{/each}}

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/app/utils/vertex-process.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/utils/vertex-process.js b/tez-ui/src/main/webapp/app/utils/vertex-process.js
index b592550..c69fb71 100644
--- a/tez-ui/src/main/webapp/app/utils/vertex-process.js
+++ b/tez-ui/src/main/webapp/app/utils/vertex-process.js
@@ -200,9 +200,10 @@ export default Process.extend({
         let properties = this.getVisibleProps().map(function (definition) {
           return {
             name: definition.get("headerTitle"),
-            value: that.get("vertex").get(definition.get("contentPath")),
+            value: definition.getCellContent(that.get("vertex")),
             type: Ember.get(definition, "cellDefinition.type"),
-            format: Ember.get(definition, "cellDefinition.format")
+            format: Ember.get(definition, "cellDefinition.format"),
+            componentName: Ember.get(definition, "cellComponentName")
           };
         });
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/tests/unit/models/timed-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/models/timed-test.js b/tez-ui/src/main/webapp/tests/unit/models/timed-test.js
index 77b4308..a9ac4e1 100644
--- a/tez-ui/src/main/webapp/tests/unit/models/timed-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/models/timed-test.js
@@ -49,11 +49,11 @@ test('duration test', function(assert) {
     assert.equal(model.get("duration"), 0);
 
     model.set("startTime", 0);
-    assert.equal(model.get("duration").message, "Start time is less than or equal to zero!");
+    assert.equal(model.get("duration"), undefined);
 
     resetAndCheckModel();
     model.set("endTime", 0);
-    assert.equal(model.get("duration").message, "End time is less than or equal to zero!");
+    assert.equal(model.get("duration"), undefined);
 
     resetAndCheckModel();
     model.set("endTime", 50);
@@ -61,11 +61,11 @@ test('duration test', function(assert) {
 
     resetAndCheckModel();
     model.set("startTime", -100);
-    assert.equal(model.get("duration").message, "Start time is less than or equal to zero!");
+    assert.equal(model.get("duration"), undefined);
 
     resetAndCheckModel();
     model.set("endTime", -200);
-    assert.equal(model.get("duration").message, "End time is less than or equal to zero!");
+    assert.equal(model.get("duration"), undefined);
 
     resetAndCheckModel();
     model.set("startTime", undefined);

http://git-wip-us.apache.org/repos/asf/tez/blob/b2be0d55/tez-ui/src/main/webapp/tests/unit/utils/vertex-process-test.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/tests/unit/utils/vertex-process-test.js b/tez-ui/src/main/webapp/tests/unit/utils/vertex-process-test.js
index f9342a5..414704a 100644
--- a/tez-ui/src/main/webapp/tests/unit/utils/vertex-process-test.js
+++ b/tez-ui/src/main/webapp/tests/unit/utils/vertex-process-test.js
@@ -220,6 +220,9 @@ test('getTooltipContents-event test', function(assert) {
 });
 
 test('getTooltipContents-process test', function(assert) {
+  function getCellContent(row) {
+    return row.get(this.get("contentPath"));
+  }
   var process = VertexProcess.create({
     name: "TestName",
     vertex: Ember.Object.create({
@@ -232,6 +235,7 @@ test('getTooltipContents-process test', function(assert) {
         id: "prop1",
         headerTitle: "Prop 1",
         contentPath: "prop1",
+        getCellContent: getCellContent,
         cellDefinition: {
           type: "Type1",
           format: "Format1"
@@ -239,7 +243,8 @@ test('getTooltipContents-process test', function(assert) {
       }), Ember.Object.create({
         id: "prop2",
         headerTitle: "Prop 2",
-        contentPath: "prop2"
+        contentPath: "prop2",
+        getCellContent: getCellContent
       })];
     }
   });


[42/50] [abbrv] tez git commit: TEZ-3663. Add harishjp to Tez teams list. (harishjp)

Posted by zh...@apache.org.
TEZ-3663. Add harishjp to Tez teams list. (harishjp)


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

Branch: refs/heads/TEZ-1190
Commit: a339d050a5773543241ae0e522754fb23a59990d
Parents: 98bf198
Author: Harish JP <ha...@gmail.com>
Authored: Fri Mar 17 07:34:41 2017 +0530
Committer: Harish JP <ha...@gmail.com>
Committed: Fri Mar 17 07:34:41 2017 +0530

----------------------------------------------------------------------
 docs/pom.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/a339d050/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index 97ae60f..61c2cf1 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -176,6 +176,15 @@
         </roles>
       </developer>
       <developer>
+        <id>harishjp</id>
+        <name>Harish JP</name>
+        <email>harishjp@apache.org</email>
+        <timezone>+5.5</timezone>
+        <roles>
+          <role>Committer</role>
+        </roles>
+      </developer>
+      <developer>
         <id>hitesh</id>
         <name>Hitesh Shah</name>
         <email>hitesh@apache.org</email>


[26/50] [abbrv] tez git commit: TEZ-3647. Add a setting which lets Tez determine Xmx. Contributed by Zhiyuan Yang.

Posted by zh...@apache.org.
TEZ-3647. Add a setting which lets Tez determine Xmx. Contributed by
Zhiyuan Yang.


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

Branch: refs/heads/TEZ-1190
Commit: 1b1eb1d4b995c221d2eece2c12fca6ca52b94f27
Parents: 4ce6ea6
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Mar 3 13:45:05 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Mar 3 13:45:05 2017 -0800

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 .../main/java/org/apache/tez/client/TezClientUtils.java   | 10 +++++++++-
 .../java/org/apache/tez/dag/api/TezConfiguration.java     |  4 ++++
 .../main/java/org/apache/tez/dag/api/TezConstants.java    |  6 ++++++
 .../java/org/apache/tez/client/TestTezClientUtils.java    | 10 ++++++++++
 5 files changed, 30 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/1b1eb1d4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 07841bf..763acd8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
 
 ALL CHANGES:
 
+  TEZ-3647. Add a setting which lets Tez determine Xmx.
   TEZ-3644. Cleanup container list stored in AMNode.
   TEZ-3646. IFile.Writer has an extra output stream flush call
   TEZ-3643. Long running AMs can go out of memory due to retained AMContainer instances.

http://git-wip-us.apache.org/repos/asf/tez/blob/1b1eb1d4/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index ab37b17..b16d1ca 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -39,6 +39,7 @@ import java.util.Map.Entry;
 
 import com.google.common.base.Strings;
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.math3.util.Precision;
 import org.apache.tez.common.JavaOptsChecker;
 import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto;
 import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
@@ -963,9 +964,16 @@ public class TezClientUtils {
         || (resource.getMemory() <= 0)) {
       return javaOpts;
     }
-    if (maxHeapFactor <= 0 || maxHeapFactor >= 1) {
+
+    if ((maxHeapFactor <= 0 && !Precision.equals(maxHeapFactor, -1, 0.01)) || maxHeapFactor >= 1) {
       return javaOpts;
     }
+
+    if (Precision.equals(maxHeapFactor, -1, 0.01)) {
+      maxHeapFactor = resource.getMemory() < TezConstants.TEZ_CONTAINER_SMALL_SLAB_BOUND_MB
+        ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB
+        : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB;
+    }
     int maxMemory = (int)(resource.getMemory() * maxHeapFactor);
     maxMemory = maxMemory <= 0 ? 1 : maxMemory;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/1b1eb1d4/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 94f40bb..5900f5e 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -356,6 +356,10 @@ public class TezConfiguration extends Configuration {
    * actual allocation of memory to tasks the cluster. The value if used as a
    * fraction that is applied to the memory allocated Factor to size Xmx based
    * on container memory size. Value should be greater than 0 and less than 1.
+   *
+   * Set this value to -1 to allow Tez to use different default max heap fraction
+   * for different container memory size. Current policy is to use 0.7 for container
+   * smaller than 4GB and use 0.8 for larger container.
    */
   @ConfigurationScope(Scope.AM)
   @ConfigurationProperty(type="float")

http://git-wip-us.apache.org/repos/asf/tez/blob/1b1eb1d4/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
index 06b9cb7..b75269a 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -123,5 +123,11 @@ public class TezConstants {
    */
   public static final long TEZ_AM_CLIENT_HEARTBEAT_POLL_INTERVAL_MILLIS_MINIMUM = 1000;
 
+  /**
+   * For container JVM Xmx auto determination
+   */
+  public static final int TEZ_CONTAINER_SMALL_SLAB_BOUND_MB = 4096;
+  public static final double TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB = 0.7;
+  public static final double TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB = 0.8;
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/1b1eb1d4/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index a5e9d3c..7ff8125 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -629,6 +629,16 @@ public class TestTezClientUtils {
     javaOpts = TezClientUtils.maybeAddDefaultMemoryJavaOpts(origJavaOpts,
         Resource.newInstance(355, 1), 100);
     Assert.assertEquals(origJavaOpts, javaOpts);
+
+    origJavaOpts = "";
+    javaOpts = TezClientUtils.maybeAddDefaultMemoryJavaOpts(origJavaOpts,
+        Resource.newInstance(1000, 1), -1);
+    Assert.assertTrue(javaOpts.contains("-Xmx700m"));
+
+    origJavaOpts = "";
+    javaOpts = TezClientUtils.maybeAddDefaultMemoryJavaOpts(origJavaOpts,
+        Resource.newInstance(5000, 1), -1);
+    Assert.assertTrue(javaOpts.contains("-Xmx4000m"));
   }
 
   @Test (timeout=5000)