You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2015/02/10 22:52:13 UTC
[01/17] tez git commit: TEZ-2035. Make timeline server putDomain
exceptions non-fatal - work-around
Repository: tez
Updated Branches:
refs/heads/TEZ-2003 f49c054c9 -> 6e2f54910 (forced update)
TEZ-2035. Make timeline server putDomain exceptions non-fatal - work-around
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/a8928aff
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/a8928aff
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/a8928aff
Branch: refs/heads/TEZ-2003
Commit: a8928aff11fc6ccf9bc704710440459eed795630
Parents: b804b8f
Author: Jonathan Eagles <je...@gmail.com>
Authored: Thu Feb 5 16:13:03 2015 -0600
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Thu Feb 5 16:13:03 2015 -0600
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java | 4 ++--
2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/a8928aff/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2c54b4b..ee0a225 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -58,6 +58,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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
http://git-wip-us.apache.org/repos/asf/tez/blob/a8928aff/tez-plugins/tez-yarn-timeline-history-with-acls/src/main/java/org/apache/tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/main/java/org/apache/tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/main/java/org/apache/tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java
index e11643f..d1138b0 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/main/java/org/apache/tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java
+++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/main/java/org/apache/tez/dag/history/ats/acls/ATSHistoryACLPolicyManager.java
@@ -111,8 +111,8 @@ public class ATSHistoryACLPolicyManager implements HistoryACLPolicyManager {
try {
timelineClient.putDomain(timelineDomain);
- } catch (YarnException e) {
- throw new IOException(e);
+ } catch (Exception e) {
+ LOG.warn("Could not post timeline domain", e);
}
}
[11/17] tez git commit: TEZ-1928. Tez local mode hang in Pig tez
local mode. (hitesh)
Posted by ss...@apache.org.
TEZ-1928. Tez local mode hang in Pig tez local mode. (hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/28941dc7
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/28941dc7
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/28941dc7
Branch: refs/heads/TEZ-2003
Commit: 28941dc7fe0df3991dbf20ff4bfe7e0e01ec15f3
Parents: 3da9566
Author: Hitesh Shah <hi...@apache.org>
Authored: Mon Feb 9 15:02:25 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Mon Feb 9 15:02:25 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
tez-dag/src/main/java/org/apache/tez/client/LocalClient.java | 3 +++
2 files changed, 4 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/28941dc7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a1b42a8..40668e1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -181,6 +181,7 @@ TEZ-UI CHANGES (TEZ-8):
Release 0.5.4: Unreleased
ALL CHANGES:
+ TEZ-1928. Tez local mode hang in Pig tez local mode.
TEZ-1893. Verify invalid -1 parallelism in DAG.verify().
TEZ-900. Confusing message for incorrect queue for some tez examples.
TEZ-2036. OneToOneEdgeManager should enforce that source and destination
http://git-wip-us.apache.org/repos/asf/tez/blob/28941dc7/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 f309b02..1956f0d 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
@@ -68,6 +68,8 @@ public class LocalClient extends FrameworkClient {
private boolean isSession;
private TezApiVersionInfo versionInfo = new TezApiVersionInfo();
private volatile Throwable amFailException = null;
+ private static final String localModeDAGSchedulerClassName =
+ "org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrderControlled";
public LocalClient() {
}
@@ -78,6 +80,7 @@ public class LocalClient extends FrameworkClient {
tezConf.set("fs.defaultFS", "file:///");
// Tez libs already in the client's classpath
tezConf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
+ tezConf.set(TezConfiguration.TEZ_AM_DAG_SCHEDULER_CLASS, localModeDAGSchedulerClassName);
isSession = tezConf.getBoolean(TezConfiguration.TEZ_AM_SESSION_MODE,
TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT);
}
[14/17] tez git commit: TEZ-2069. Tez UI: appId should link to
application in dag details view. (Sreenath Somarajapuram via hitesh)
Posted by ss...@apache.org.
TEZ-2069. Tez UI: appId should link to application in dag details view. (Sreenath Somarajapuram via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/48055300
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/48055300
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/48055300
Branch: refs/heads/TEZ-2003
Commit: 48055300dbcb7dc4c5d0f8105fcb4b772aae176a
Parents: 4ce0b74
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue Feb 10 12:42:05 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue Feb 10 12:42:05 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../app/scripts/controllers/dag_controller.js | 44 +++++++++++++-------
tez-ui/src/main/webapp/app/templates/dag.hbs | 8 +++-
3 files changed, 36 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/48055300/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cb4c468..4da24a7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/48055300/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
index 1fdf053..b17157f 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
@@ -17,19 +17,18 @@
*/
App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
- controllerName: 'DagController',
+ controllerName: 'DagController',
+ pageTitle: 'Dag',
+ loading: true,
- pageTitle: 'Dag',
-
- loading: true,
-
- updateLoading: function() {
+ updateLoading: function() {
var that = this;
var loaders = [];
+ var applicationId = this.get('applicationId');
if (this.get('status') === 'RUNNING') {
// update the progress info if available. this need not block the UI
var aminfoLoader = that.store.find('dagProgress', that.get('id'), {
- appId: that.get('applicationId'),
+ appId: applicationId,
dagIdx: that.get('idx')
}).then(function(dagProgressInfo) {
that.set('progress', dagProgressInfo.get('progress'));
@@ -38,19 +37,32 @@ App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
});
loaders.push(aminfoLoader);
}
+ var appDetailLoader = this.store.find('appDetail', applicationId)
+ .then(function(app){
+ that.set('appDetail', app);
+ });
+ var tezAppLoader = this.store.find('tezApp', 'tez_' + applicationId)
+ .then(function(app){
+ that.set('tezApp', app);
+ });
+
Em.RSVP.allSettled(loaders).then(function(){
that.set('loading', false);
});
}.observes('content'),
- childDisplayViews: [
- Ember.Object.create({title: 'Details', linkTo: 'dag.index'}),
- Ember.Object.create({title: 'View', linkTo: 'dag.view'}),
- Ember.Object.create({title: 'Vertices', linkTo: 'dag.vertices'}),
- Ember.Object.create({title: 'Tasks', linkTo: 'dag.tasks'}),
- Ember.Object.create({title: 'Task Attempts', linkTo: 'dag.taskAttempts'}),
- Ember.Object.create({title: 'Counters', linkTo: 'dag.counters'}),
- Ember.Object.create({title: 'Swimlane', linkTo: 'dag.swimlane'})
- ],
+ enableAppIdLink: function() {
+ return !!(this.get('tezApp') && this.get('appDetail'));
+ }.property('applicationId', 'appDetail', 'tezApp'),
+
+ childDisplayViews: [
+ Ember.Object.create({title: 'Details', linkTo: 'dag.index'}),
+ Ember.Object.create({title: 'View', linkTo: 'dag.view'}),
+ Ember.Object.create({title: 'Vertices', linkTo: 'dag.vertices'}),
+ Ember.Object.create({title: 'Tasks', linkTo: 'dag.tasks'}),
+ Ember.Object.create({title: 'Task Attempts', linkTo: 'dag.taskAttempts'}),
+ Ember.Object.create({title: 'Counters', linkTo: 'dag.counters'}),
+ Ember.Object.create({title: 'Swimlane', linkTo: 'dag.swimlane'})
+ ],
});
http://git-wip-us.apache.org/repos/asf/tez/blob/48055300/tez-ui/src/main/webapp/app/templates/dag.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dag.hbs b/tez-ui/src/main/webapp/app/templates/dag.hbs
index 6980401..b34aebe 100644
--- a/tez-ui/src/main/webapp/app/templates/dag.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag.hbs
@@ -33,7 +33,13 @@
</tr>
<tr>
<td>{{t 'common.applicationId'}}</td>
- <td>{{applicationId}}</td>
+ <td>
+ {{#if enableAppIdLink}}
+ {{#link-to 'tez-app' applicationId class='ember-table-content'}}{{applicationId}}{{/link-to}}
+ {{else}}
+ <span class='ember-table-content'>{{applicationId}}</span>
+ {{/if}}
+ </td>
</tr>
<tr>
<td>{{t 'common.user'}}</td>
[03/17] tez git commit: TEZ-2032. Update CHANGES.txt to show 0.6.0 is
released (jeagles)
Posted by ss...@apache.org.
TEZ-2032. Update CHANGES.txt to show 0.6.0 is released (jeagles)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/f4667a69
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/f4667a69
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/f4667a69
Branch: refs/heads/TEZ-2003
Commit: f4667a6958c805bd80377ae02702c4c54d7ca0ad
Parents: 4b74df5
Author: Jonathan Eagles <je...@gmail.com>
Authored: Thu Feb 5 16:51:43 2015 -0600
Committer: Jonathan Eagles <je...@gmail.com>
Committed: Thu Feb 5 16:51:43 2015 -0600
----------------------------------------------------------------------
CHANGES.txt | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/f4667a69/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 02a0625..0d02ae5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -58,6 +58,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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
@@ -71,7 +72,7 @@ ALL CHANGES:
TEZ-1987. Tez UI non-standalone mode uses invalid protocol.
TEZ-1983. Tez UI swimlane task attempt link is broken
-Release 0.6.0: Unreleased
+Release 0.6.0: 2015-01-23
INCOMPATIBLE CHANGES
[02/17] tez git commit: TEZ-2018. App Tracking and History URL should
point to the Tez UI. (Prakash Ramachandran via hitesh)
Posted by ss...@apache.org.
TEZ-2018. App Tracking and History URL should point to the Tez UI. (Prakash Ramachandran via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/4b74df55
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4b74df55
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4b74df55
Branch: refs/heads/TEZ-2003
Commit: 4b74df55b56197eb3c6878b37a805b81d8e8cdbe
Parents: a8928af
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Feb 5 14:24:04 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Feb 5 14:24:04 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
pom.xml | 2 +-
.../apache/tez/dag/api/TezConfiguration.java | 37 ++
tez-dag/pom.xml | 22 ++
.../java/org/apache/tez/dag/app/AppContext.java | 2 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 29 +-
.../dag/app/rm/TaskSchedulerEventHandler.java | 52 ++-
.../apache/tez/dag/app/web/AMWebController.java | 354 +++++++++++++++++++
.../apache/tez/dag/app/web/WebUIService.java | 161 +++++++++
.../tez/dag/app/rm/TestContainerReuse.java | 7 +
.../app/rm/TestTaskSchedulerEventHandler.java | 31 +-
.../dag/app/rm/TestTaskSchedulerHelpers.java | 2 +-
.../tez/dag/app/web/TestAMWebController.java | 169 +++++++++
13 files changed, 856 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ee0a225..02a0625 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -58,6 +58,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a05ea3b..3396587 100644
--- a/pom.xml
+++ b/pom.xml
@@ -38,7 +38,7 @@
<maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
<clover.license>${user.home}/clover.license</clover.license>
<hadoop.version>2.6.0</hadoop.version>
- <jetty.version>7.6.16.v20140903</jetty.version>
+ <jetty.version>6.1.26</jetty.version>
<distMgmtSnapshotsId>apache.snapshots.https</distMgmtSnapshotsId>
<distMgmtSnapshotsName>Apache Development Snapshot Repository</distMgmtSnapshotsName>
<distMgmtSnapshotsUrl>https://repository.apache.org/content/repositories/snapshots</distMgmtSnapshotsUrl>
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/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 43307cb..bff5c6a 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
@@ -972,4 +972,41 @@ public class TezConfiguration extends Configuration {
+ "allow.disabled.timeline-domains";
public static final boolean TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS_DEFAULT = false;
+ /**
+ * String value
+ * Tez UI URL template for the application.
+ * Expert level setting.
+ *
+ * The AM will redirect the user to the Tez UI via this url. Template supports the following
+ * parameters to be replaced with the actual runtime information:
+ *
+ * __APPLICATION_ID__ : Replaces this with application ID
+ * __HISTORY_URL_BASE__: replaces this with TEZ_HISTORY_URL_BASE
+ *
+ * For example, "http://uihost:9001/#/tez-app/__APPLICATION_ID__/ will be replaced to
+ * http://uihost:9001/#/tez-app/application_1421880306565_0001/
+ */
+ public static final String TEZ_AM_TEZ_UI_HISTORY_URL_TEMPLATE = TEZ_AM_PREFIX
+ + "tez-ui.history-url.template";
+ public static final String TEZ_AM_TEZ_UI_HISTORY_URL_TEMPLATE_DEFAULT =
+ "__HISTORY_URL_BASE__/#/tez-app/__APPLICATION_ID__";
+
+ /**
+ * String value
+ * Tez-UI Url base. This gets replaced in the TEZ_AM_TEZ_UI_HISTORY_URL_TEMPLATE
+ * ex http://ui-host:9001 or if its hosted with a prefix http://ui-host:9001/~user
+ * if the ui is hosted on the default port (80 for http and 443 for https), the port should not
+ * be specified.
+ */
+ public static final String TEZ_HISTORY_URL_BASE = TEZ_PREFIX
+ + "tez-ui.history-url.base";
+
+ /**
+ * String value
+ * Allow disabling of the Tez AM webservice. If set to false the Tez-UI wont show progress
+ * updates for running application.
+ */
+ public static final String TEZ_AM_WEBSERVICE_ENABLE = TEZ_AM_PREFIX
+ + "tez-ui.webservice.enable";
+ public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 82757d2..d24b383 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -77,6 +77,11 @@
<artifactId>hadoop-yarn-client</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
</dependency>
@@ -131,6 +136,23 @@
<groupId>org.codehaus.jettison</groupId>
<artifactId>jettison</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.mortbay.jetty</groupId>
+ <artifactId>jetty</artifactId>
+ <scope>compile</scope>
+ <version>${jetty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.mortbay.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ <scope>compile</scope>
+ <version>${jetty.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
<build>
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/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 e92fe95..f8086d0 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
@@ -96,4 +96,6 @@ public interface AppContext {
String[] getLogDirs();
String[] getLocalDirs();
+
+ String getAMUser();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/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 e1ab3b7..c7e1e83 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
@@ -143,6 +143,7 @@ import org.apache.tez.dag.app.rm.container.ContainerContextMatcher;
import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
import org.apache.tez.dag.app.rm.node.AMNodeEventType;
import org.apache.tez.dag.app.rm.node.AMNodeTracker;
+import org.apache.tez.dag.app.web.WebUIService;
import org.apache.tez.common.security.ACLManager;
import org.apache.tez.dag.history.DAGHistoryEvent;
import org.apache.tez.dag.history.HistoryEventHandler;
@@ -223,6 +224,7 @@ public class DAGAppMaster extends AbstractService {
private DagEventDispatcher dagEventDispatcher;
private VertexEventDispatcher vertexEventDispatcher;
private TaskSchedulerEventHandler taskSchedulerEventHandler;
+ private WebUIService webUIService;
private HistoryEventHandler historyEventHandler;
private final Map<String, LocalResource> amResources = new HashMap<String, LocalResource>();
private final Map<String, LocalResource> cumulativeAdditionalResources = new HashMap<String, LocalResource>();
@@ -412,9 +414,24 @@ public class DAGAppMaster extends AbstractService {
// register other delegating dispatchers
dispatcher.registerAndCreateDispatcher(SpeculatorEventType.class, new SpeculatorEventHandler(), "Speculator");
+
+ if (enableWebUIService()) {
+ this.webUIService = new WebUIService(context);
+ addIfService(webUIService, false);
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Web UI Service is not enabled.");
+ }
+ }
+
this.taskSchedulerEventHandler = new TaskSchedulerEventHandler(context,
- clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher);
+ clientRpcServer, dispatcher.getEventHandler(), containerSignatureMatcher, webUIService);
addIfService(taskSchedulerEventHandler, true);
+
+ if (enableWebUIService()) {
+ addIfServiceDependency(taskSchedulerEventHandler, webUIService);
+ }
+
if (isLastAMRetry) {
LOG.info("AM will unregister as this is the last attempt"
+ ", currentAttempt=" + appAttemptID.getAttemptId()
@@ -1326,6 +1343,11 @@ public class DAGAppMaster extends AbstractService {
}
@Override
+ public String getAMUser() {
+ return appMasterUgi.getShortUserName();
+ }
+
+ @Override
public Map<ApplicationAccessType, String> getApplicationACLs() {
if (getServiceState() != STATE.STARTED) {
throw new TezUncheckedException(
@@ -2051,4 +2073,9 @@ public class DAGAppMaster extends AbstractService {
synchronized void setDAGCounter(int dagCounter) {
this.dagCounter.set(dagCounter);
}
+
+ private boolean enableWebUIService() {
+ return amConf.getBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE,
+ TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 625b09e..616690c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -26,8 +26,10 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.Container;
@@ -69,6 +71,7 @@ import org.apache.tez.dag.app.rm.node.AMNodeEventNodeCountUpdated;
import org.apache.tez.dag.app.rm.node.AMNodeEventStateChanged;
import org.apache.tez.dag.app.rm.node.AMNodeEventTaskAttemptEnded;
import org.apache.tez.dag.app.rm.node.AMNodeEventTaskAttemptSucceeded;
+import org.apache.tez.dag.app.web.WebUIService;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
import com.google.common.base.Preconditions;
@@ -79,9 +82,13 @@ public class TaskSchedulerEventHandler extends AbstractService
EventHandler<AMSchedulerEvent> {
static final Log LOG = LogFactory.getLog(TaskSchedulerEventHandler.class);
+ static final String APPLICATION_ID_PLACEHOLDER = "__APPLICATION_ID__";
+ static final String HISTORY_URL_BASE = "__HISTORY_URL_BASE__";
+
protected final AppContext appContext;
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;
+ private final String historyUrl;
protected TaskSchedulerService taskScheduler;
private DAGAppMaster dagAppMaster;
private Map<ApplicationAccessType, String> appAcls = null;
@@ -94,18 +101,25 @@ public class TaskSchedulerEventHandler extends AbstractService
private int cachedNodeCount = -1;
private AtomicBoolean shouldUnregisterFlag =
new AtomicBoolean(false);
+ private final WebUIService webUI;
BlockingQueue<AMSchedulerEvent> eventQueue
= new LinkedBlockingQueue<AMSchedulerEvent>();
@SuppressWarnings("rawtypes")
public TaskSchedulerEventHandler(AppContext appContext,
- DAGClientServer clientService, EventHandler eventHandler, ContainerSignatureMatcher containerSignatureMatcher) {
+ DAGClientServer clientService, EventHandler eventHandler,
+ ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
super(TaskSchedulerEventHandler.class.getName());
this.appContext = appContext;
this.eventHandler = eventHandler;
this.clientService = clientService;
this.containerSignatureMatcher = containerSignatureMatcher;
+ this.webUI = webUI;
+ this.historyUrl = getHistoryUrl();
+ if (this.webUI != null) {
+ this.webUI.setHistoryUrl(this.historyUrl);
+ }
}
public Map<ApplicationAccessType, String> getApplicationAcls() {
@@ -328,8 +342,13 @@ public class TaskSchedulerEventHandler extends AbstractService
public synchronized void serviceStart() {
InetSocketAddress serviceAddr = clientService.getBindAddress();
dagAppMaster = appContext.getAppMaster();
+ // if web service is enabled then set tracking url. else disable it (value = "").
+ // the actual url set on the rm web ui will be the proxy url set by WebAppProxyServlet, which
+ // always try to connect to AM and proxy the response. hence it wont work if the webUIService
+ // is not enabled.
+ String trackingUrl = (webUI != null) ? webUI.getURL() : "";
taskScheduler = createTaskScheduler(serviceAddr.getHostName(),
- serviceAddr.getPort(), "", appContext);
+ serviceAddr.getPort(), trackingUrl, appContext);
taskScheduler.init(getConfig());
taskScheduler.start();
if (shouldUnregisterFlag.get()) {
@@ -514,11 +533,8 @@ public class TaskSchedulerEventHandler extends AbstractService
LOG.debug("Setting job diagnostics to " + sb.toString());
}
- String historyUrl = "";
- /*String historyUrl = JobHistoryUtils.getHistoryUrl(getConfig(),
- appContext.getApplicationID());
- LOG.info("History url is " + historyUrl);*/
-
+ // if history url is set use the same, if historyUrl is set to "" then rm ui disables the
+ // history url
return new AppFinalStatus(finishState, sb.toString(), historyUrl);
}
@@ -569,4 +585,26 @@ public class TaskSchedulerEventHandler extends AbstractService
public boolean hasUnregistered() {
return this.taskScheduler.hasUnregistered();
}
+
+ @VisibleForTesting
+ public String getHistoryUrl() {
+ Configuration config = this.appContext.getAMConf();
+ String historyUrl = "";
+
+ String loggingClass = config.get(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS, "");
+ String historyUrlTemplate = config.get(TezConfiguration.TEZ_AM_TEZ_UI_HISTORY_URL_TEMPLATE,
+ TezConfiguration.TEZ_AM_TEZ_UI_HISTORY_URL_TEMPLATE_DEFAULT);
+ String historyUrlBase = config.get(TezConfiguration.TEZ_HISTORY_URL_BASE, "");
+
+
+ if (loggingClass.equals("org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService") &&
+ !historyUrlTemplate.isEmpty() &&
+ !historyUrlBase.isEmpty()) {
+ historyUrl = historyUrlTemplate
+ .replaceAll(APPLICATION_ID_PLACEHOLDER, appContext.getApplicationID().toString())
+ .replaceAll(HISTORY_URL_BASE, historyUrlBase);
+ }
+
+ return historyUrl;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
new file mode 100644
index 0000000..b3e404a
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
@@ -0,0 +1,354 @@
+/**
+ * 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.web;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import com.google.inject.name.Named;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.webapp.Controller;
+import org.apache.hadoop.yarn.webapp.MimeType;
+import org.apache.hadoop.yarn.webapp.View;
+import org.apache.hadoop.yarn.webapp.WebAppException;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezVertexID;
+
+public class AMWebController extends Controller {
+
+ private final static Log LOG = LogFactory.getLog(AMWebController.class);
+
+ // HTTP CORS Response Headers
+ static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin";
+ static final String ACCESS_CONTROL_ALLOW_CREDENTIALS = "Access-Control-Allow-Credentials";
+ static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods";
+ static final String ACCESS_CONTROL_ALLOW_HEADERS = "Access-Control-Allow-Headers";
+ static final String ACCESS_CONTROL_MAX_AGE = "Access-Control-Max-Age";
+
+ // CORS default responses.
+ static final String ALLOWED_METHODS = "GET, HEAD";
+ static final String ALLOWED_HEADERS = "X-Requested-With,Content-Type,Accept,Origin";
+
+ static final String DAG_PROGRESS = "dagProgress";
+ static final String VERTEX_PROGRESS = "vertexProgress";
+ static final String VERTEX_PROGRESSES = "vertexProgresses";
+
+ static final int MAX_VERTICES_QUERIED = 100;
+
+ private AppContext appContext;
+ private String historyUrl;
+
+ @Inject
+ public AMWebController(RequestContext requestContext,
+ AppContext appContext,
+ @Named("TezUIHistoryURL") String historyUrl) {
+ super(requestContext);
+ this.appContext = appContext;
+ this.historyUrl = historyUrl;
+ }
+
+ @Override
+ public void index() {
+ ui();
+ }
+
+ public void ui() {
+ render(StaticAMView.class);
+ }
+
+ public void main() {
+ ui();
+ }
+
+ public void about() {
+ renderJSON("Tez AM UI WebServices");
+ }
+
+ @VisibleForTesting
+ public void setCorsHeaders() {
+ final HttpServletResponse res = response();
+
+ /*
+ * ideally the Origin and other CORS headers should be checked and response headers set only
+ * if it matches the allowed origins. however rm does not forward these headers.
+ */
+ String historyUrlBase = appContext.getAMConf().get(TezConfiguration.TEZ_HISTORY_URL_BASE, "");
+ String origin = null;
+ try {
+ URL url = new URL(historyUrlBase);
+ origin = url.getProtocol() + "://" + url.getAuthority();
+ } catch (MalformedURLException e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Invalid url set for tez history url base: " + historyUrlBase, e);
+ }
+ }
+
+ if (origin != null) {
+ res.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, origin);
+ }
+ res.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS);
+ res.setHeader(ACCESS_CONTROL_ALLOW_CREDENTIALS, Boolean.TRUE.toString());
+ res.setHeader(ACCESS_CONTROL_ALLOW_HEADERS, ALLOWED_HEADERS);
+ res.setHeader(ACCESS_CONTROL_MAX_AGE, "1800");
+ }
+
+ void sendErrorResponse(int sc, String msg, Exception e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(msg, e);
+ }
+
+ try {
+ response().sendError(sc, msg);
+ } catch (IOException e1) {
+ throw new WebAppException(e);
+ }
+ }
+
+ @VisibleForTesting
+ public boolean hasAccess() {
+ String remoteUser = request().getRemoteUser();
+ UserGroupInformation callerUGI = null;
+ if (remoteUser != null && !remoteUser.isEmpty()) {
+ callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+ }
+
+ if (callerUGI != null && appContext.getAMACLManager().checkDAGViewAccess(callerUGI)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ public void getDagProgress() {
+
+ setCorsHeaders();
+
+ if (!hasAccess()) {
+ sendErrorResponse(HttpServletResponse.SC_UNAUTHORIZED, "Access denied for user: " +
+ request().getRemoteUser(), null);
+ return;
+ }
+
+ int dagID;
+ try {
+ dagID = getQueryParamInt(WebUIService.DAG_ID);
+ } catch (NumberFormatException e) {
+ sendErrorResponse(HttpServletResponse.SC_BAD_REQUEST, "Invalid dag id:", e);
+ return;
+ }
+
+ DAG currentDAG = appContext.getCurrentDAG();
+
+ if (currentDAG == null || dagID != currentDAG.getID().getId()) {
+ sendErrorResponse(HttpServletResponse.SC_NOT_FOUND, "Not current Dag: " + dagID, null);
+ return;
+ }
+
+ Map<String, ProgressInfo> result = new HashMap<String, ProgressInfo>();
+ result.put(DAG_PROGRESS,
+ new ProgressInfo(currentDAG.getID().toString(), currentDAG.getProgress()));
+ renderJSON(result);
+ }
+
+ public void getVertexProgress() {
+ int dagID;
+ int vertexID;
+
+ setCorsHeaders();
+
+ if (!hasAccess()) {
+ sendErrorResponse(HttpServletResponse.SC_UNAUTHORIZED, "Access denied for user: " +
+ request().getRemoteUser(), null);
+ return;
+ }
+
+ try {
+ dagID = getQueryParamInt(WebUIService.DAG_ID);
+ vertexID = getQueryParamInt(WebUIService.VERTEX_ID);
+ } catch (NumberFormatException e) {
+ sendErrorResponse(HttpServletResponse.SC_BAD_REQUEST, "Invalid dag or vertex id", e);
+ return;
+ }
+
+ DAG currentDAG = appContext.getCurrentDAG();
+
+ if (currentDAG == null || currentDAG.getID().getId() != dagID) {
+ sendErrorResponse(HttpServletResponse.SC_NOT_FOUND, "Not current Dag: " + dagID, null);
+ return;
+ }
+
+ final TezVertexID tezVertexID = TezVertexID.getInstance(currentDAG.getID(), vertexID);
+ Vertex vertex = currentDAG.getVertex(tezVertexID);
+ if (vertex == null) {
+ sendErrorResponse(HttpServletResponse.SC_NOT_FOUND, "vertex not found: " + vertexID, null);
+ return;
+ }
+
+ Map<String, ProgressInfo> result = new HashMap<String, ProgressInfo>();
+ result.put(VERTEX_PROGRESS, new ProgressInfo(tezVertexID.toString(), vertex.getProgress()));
+ renderJSON(result);
+ }
+
+
+ Collection<Vertex> getVerticesByIdx(DAG dag, Collection<Integer> indexes) {
+ Collection<Vertex> vertices = new ArrayList<Vertex>(indexes.size());
+ final TezDAGID tezDAGID = dag.getID();
+
+ for (Integer idx : indexes) {
+ final TezVertexID tezVertexID = TezVertexID.getInstance(tezDAGID, idx);
+ if (tezVertexID == null) {
+ continue;
+ }
+ final Vertex vertex = dag.getVertex(tezVertexID);
+ if (vertex != null) {
+ vertices.add(vertex);
+ }
+ }
+
+ return vertices;
+ }
+
+ int getQueryParamInt(String name) throws NumberFormatException {
+ final String valueStr = $(name).trim();
+
+ return Integer.parseInt(valueStr);
+ }
+
+ public void getVertexProgresses() {
+ int dagID;
+
+ setCorsHeaders();
+ if (!hasAccess()) {
+ sendErrorResponse(HttpServletResponse.SC_UNAUTHORIZED, "Access denied for user: " +
+ request().getRemoteUser(), null);
+ return;
+ }
+
+ List<Integer> vertexIDs = new ArrayList<Integer>();
+ try {
+ dagID = getQueryParamInt(WebUIService.DAG_ID);
+ for (String vertexIDStr : $(WebUIService.VERTEX_ID).trim().split(",", MAX_VERTICES_QUERIED)) {
+ vertexIDs.add(Integer.parseInt(vertexIDStr));
+ }
+ } catch (NumberFormatException e) {
+ sendErrorResponse(HttpServletResponse.SC_BAD_REQUEST, "Invalid dag or vertices id", e);
+ return;
+ }
+
+ DAG currentDAG = appContext.getCurrentDAG();
+ if (currentDAG == null || currentDAG.getID().getId() != dagID) {
+ sendErrorResponse(HttpServletResponse.SC_NOT_FOUND, "Not current Dag: " + dagID, null);
+ return;
+ }
+
+ Collection<Vertex> vertices;
+ if (vertexIDs.isEmpty()) {
+ vertices = currentDAG.getVertices().values();
+ } else {
+ vertices = getVerticesByIdx(currentDAG, vertexIDs);
+ }
+
+ Collection<ProgressInfo> progresses = new ArrayList<ProgressInfo>(vertices.size());
+ for(Vertex vertex : vertices) {
+ progresses.add(new ProgressInfo(vertex.getVertexId().toString(), vertex.getProgress()));
+ }
+
+ Map<String, Collection<ProgressInfo>> result = new HashMap<String, Collection<ProgressInfo>>();
+ result.put(VERTEX_PROGRESSES, progresses);
+ renderJSON(result);
+ }
+
+ @Override
+ @VisibleForTesting
+ public void renderJSON(Object object) {
+ super.renderJSON(object);
+ }
+
+ public static class StaticAMView extends View {
+ @Inject
+ AppContext appContext;
+ @Inject
+ @Named("TezUIHistoryURL") String historyUrl;
+
+ @Override
+ public void render() {
+ response().setContentType(MimeType.HTML);
+ PrintWriter pw = writer();
+ pw.write("<html>");
+ pw.write("<head>");
+ pw.write("<meta charset=\"utf-8\">");
+ pw.write("<title>Redirecting to Tez UI</title>");
+ pw.write("</head>");
+ pw.write("<body>");
+ if (historyUrl == null || historyUrl.isEmpty()) {
+ pw.write("<h1>Tez UI Url is not defined.</h1>" +
+ "<p>To enable tracking url pointing to Tez UI, set the config <b>" +
+ TezConfiguration.TEZ_HISTORY_URL_BASE + "</b> in the tez-site.xml.</p>");
+ } else {
+ pw.write("<h1>Redirecting to Tez UI</h1>. <p>If you are not redirected shortly, click" +
+ "<a href='" + historyUrl + "'><b>here</b></a></p>"
+ );
+ pw.write("<script type='text/javascript'>setTimeout(function() { " +
+ "window.location.assign('" + historyUrl + "');" +
+ "}, 0); </script>");
+ }
+ pw.write("</body>");
+ pw.write("</html>");
+ pw.flush();
+ }
+ }
+
+ @VisibleForTesting
+ static class ProgressInfo {
+ private String id;
+
+ public float getProgress() {
+ return progress;
+ }
+
+ public String getId() {
+ return id;
+ }
+
+ private float progress;
+
+ public ProgressInfo(String id, float progress) {
+ this.id = id;
+ this.progress = progress;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java
new file mode 100644
index 0000000..44f99c8
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java
@@ -0,0 +1,161 @@
+/**
+ * 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.web;
+
+import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
+
+import java.net.InetSocketAddress;
+
+import com.google.common.base.Preconditions;
+import com.google.inject.name.Names;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.WebApps;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+
+public class WebUIService extends AbstractService {
+ private static final String WS_PREFIX = "/ui/ws/v1/tez/";
+ public static final String VERTEX_ID = "vertexID";
+ public static final String DAG_ID = "dagID";
+
+ private static final Log LOG = LogFactory.getLog(WebUIService.class);
+
+ private final AppContext context;
+ private TezAMWebApp tezAMWebApp;
+ private WebApp webApp;
+ private int port;
+ private String historyUrl = "";
+
+ public WebUIService(AppContext context) {
+ super(WebUIService.class.getName());
+ this.context = context;
+ this.tezAMWebApp = new TezAMWebApp(context);
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ if (historyUrl == null || historyUrl.isEmpty()) {
+ LOG.error("Tez UI History URL is not set");
+ } else {
+ LOG.info("Tez UI History URL: " + historyUrl);
+ }
+
+ if (tezAMWebApp != null) {
+ this.tezAMWebApp.setHistoryUrl(historyUrl);
+ }
+ super.serviceInit(conf);
+ }
+
+ @Override
+ protected void serviceStart() throws Exception {
+ if (tezAMWebApp != null) {
+ // use AmIpFilter to restrict connections only from the rm proxy
+ final Configuration conf = getConfig();
+ conf.set("hadoop.http.filter.initializers",
+ "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer");
+ try {
+ // Explicitly disabling SSL for the web service. For https we do not want AM users to allow
+ // access to the keystore file for opening SSL listener. We can trust RM/NM to issue SSL
+ // certificates, however AM user is not trusted.
+ this.webApp = WebApps
+ .$for(this.tezAMWebApp)
+ .with(conf)
+ .withHttpPolicy(conf, HttpConfig.Policy.HTTP_ONLY)
+ .start(this.tezAMWebApp);
+ this.port = this.webApp.httpServer().getConnectorAddress(0).getPort();
+ } catch (Exception e) {
+ LOG.error("Tez UI WebService failed to start.", e);
+ throw new TezUncheckedException(e);
+ }
+ }
+ super.serviceStart();
+ }
+
+ @Override
+ protected void serviceStop() throws Exception {
+ if (this.webApp != null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Stopping WebApp");
+ }
+ this.webApp.stop();
+ }
+ super.serviceStop();
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public String getURL() {
+ String url = "";
+ InetSocketAddress address = webApp.getListenerAddress();
+
+ if (address != null) {
+ final String hostName = address.getAddress().getCanonicalHostName();
+ final int port = address.getPort();
+ url = "http://" + hostName + ":" + port + "/ui/";
+ }
+
+ return url;
+ }
+
+ public String getHistoryUrl() {
+ return historyUrl;
+ }
+
+ public void setHistoryUrl(String historyUrl) {
+ this.historyUrl = historyUrl;
+ }
+
+ private static class TezAMWebApp extends WebApp implements YarnWebParams {
+
+ private String historyUrl;
+ AppContext context;
+
+ public TezAMWebApp(AppContext context) {
+ this.context = context;
+ }
+
+ public void setHistoryUrl(String historyUrl) {
+ this.historyUrl = historyUrl;
+ }
+
+ @Override
+ public void setup() {
+ Preconditions.checkArgument(historyUrl != null);
+ bind(AppContext.class).toInstance(context);
+ bind(String.class).annotatedWith(Names.named("TezUIHistoryURL")).toInstance(historyUrl);
+ route("/", AMWebController.class, "ui");
+ route("/ui", AMWebController.class, "ui");
+ route("/main", AMWebController.class, "main");
+ route(WS_PREFIX + "about", AMWebController.class, "about");
+ route(WS_PREFIX + pajoin("dagProgress", DAG_ID), AMWebController.class, "getDagProgress");
+ route(WS_PREFIX + pajoin("vertexProgress", VERTEX_ID), AMWebController.class,
+ "getVertexProgress");
+ route(WS_PREFIX + pajoin("vertexProgresses", VERTEX_ID, DAG_ID), AMWebController.class,
+ "getVertexProgresses");
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 311e762..d0d3df8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -137,6 +137,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(conf).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(
mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class),
@@ -274,6 +275,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(
mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class),
@@ -378,6 +380,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class), new ContainerContextMatcher(), appContext);
AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
@@ -517,6 +520,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class), new ContainerContextMatcher(), appContext);
AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext);
@@ -707,6 +711,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(
mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class),
@@ -835,6 +840,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
AMContainerMap amContainerMap = new AMContainerMap(
mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class),
@@ -956,6 +962,7 @@ public class TestContainerReuse {
doReturn(finalStatus).when(mockApp).getFinalAppStatus();
AppContext appContext = mock(AppContext.class);
+ doReturn(new Configuration(false)).when(appContext).getAMConf();
ChangingDAGIDAnswer dagIDAnswer = new ChangingDAGIDAnswer(dagID1);
AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class),
mock(TaskAttemptListener.class), new ContainerContextMatcher(), appContext);
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
index 62618cc..2bb2fcd 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerEventHandler.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.rm;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -31,6 +32,7 @@ import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -39,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.dag.api.TaskLocationHint;
+import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.client.DAGClientServer;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
@@ -49,6 +52,7 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventCompleted;
import org.apache.tez.dag.app.rm.container.AMContainerEventType;
import org.apache.tez.dag.app.rm.container.AMContainerMap;
import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher;
+import org.apache.tez.dag.app.web.WebUIService;
import org.apache.tez.dag.records.TaskAttemptTerminationCause;
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.junit.Assert;
@@ -74,8 +78,8 @@ public class TestTaskSchedulerEventHandler {
public MockTaskSchedulerEventHandler(AppContext appContext,
DAGClientServer clientService, EventHandler eventHandler,
- ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, clientService, eventHandler, containerSignatureMatcher);
+ ContainerSignatureMatcher containerSignatureMatcher, WebUIService webUI) {
+ super(appContext, clientService, eventHandler, containerSignatureMatcher, webUI);
}
@Override
@@ -101,19 +105,22 @@ public class TestTaskSchedulerEventHandler {
MockTaskSchedulerEventHandler schedulerHandler;
TaskSchedulerService mockTaskScheduler;
AMContainerMap mockAMContainerMap;
+ WebUIService mockWebUIService;
@Before
public void setup() {
mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ doReturn(new Configuration(false)).when(mockAppContext).getAMConf();
mockClientService = mock(DAGClientServer.class);
mockEventHandler = new TestEventHandler();
mockSigMatcher = mock(ContainerSignatureMatcher.class);
mockTaskScheduler = mock(TaskSchedulerService.class);
mockAMContainerMap = mock(AMContainerMap.class);
+ mockWebUIService = mock(WebUIService.class);
when(mockAppContext.getAllContainers()).thenReturn(mockAMContainerMap);
when(mockClientService.getBindAddress()).thenReturn(new InetSocketAddress(10000));
schedulerHandler = new MockTaskSchedulerEventHandler(
- mockAppContext, mockClientService, mockEventHandler, mockSigMatcher);
+ mockAppContext, mockClientService, mockEventHandler, mockSigMatcher, mockWebUIService);
}
@Test (timeout = 5000)
@@ -246,4 +253,22 @@ public class TestTaskSchedulerEventHandler {
schedulerHandler.close();
}
+ @Test (timeout = 5000)
+ public void testHistoryUrlConf() throws Exception {
+ Configuration conf = schedulerHandler.appContext.getAMConf();
+
+ // ensure history url is empty when timeline server is not the logging class
+ conf.set(TezConfiguration.TEZ_HISTORY_URL_BASE, "http://ui-host:9999");
+ Assert.assertTrue("".equals(schedulerHandler.getHistoryUrl()));
+
+ // ensure expansion of url happens
+ conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ "org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService");
+ final ApplicationId mockApplicationId = mock(ApplicationId.class);
+ doReturn("TEST_APP_ID").when(mockApplicationId).toString();
+ doReturn(mockApplicationId).when(mockAppContext).getApplicationID();
+ Assert.assertTrue("http://ui-host:9999/#/tez-app/TEST_APP_ID"
+ .equals(schedulerHandler.getHistoryUrl()));
+ }
+
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/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 b0ea644..bec5320 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
@@ -127,7 +127,7 @@ class TestTaskSchedulerHelpers {
EventHandler eventHandler,
TezAMRMClientAsync<CookieContainerRequest> amrmClientAsync,
ContainerSignatureMatcher containerSignatureMatcher) {
- super(appContext, null, eventHandler, containerSignatureMatcher);
+ super(appContext, null, eventHandler, containerSignatureMatcher, null);
this.amrmClientAsync = amrmClientAsync;
this.containerSignatureMatcher = containerSignatureMatcher;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/4b74df55/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
new file mode 100644
index 0000000..588eb21
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
@@ -0,0 +1,169 @@
+/**
+ * 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.web;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.webapp.Controller;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.MockitoAnnotations;
+
+public class TestAMWebController {
+ AppContext mockAppContext;
+ Controller.RequestContext mockRequestContext;
+ HttpServletResponse mockResponse;
+ HttpServletRequest mockRequst;
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+
+ mockAppContext = mock(AppContext.class);
+ Configuration conf = new Configuration(false);
+ conf.set(TezConfiguration.TEZ_HISTORY_URL_BASE, "http://uihost:9001/foo");
+ when(mockAppContext.getAMConf()).thenReturn(conf);
+ mockRequestContext = mock(Controller.RequestContext.class);
+ mockResponse = mock(HttpServletResponse.class);
+ mockRequst = mock(HttpServletRequest.class);
+ }
+
+ @Test(timeout = 5000)
+ public void testCorsHeadersAreSet() {
+ AMWebController amWebController = new AMWebController(mockRequestContext, mockAppContext,
+ "TEST_HISTORY_URL");
+ AMWebController spy = spy(amWebController);
+ doReturn(mockResponse).when(spy).response();
+ spy.setCorsHeaders();
+
+ verify(mockResponse).setHeader("Access-Control-Allow-Origin", "http://uihost:9001");
+ verify(mockResponse).setHeader("Access-Control-Allow-Credentials", "true");
+ verify(mockResponse).setHeader("Access-Control-Allow-Methods", "GET, HEAD");
+ verify(mockResponse).setHeader("Access-Control-Allow-Headers",
+ "X-Requested-With,Content-Type,Accept,Origin");
+ }
+
+ @Test (timeout = 5000)
+ public void sendErrorResponseIfNoAccess() throws Exception {
+ AMWebController amWebController = new AMWebController(mockRequestContext, mockAppContext,
+ "TEST_HISTORY_URL");
+ AMWebController spy = spy(amWebController);
+
+ doReturn(false).when(spy).hasAccess();
+ doNothing().when(spy).setCorsHeaders();
+ doReturn(mockResponse).when(spy).response();
+ doReturn(mockRequst).when(spy).request();
+ doReturn("dummyuser").when(mockRequst).getRemoteUser();
+
+ spy.getDagProgress();
+ verify(mockResponse).sendError(eq(HttpServletResponse.SC_UNAUTHORIZED), anyString());
+ reset(mockResponse);
+
+ spy.getVertexProgress();
+ verify(mockResponse).sendError(eq(HttpServletResponse.SC_UNAUTHORIZED), anyString());
+ reset(mockResponse);
+
+ spy.getVertexProgresses();
+ verify(mockResponse).sendError(eq(HttpServletResponse.SC_UNAUTHORIZED), anyString());
+ }
+
+ @Captor
+ ArgumentCaptor<Map<String, AMWebController.ProgressInfo>> singleResultCaptor;
+
+ @Test (timeout = 5000)
+ public void testDagProgressResponse() {
+ AMWebController amWebController = new AMWebController(mockRequestContext, mockAppContext,
+ "TEST_HISTORY_URL");
+ AMWebController spy = spy(amWebController);
+ DAG mockDAG = mock(DAG.class);
+
+ doReturn(true).when(spy).hasAccess();
+ doNothing().when(spy).setCorsHeaders();
+ doReturn("42").when(spy).$(WebUIService.DAG_ID);
+ doReturn(mockResponse).when(spy).response();
+ doReturn(TezDAGID.fromString("dag_1422960590892_0007_42")).when(mockDAG).getID();
+ doReturn(66.0f).when(mockDAG).getProgress();
+ doReturn(mockDAG).when(mockAppContext).getCurrentDAG();
+ doNothing().when(spy).renderJSON(any());
+ spy.getDagProgress();
+ verify(spy).renderJSON(singleResultCaptor.capture());
+
+ final Map<String, AMWebController.ProgressInfo> result = singleResultCaptor.getValue();
+ Assert.assertEquals(1, result.size());
+ Assert.assertTrue(result.containsKey("dagProgress"));
+ AMWebController.ProgressInfo progressInfo = result.get("dagProgress");
+ Assert.assertTrue("dag_1422960590892_0007_42".equals(progressInfo.getId()));
+ Assert.assertEquals(66.0, progressInfo.getProgress(), 0.1);
+ }
+
+ @Test (timeout = 5000)
+ public void testVertexProgressResponse() {
+ AMWebController amWebController = new AMWebController(mockRequestContext, mockAppContext,
+ "TEST_HISTORY_URL");
+ AMWebController spy = spy(amWebController);
+ DAG mockDAG = mock(DAG.class);
+ Vertex mockVertex = mock(Vertex.class);
+
+ doReturn(true).when(spy).hasAccess();
+ doReturn("42").when(spy).$(WebUIService.DAG_ID);
+ doReturn("43").when(spy).$(WebUIService.VERTEX_ID);
+ doReturn(mockResponse).when(spy).response();
+
+ doReturn(TezDAGID.fromString("dag_1422960590892_0007_42")).when(mockDAG).getID();
+ doReturn(mockDAG).when(mockAppContext).getCurrentDAG();
+ doReturn(mockVertex).when(mockDAG).getVertex(any(TezVertexID.class));
+ doReturn(66.0f).when(mockVertex).getProgress();
+ doNothing().when(spy).renderJSON(any());
+ doNothing().when(spy).setCorsHeaders();
+
+ spy.getVertexProgress();
+ verify(spy).renderJSON(singleResultCaptor.capture());
+
+ final Map<String, AMWebController.ProgressInfo> result = singleResultCaptor.getValue();
+ Assert.assertEquals(1, result.size());
+ Assert.assertTrue(result.containsKey("vertexProgress"));
+ AMWebController.ProgressInfo progressInfo = result.get("vertexProgress");
+ Assert.assertTrue("vertex_1422960590892_0007_42_43".equals(progressInfo.getId()));
+ Assert.assertEquals(66.0f, progressInfo.getProgress(), 0.1);
+ }
+}
[16/17] tez git commit: TEZ-2045. TaskAttemptListener should not pull
Tasks from AMContainer. (sseth)
Posted by ss...@apache.org.
TEZ-2045. TaskAttemptListener should not pull Tasks from AMContainer.
(sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/fe39ede3
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/fe39ede3
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/fe39ede3
Branch: refs/heads/TEZ-2003
Commit: fe39ede3305bab665fcdbca07fd381be0e875e80
Parents: f035468
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 10 13:42:23 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 10 13:42:23 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/tez/dag/app/TaskAttemptListener.java | 35 +--
.../dag/app/TaskAttemptListenerImpTezDag.java | 93 +++---
.../tez/dag/app/rm/container/AMContainer.java | 5 +-
.../app/rm/container/AMContainerEventType.java | 3 -
.../dag/app/rm/container/AMContainerImpl.java | 304 ++++++------------
.../dag/app/rm/container/AMContainerTask.java | 10 +-
.../app/TestTaskAttemptListenerImplTezDag.java | 182 +++++++++++
.../dag/app/rm/container/TestAMContainer.java | 313 ++++++++++---------
9 files changed, 505 insertions(+), 441 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d617bee..9979c50 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ Release 0.7.0: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
index e80c8b3..aeb0cd5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListener.java
@@ -21,6 +21,7 @@ package org.apache.tez.dag.app;
import java.net.InetSocketAddress;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.records.TezTaskAttemptID;
/**
* This class listens for changes to the state of a Task.
@@ -30,41 +31,11 @@ public interface TaskAttemptListener {
InetSocketAddress getAddress();
void registerRunningContainer(ContainerId containerId);
-// void registerRunningJvm(WrappedJvmID jvmID, ContainerId containerId);
-
- void registerTaskAttempt(TezTaskAttemptID attemptId, ContainerId containerId);
-
-// void registerTaskAttempt(TezTaskAttemptID attemptId, WrappedJvmID jvmId);
+
+ void registerTaskAttempt(AMContainerTask amContainerTask, ContainerId containerId);
void unregisterRunningContainer(ContainerId containerId);
-// void unregisterRunningJvm(WrappedJvmID jvmID);
-
void unregisterTaskAttempt(TezTaskAttemptID attemptID);
- /**
- * Register a JVM with the listener. This should be called as soon as a
- * JVM ID is assigned to a task attempt, before it has been launched.
- * @param task the task itself for this JVM.
- * @param jvmID The ID of the JVM .
- */
-// void registerPendingTask(Task task, WrappedJvmID jvmID);
-
- /**
- * Register task attempt. This should be called when the JVM has been
- * launched.
- *
- * @param attemptID
- * the id of the attempt for this JVM.
- * @param jvmID the ID of the JVM.
- */
-// void registerLaunchedTask(TezTaskAttemptID attemptID, WrappedJvmID jvmID);
-
- /**
- * Unregister the JVM and the attempt associated with it. This should be
- * called when the attempt/JVM has finished executing and is being cleaned up.
- * @param attemptID the ID of the attempt.
- * @param jvmID the ID of the JVM for that attempt.
- */
-// void unregister(TezTaskAttemptID attemptID, WrappedJvmID jvmID);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index b1cb3f6..28f2c32 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -53,7 +53,6 @@ import org.apache.tez.dag.app.dag.DAG;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
-import org.apache.tez.dag.app.rm.container.AMContainerImpl;
import org.apache.tez.dag.app.rm.container.AMContainerTask;
import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -87,11 +86,13 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
ContainerInfo() {
this.lastReponse = null;
this.lastRequestId = 0;
- this.currentAttemptId = null;
+ this.amContainerTask = null;
+ this.taskPulled = false;
}
long lastRequestId;
TezHeartbeatResponse lastReponse;
- TezTaskAttemptID currentAttemptId;
+ AMContainerTask amContainerTask;
+ boolean taskPulled;
}
private ConcurrentMap<TezTaskAttemptID, ContainerId> attemptToInfoMap =
@@ -212,30 +213,18 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
task = TASK_FOR_INVALID_JVM;
} else {
pingContainerHeartbeatHandler(containerId);
- AMContainerTask taskContext = pullTaskAttemptContext(containerId);
- if (taskContext.shouldDie()) {
- LOG.info("No more tasks for container with id : " + containerId
- + ". Asking it to die");
- task = TASK_FOR_INVALID_JVM; // i.e. ask the child to die.
+ task = getContainerTask(containerId);
+ if (task == null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("No task current assigned to Container with id: " + containerId);
+ }
} else {
- if (taskContext.getTask() == null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("No task currently assigned to Container with id: "
- + containerId);
- }
- } else {
- registerTaskAttempt(taskContext.getTask().getTaskAttemptID(),
- containerId);
- task = new ContainerTask(taskContext.getTask(), false,
- convertLocalResourceMap(taskContext.getAdditionalResources()),
- taskContext.getCredentials(), taskContext.haveCredentialsChanged());
context.getEventHandler().handle(
- new TaskAttemptEventStartedRemotely(taskContext.getTask()
+ new TaskAttemptEventStartedRemotely(task.getTaskSpec()
.getTaskAttemptID(), containerId, context
.getApplicationACLs()));
LOG.info("Container with id: " + containerId + " given task: "
- + taskContext.getTask().getTaskAttemptID());
- }
+ + task.getTaskSpec().getTaskAttemptID());
}
}
}
@@ -283,18 +272,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
return;
}
synchronized (containerInfo) {
- containerInfo.currentAttemptId = null;
+ containerInfo.amContainerTask = null;
attemptToInfoMap.remove(attemptId);
}
}
- public AMContainerTask pullTaskAttemptContext(ContainerId containerId) {
- AMContainerImpl container = (AMContainerImpl) context.getAllContainers()
- .get(containerId);
- return container.pullTaskContext();
- }
-
@Override
public void registerRunningContainer(ContainerId containerId) {
if (LOG.isDebugEnabled()) {
@@ -309,24 +292,27 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
@Override
- public void registerTaskAttempt(TezTaskAttemptID attemptId,
+ public void registerTaskAttempt(AMContainerTask amContainerTask,
ContainerId containerId) {
ContainerInfo containerInfo = registeredContainers.get(containerId);
if(containerInfo == null) {
throw new TezUncheckedException("Registering task attempt: "
- + attemptId + " to unknown container: " + containerId);
+ + amContainerTask.getTask().getTaskAttemptID() + " to unknown container: " + containerId);
}
synchronized (containerInfo) {
- if(containerInfo.currentAttemptId != null) {
+ if(containerInfo.amContainerTask != null) {
throw new TezUncheckedException("Registering task attempt: "
- + attemptId + " to container: " + containerId
- + " with existing assignment to: " + containerInfo.currentAttemptId);
+ + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ + " with existing assignment to: " + containerInfo.amContainerTask.getTask().getTaskAttemptID());
}
- containerInfo.currentAttemptId = attemptId;
- ContainerId containerIdFromMap = attemptToInfoMap.put(attemptId, containerId);
+ containerInfo.amContainerTask = amContainerTask;
+ containerInfo.taskPulled = false;
+
+ ContainerId containerIdFromMap =
+ attemptToInfoMap.put(amContainerTask.getTask().getTaskAttemptID(), containerId);
if(containerIdFromMap != null) {
throw new TezUncheckedException("Registering task attempt: "
- + attemptId + " to container: " + containerId
+ + amContainerTask.getTask().getTaskAttemptID() + " to container: " + containerId
+ " when already assigned to: " + containerIdFromMap);
}
}
@@ -368,6 +354,8 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
ContainerInfo containerInfo = registeredContainers.get(containerId);
if(containerInfo == null) {
+ LOG.warn("Received task heartbeat from unknown container with id: " + containerId +
+ ", asking it to die");
TezHeartbeatResponse response = new TezHeartbeatResponse();
response.setLastRequestId(requestId);
response.setShouldDie();
@@ -442,4 +430,35 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
}
return tlrs;
}
+
+ private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
+ ContainerTask containerTask = null;
+ ContainerInfo containerInfo = registeredContainers.get(containerId);
+ if (containerInfo == null) {
+ // This can happen if an unregisterTask comes in after we've done the initial checks for
+ // registered containers. (Race between getTask from the container, and a potential STOP_CONTAINER
+ // from somewhere within the AM)
+ // Implies that an un-registration has taken place and the container needs to be asked to die.
+ LOG.info("Container with id: " + containerId
+ + " is valid, but no longer registered, and will be killed");
+ containerTask = TASK_FOR_INVALID_JVM;
+ } else {
+ synchronized (containerInfo) {
+ if (containerInfo.amContainerTask != null) {
+ if (!containerInfo.taskPulled) {
+ containerInfo.taskPulled = true;
+ AMContainerTask amContainerTask = containerInfo.amContainerTask;
+ containerTask = new ContainerTask(amContainerTask.getTask(), false,
+ convertLocalResourceMap(amContainerTask.getAdditionalResources()),
+ amContainerTask.getCredentials(), amContainerTask.haveCredentialsChanged());
+ } else {
+ containerTask = null;
+ }
+ } else {
+ containerTask = null;
+ }
+ }
+ }
+ return containerTask;
+ }
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/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 e00ad3d..a6b403d 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
@@ -31,9 +31,6 @@ public interface AMContainer extends EventHandler<AMContainerEvent>{
public ContainerId getContainerId();
public Container getContainer();
public List<TezTaskAttemptID> getAllTaskAttempts();
- public TezTaskAttemptID getRunningTaskAttempt();
- public List<TezTaskAttemptID> getQueuedTaskAttempts();
+ public TezTaskAttemptID getCurrentTaskAttempt();
- // TODO Add a method to get the containers capabilities - to match taskAttempts.
-
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventType.java
index 582ec91..330ad57 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventType.java
@@ -28,9 +28,6 @@ public enum AMContainerEventType {
C_LAUNCHED,
C_LAUNCH_FAILED,
- //Producer: TAL: PULL_TA is a sync call.
- C_PULL_TA,
-
//Producer: Scheduler via TA
C_TA_SUCCEEDED, // maybe change this to C_TA_FINISHED with a status.
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/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 5c5a8c5..f72e62a 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
@@ -18,7 +18,6 @@
package org.apache.tez.dag.app.rm.container;
-import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.LinkedList;
@@ -62,8 +61,6 @@ import org.apache.tez.dag.history.events.ContainerStoppedEvent;
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.app.dag.event.TaskAttemptEventDiagnosticsUpdate;
-import org.apache.tez.runtime.api.impl.TaskSpec;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -87,12 +84,6 @@ public class AMContainerImpl implements AMContainer {
private final List<TezTaskAttemptID> completedAttempts =
new LinkedList<TezTaskAttemptID>();
- // TODO Maybe this should be pulled from the TaskAttempt.s
- private final Map<TezTaskAttemptID, TaskSpec> remoteTaskMap =
- new HashMap<TezTaskAttemptID, TaskSpec>();
-
- // TODO ?? Convert to list and hash.
-
private long idleTimeBetweenTasks = 0;
private long lastTaskFinishTime;
@@ -103,17 +94,8 @@ public class AMContainerImpl implements AMContainer {
// be modelled as a separate state.
private boolean nodeFailed = false;
- private TezTaskAttemptID pendingAttempt;
- private TezTaskAttemptID runningAttempt;
+ private TezTaskAttemptID currentAttempt;
private List<TezTaskAttemptID> failedAssignments;
- private TezTaskAttemptID pullAttempt;
-
- private AMContainerTask noAllocationContainerTask;
-
- private static final AMContainerTask NO_MORE_TASKS = new AMContainerTask(
- true, null, null, null, false);
- private static final AMContainerTask WAIT_TASK = new AMContainerTask(false,
- null, null, null, false);
private boolean inError = false;
@@ -160,26 +142,19 @@ public class AMContainerImpl implements AMContainer {
AMContainerState.COMPLETED,
EnumSet.of(AMContainerEventType.C_LAUNCHED,
AMContainerEventType.C_LAUNCH_FAILED,
- AMContainerEventType.C_PULL_TA,
AMContainerEventType.C_TA_SUCCEEDED,
AMContainerEventType.C_NM_STOP_SENT,
AMContainerEventType.C_NM_STOP_FAILED,
AMContainerEventType.C_TIMED_OUT), new ErrorTransition())
-
.addTransition(
AMContainerState.LAUNCHING,
- EnumSet.of(AMContainerState.LAUNCHING,
- AMContainerState.STOP_REQUESTED),
+ EnumSet.of(AMContainerState.LAUNCHING, AMContainerState.STOP_REQUESTED),
AMContainerEventType.C_ASSIGN_TA, new AssignTaskAttemptTransition())
- .addTransition(AMContainerState.LAUNCHING, AMContainerState.IDLE,
+ .addTransition(AMContainerState.LAUNCHING,
+ EnumSet.of(AMContainerState.IDLE, AMContainerState.RUNNING),
AMContainerEventType.C_LAUNCHED, new LaunchedTransition())
.addTransition(AMContainerState.LAUNCHING, AMContainerState.STOPPING,
AMContainerEventType.C_LAUNCH_FAILED, new LaunchFailedTransition())
- // TODO CREUSE : Maybe, consider sending back an attempt if the container
- // asks for one in this state. Waiting for a LAUNCHED event from the
- // NMComm may delay the task allocation.
- .addTransition(AMContainerState.LAUNCHING, AMContainerState.LAUNCHING,
- AMContainerEventType.C_PULL_TA)
// Is assuming the pullAttempt will be null.
.addTransition(AMContainerState.LAUNCHING, AMContainerState.COMPLETED,
AMContainerEventType.C_COMPLETED,
@@ -201,12 +176,9 @@ public class AMContainerImpl implements AMContainer {
new ErrorAtLaunchingTransition())
.addTransition(AMContainerState.IDLE,
- EnumSet.of(AMContainerState.IDLE, AMContainerState.STOP_REQUESTED),
+ EnumSet.of(AMContainerState.RUNNING, AMContainerState.STOP_REQUESTED),
AMContainerEventType.C_ASSIGN_TA,
- new AssignTaskAttemptAtIdleTransition())
- .addTransition(AMContainerState.IDLE,
- EnumSet.of(AMContainerState.RUNNING, AMContainerState.IDLE),
- AMContainerEventType.C_PULL_TA, new PullTAAtIdleTransition())
+ new AssignTaskAttemptTransition())
.addTransition(AMContainerState.IDLE, AMContainerState.COMPLETED,
AMContainerEventType.C_COMPLETED, new CompletedAtIdleTransition())
.addTransition(AMContainerState.IDLE, AMContainerState.STOP_REQUESTED,
@@ -230,8 +202,6 @@ public class AMContainerImpl implements AMContainer {
.addTransition(AMContainerState.RUNNING, AMContainerState.STOP_REQUESTED,
AMContainerEventType.C_ASSIGN_TA,
new AssignTaskAttemptAtRunningTransition())
- .addTransition(AMContainerState.RUNNING, AMContainerState.RUNNING,
- AMContainerEventType.C_PULL_TA)
.addTransition(AMContainerState.RUNNING, AMContainerState.IDLE,
AMContainerEventType.C_TA_SUCCEEDED,
new TASucceededAtRunningTransition())
@@ -259,9 +229,6 @@ public class AMContainerImpl implements AMContainer {
AMContainerState.STOP_REQUESTED, AMContainerEventType.C_ASSIGN_TA,
new AssignTAAtWindDownTransition())
.addTransition(AMContainerState.STOP_REQUESTED,
- AMContainerState.STOP_REQUESTED, AMContainerEventType.C_PULL_TA,
- new PullTAAfterStopTransition())
- .addTransition(AMContainerState.STOP_REQUESTED,
AMContainerState.COMPLETED, AMContainerEventType.C_COMPLETED,
new CompletedAtWindDownTransition())
.addTransition(AMContainerState.STOP_REQUESTED,
@@ -285,10 +252,10 @@ public class AMContainerImpl implements AMContainer {
AMContainerEventType.C_LAUNCH_REQUEST,
new ErrorAtNMStopRequestedTransition())
+
+
.addTransition(AMContainerState.STOPPING, AMContainerState.STOPPING,
AMContainerEventType.C_ASSIGN_TA, new AssignTAAtWindDownTransition())
- .addTransition(AMContainerState.STOPPING, AMContainerState.STOPPING,
- AMContainerEventType.C_PULL_TA, new PullTAAfterStopTransition())
// TODO This transition is wrong. Should be a noop / error.
.addTransition(AMContainerState.STOPPING, AMContainerState.COMPLETED,
AMContainerEventType.C_COMPLETED, new CompletedAtWindDownTransition())
@@ -311,8 +278,6 @@ public class AMContainerImpl implements AMContainer {
.addTransition(AMContainerState.COMPLETED, AMContainerState.COMPLETED,
AMContainerEventType.C_ASSIGN_TA, new AssignTAAtCompletedTransition())
.addTransition(AMContainerState.COMPLETED, AMContainerState.COMPLETED,
- AMContainerEventType.C_PULL_TA, new PullTAAfterStopTransition())
- .addTransition(AMContainerState.COMPLETED, AMContainerState.COMPLETED,
AMContainerEventType.C_NODE_FAILED, new NodeFailedBaseTransition())
.addTransition(
AMContainerState.COMPLETED,
@@ -348,7 +313,6 @@ public class AMContainerImpl implements AMContainer {
this.containerHeartbeatHandler = chh;
this.taskAttemptListener = tal;
this.failedAssignments = new LinkedList<TezTaskAttemptID>();
- this.noAllocationContainerTask = WAIT_TASK;
this.stateMachine = stateMachineFactory.make(this);
}
@@ -379,11 +343,8 @@ public class AMContainerImpl implements AMContainer {
List<TezTaskAttemptID> allAttempts = new LinkedList<TezTaskAttemptID>();
allAttempts.addAll(this.completedAttempts);
allAttempts.addAll(this.failedAssignments);
- if (this.pendingAttempt != null) {
- allAttempts.add(this.pendingAttempt);
- }
- if (this.runningAttempt != null) {
- allAttempts.add(this.runningAttempt);
+ if (this.currentAttempt != null) {
+ allAttempts.add(this.currentAttempt);
}
return allAttempts;
} finally {
@@ -392,24 +353,10 @@ public class AMContainerImpl implements AMContainer {
}
@Override
- public List<TezTaskAttemptID> getQueuedTaskAttempts() {
- readLock.lock();
- try {
- if (pendingAttempt != null) {
- return Collections.singletonList(this.pendingAttempt);
- } else {
- return Collections.emptyList();
- }
- } finally {
- readLock.unlock();
- }
- }
-
- @Override
- public TezTaskAttemptID getRunningTaskAttempt() {
+ public TezTaskAttemptID getCurrentTaskAttempt() {
readLock.lock();
try {
- return this.runningAttempt;
+ return this.currentAttempt;
} finally {
readLock.unlock();
}
@@ -453,32 +400,6 @@ public class AMContainerImpl implements AMContainer {
this.eventHandler.handle(event);
}
- // Push the TaskAttempt to the TAL, instead of the TAL pulling when a JVM asks
- // for a TaskAttempt.
- public AMContainerTask pullTaskContext() {
- this.writeLock.lock();
- try {
- this.handle(
- new AMContainerEvent(containerId, AMContainerEventType.C_PULL_TA));
- if (pullAttempt == null) {
- // As a later optimization, it should be possible for a running container to localize
- // additional resources before a task is assigned to the container.
- return noAllocationContainerTask;
- } else {
- // Avoid sending credentials if credentials have not changed.
- AMContainerTask amContainerTask = new AMContainerTask(false,
- remoteTaskMap.remove(pullAttempt), this.additionalLocalResources,
- this.credentialsChanged ? this.credentials : null, this.credentialsChanged);
- this.additionalLocalResources = null;
- this.credentialsChanged = false;
- this.pullAttempt = null;
- return amContainerTask;
- }
- } finally {
- this.writeLock.unlock();
- }
- }
-
//////////////////////////////////////////////////////////////////////////////
// Start of Transition Classes //
//////////////////////////////////////////////////////////////////////////////
@@ -591,14 +512,15 @@ public class AMContainerImpl implements AMContainer {
public AMContainerState transition(
AMContainerImpl container, AMContainerEvent cEvent) {
AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
- if (container.pendingAttempt != null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("AssignTaskAttempt at state " + container.getState() + ", attempt: " +
+ ((AMContainerEventAssignTA) cEvent).getRemoteTaskSpec());
+ }
+ if (container.currentAttempt != null) {
// This may include a couple of additional (harmless) unregister calls
// to the taskAttemptListener and containerHeartbeatHandler - in case
// of assign at any state prior to IDLE.
- container.handleExtraTAAssign(event, container.pendingAttempt);
- // TODO XXX: Verify that it's ok to send in a NM_STOP_REQUEST. The
- // NMCommunicator should be able to handle this. The STOP_REQUEST would
- // only go out after the START_REQUEST.
+ container.handleExtraTAAssign(event, container.currentAttempt);
return AMContainerState.STOP_REQUESTED;
}
@@ -609,7 +531,7 @@ public class AMContainerImpl implements AMContainer {
container.containerLocalResources, taskLocalResources);
// Register the additional resources back for this container.
container.containerLocalResources.putAll(container.additionalLocalResources);
- container.pendingAttempt = event.getTaskAttemptId();
+ container.currentAttempt = event.getTaskAttemptId();
if (LOG.isDebugEnabled()) {
LOG.debug("AssignTA: attempt: " + event.getRemoteTaskSpec());
LOG.debug("AdditionalLocalResources: " + container.additionalLocalResources);
@@ -625,17 +547,46 @@ public class AMContainerImpl implements AMContainer {
container.credentialsChanged = false;
}
- container.remoteTaskMap
- .put(event.getTaskAttemptId(), event.getRemoteTaskSpec());
- return container.getState();
+ if (container.lastTaskFinishTime != 0) {
+ // This effectively measures the time during which nothing was scheduler to execute on a container.
+ // The time from this point to the task actually being available to containers needs to be computed elsewhere.
+ long idleTimeDiff =
+ System.currentTimeMillis() - container.lastTaskFinishTime;
+ container.idleTimeBetweenTasks += idleTimeDiff;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Computing idle (scheduling) time for container: " +
+ container.getContainerId() + ", lastFinishTime: " +
+ container.lastTaskFinishTime + ", Incremented by: " +
+ idleTimeDiff);
+ }
+ }
+
+ LOG.info("Assigned taskAttempt + [" + container.currentAttempt +
+ "] to container: [" + container.getContainerId() + "]");
+ AMContainerTask amContainerTask = new AMContainerTask(
+ event.getRemoteTaskSpec(), container.additionalLocalResources,
+ container.credentialsChanged ? container.credentials : null, container.credentialsChanged);
+ container.registerAttemptWithListener(amContainerTask);
+ container.additionalLocalResources = null;
+ container.credentialsChanged = false;
+ if (container.getState() == AMContainerState.IDLE) {
+ return AMContainerState.RUNNING;
+ } else {
+ return container.getState();
+ }
}
}
- protected static class LaunchedTransition implements
- SingleArcTransition<AMContainerImpl, AMContainerEvent> {
+ protected static class LaunchedTransition
+ implements MultipleArcTransition<AMContainerImpl, AMContainerEvent, AMContainerState> {
@Override
- public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
+ public AMContainerState transition(AMContainerImpl container, AMContainerEvent cEvent) {
container.registerWithContainerListener();
+ if (container.currentAttempt != null) {
+ return AMContainerState.RUNNING;
+ } else {
+ return AMContainerState.IDLE;
+ }
}
}
@@ -643,11 +594,11 @@ public class AMContainerImpl implements AMContainer {
SingleArcTransition<AMContainerImpl, AMContainerEvent> {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
- if (container.pendingAttempt != null) {
+ if (container.currentAttempt != null) {
AMContainerEventLaunchFailed event = (AMContainerEventLaunchFailed) cEvent;
// for a properly setup cluster this should almost always be an app error
// need to differentiate between launch failed due to framework/cluster or app
- container.sendTerminatingToTaskAttempt(container.pendingAttempt,
+ container.sendTerminatingToTaskAttempt(container.currentAttempt,
event.getMessage(), TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED);
}
container.unregisterFromTAListener();
@@ -660,22 +611,22 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
AMContainerEventCompleted event = (AMContainerEventCompleted) cEvent;
- if (container.pendingAttempt != null) {
+ if (container.currentAttempt!= null) {
String errorMessage = getMessage(container, event);
if (event.isSystemAction()) {
- container.sendContainerTerminatedBySystemToTaskAttempt(container.pendingAttempt,
+ container.sendContainerTerminatedBySystemToTaskAttempt(container.currentAttempt,
errorMessage, event.getTerminationCause());
} else {
container
.sendTerminatedToTaskAttempt(
- container.pendingAttempt,
+ container.currentAttempt,
errorMessage,
// if termination cause is generic exited then replace with specific
(event.getTerminationCause() == TaskAttemptTerminationCause.CONTAINER_EXITED ?
TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED : event.getTerminationCause()));
}
- container.registerFailedAttempt(container.pendingAttempt);
- container.pendingAttempt = null;
+ container.registerFailedAttempt(container.currentAttempt);
+ container.currentAttempt = null;
LOG.warn(errorMessage);
}
container.containerLocalResources = null;
@@ -702,12 +653,12 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
- if (container.pendingAttempt != null) {
- container.sendTerminatingToTaskAttempt(container.pendingAttempt,
+ if (container.currentAttempt != null) {
+ container.sendTerminatingToTaskAttempt(container.currentAttempt,
getMessage(container, cEvent), TaskAttemptTerminationCause.CONTAINER_STOPPED);
}
container.unregisterFromTAListener();
- container.logStopped(container.pendingAttempt == null ?
+ container.logStopped(container.currentAttempt == null ?
ContainerExitStatus.SUCCESS
: ContainerExitStatus.INVALID);
container.sendStopRequestToNM();
@@ -742,18 +693,11 @@ public class AMContainerImpl implements AMContainer {
container.sendNodeFailureToTA(taId, errorMessage, TaskAttemptTerminationCause.NODE_FAILED);
}
- if (container.pendingAttempt != null) {
+ if (container.currentAttempt != null) {
// Will be null in COMPLETED state.
- container.sendNodeFailureToTA(container.pendingAttempt, errorMessage,
+ container.sendNodeFailureToTA(container.currentAttempt, errorMessage,
TaskAttemptTerminationCause.NODE_FAILED);
- container.sendTerminatingToTaskAttempt(container.pendingAttempt, errorMessage,
- TaskAttemptTerminationCause.NODE_FAILED);
- }
- if (container.runningAttempt != null) {
- // Will be null in COMPLETED state.
- container.sendNodeFailureToTA(container.runningAttempt, errorMessage,
- TaskAttemptTerminationCause.NODE_FAILED);
- container.sendTerminatingToTaskAttempt(container.runningAttempt, errorMessage,
+ container.sendTerminatingToTaskAttempt(container.currentAttempt, errorMessage,
TaskAttemptTerminationCause.NODE_FAILED);
}
container.logStopped(ContainerExitStatus.ABORTED);
@@ -775,61 +719,15 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- if (container.pendingAttempt != null) {
- container.sendTerminatingToTaskAttempt(container.pendingAttempt,
+ if (container.currentAttempt != null) {
+ container.sendTerminatingToTaskAttempt(container.currentAttempt,
"Container " + container.getContainerId() +
" hit an invalid transition - " + cEvent.getType() + " at " +
container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
}
container.logStopped(ContainerExitStatus.ABORTED);
- container.sendStopRequestToNM();
container.unregisterFromTAListener();
- }
- }
-
- protected static class AssignTaskAttemptAtIdleTransition
- extends AssignTaskAttemptTransition {
- @Override
- public AMContainerState transition(
- AMContainerImpl container, AMContainerEvent cEvent) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("AssignTAAtIdle: attempt: " +
- ((AMContainerEventAssignTA) cEvent).getRemoteTaskSpec());
- }
- return super.transition(container, cEvent);
- }
- }
-
- protected static class PullTAAtIdleTransition implements
- MultipleArcTransition<AMContainerImpl, AMContainerEvent, AMContainerState> {
-
- @Override
- public AMContainerState transition(
- AMContainerImpl container, AMContainerEvent cEvent) {
- if (container.pendingAttempt != null) {
- // This will be invoked as part of the PULL_REQUEST - so pullAttempt pullAttempt
- // should ideally only end up being populated during the duration of this call,
- // which is in a write lock. pullRequest() should move this to the running state.
- container.pullAttempt = container.pendingAttempt;
- container.runningAttempt = container.pendingAttempt;
- container.pendingAttempt = null;
- if (container.lastTaskFinishTime != 0) {
- long idleTimeDiff =
- System.currentTimeMillis() - container.lastTaskFinishTime;
- container.idleTimeBetweenTasks += idleTimeDiff;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Computing idle time for container: " +
- container.getContainerId() + ", lastFinishTime: " +
- container.lastTaskFinishTime + ", Incremented by: " +
- idleTimeDiff);
- }
- }
- LOG.info("Assigned taskAttempt + [" + container.runningAttempt +
- "] to container: [" + container.getContainerId() + "]");
- return AMContainerState.RUNNING;
- } else {
- return AMContainerState.IDLE;
- }
+ container.sendStopRequestToNM();
}
}
@@ -900,8 +798,8 @@ public class AMContainerImpl implements AMContainer {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
AMContainerEventAssignTA event = (AMContainerEventAssignTA) cEvent;
- container.unregisterAttemptFromListener(container.runningAttempt);
- container.handleExtraTAAssign(event, container.runningAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt);
+ container.handleExtraTAAssign(event, container.currentAttempt);
}
}
@@ -910,9 +808,9 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
container.lastTaskFinishTime = System.currentTimeMillis();
- container.completedAttempts.add(container.runningAttempt);
- container.unregisterAttemptFromListener(container.runningAttempt);
- container.runningAttempt = null;
+ container.completedAttempts.add(container.currentAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt);
+ container.currentAttempt = null;
}
}
@@ -922,15 +820,15 @@ public class AMContainerImpl implements AMContainer {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
AMContainerEventCompleted event = (AMContainerEventCompleted) cEvent;
if (event.isSystemAction()) {
- container.sendContainerTerminatedBySystemToTaskAttempt(container.runningAttempt,
+ container.sendContainerTerminatedBySystemToTaskAttempt(container.currentAttempt,
getMessage(container, event), event.getTerminationCause());
} else {
- container.sendTerminatedToTaskAttempt(container.runningAttempt,
+ container.sendTerminatedToTaskAttempt(container.currentAttempt,
getMessage(container, event), event.getTerminationCause());
}
- container.unregisterAttemptFromListener(container.runningAttempt);
- container.registerFailedAttempt(container.runningAttempt);
- container.runningAttempt = null;
+ container.unregisterAttemptFromListener(container.currentAttempt);
+ container.registerFailedAttempt(container.currentAttempt);
+ container.currentAttempt= null;
super.transition(container, cEvent);
}
}
@@ -938,11 +836,7 @@ public class AMContainerImpl implements AMContainer {
protected static class StopRequestAtRunningTransition
extends StopRequestAtIdleTransition {
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
-
- container.unregisterAttemptFromListener(container.runningAttempt);
- container.sendTerminatingToTaskAttempt(container.runningAttempt,
- " Container" + container.getContainerId() + " received a STOP_REQUEST",
- TaskAttemptTerminationCause.CONTAINER_STOPPED);
+ container.unregisterAttemptFromListener(container.currentAttempt);
super.transition(container, cEvent);
}
}
@@ -963,7 +857,7 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- container.unregisterAttemptFromListener(container.runningAttempt);
+ container.unregisterAttemptFromListener(container.currentAttempt);
}
}
@@ -972,8 +866,8 @@ public class AMContainerImpl implements AMContainer {
@Override
public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
super.transition(container, cEvent);
- container.unregisterAttemptFromListener(container.runningAttempt);
- container.sendTerminatingToTaskAttempt(container.runningAttempt,
+ container.unregisterAttemptFromListener(container.currentAttempt);
+ container.sendTerminatingToTaskAttempt(container.currentAttempt,
"Container " + container.getContainerId() +
" hit an invalid transition - " + cEvent.getType() + " at " +
container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
@@ -996,17 +890,6 @@ public class AMContainerImpl implements AMContainer {
}
}
- // Hack to some extent. This allocation should be done while entering one of
- // the post-running states, insetad of being a transition on the post stop
- // states.
- protected static class PullTAAfterStopTransition
- implements SingleArcTransition<AMContainerImpl, AMContainerEvent> {
- @Override
- public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
- container.noAllocationContainerTask = NO_MORE_TASKS;
- }
- }
-
protected static class CompletedAtWindDownTransition implements
SingleArcTransition<AMContainerImpl, AMContainerEvent> {
@Override
@@ -1017,17 +900,11 @@ public class AMContainerImpl implements AMContainer {
container.sendTerminatedToTaskAttempt(taId, diag,
TaskAttemptTerminationCause.CONTAINER_EXITED);
}
- if (container.pendingAttempt != null) {
- container.sendTerminatedToTaskAttempt(container.pendingAttempt, diag,
+ if (container.currentAttempt != null) {
+ container.sendTerminatedToTaskAttempt(container.currentAttempt, diag,
TaskAttemptTerminationCause.CONTAINER_EXITED);
- container.registerFailedAttempt(container.pendingAttempt);
- container.pendingAttempt = null;
- }
- if (container.runningAttempt != null) {
- container.sendTerminatedToTaskAttempt(container.runningAttempt, diag,
- TaskAttemptTerminationCause.CONTAINER_EXITED);
- container.registerFailedAttempt(container.runningAttempt);
- container.runningAttempt = null;
+ container.registerFailedAttempt(container.currentAttempt);
+ container.currentAttempt = null;
}
if (!(diag == null || diag.equals(""))) {
LOG.info("Container " + container.getContainerId()
@@ -1177,6 +1054,10 @@ public class AMContainerImpl implements AMContainer {
taskAttemptListener.unregisterTaskAttempt(attemptId);
}
+ protected void registerAttemptWithListener(AMContainerTask amContainerTask) {
+ taskAttemptListener.registerTaskAttempt(amContainerTask, this.containerId);
+ }
+
protected void registerWithTAListener() {
taskAttemptListener.registerRunningContainer(containerId);
}
@@ -1185,7 +1066,6 @@ public class AMContainerImpl implements AMContainer {
this.taskAttemptListener.unregisterRunningContainer(containerId);
}
-
protected void registerWithContainerListener() {
this.containerHeartbeatHandler.register(this.containerId);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
index efe2cca..89a434b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
@@ -20,30 +20,26 @@ package org.apache.tez.dag.app.rm.container;
import java.util.Map;
+import com.google.common.base.Preconditions;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.runtime.api.impl.TaskSpec;
public class AMContainerTask {
- private final boolean shouldDie;
private final Map<String, LocalResource> additionalResources;
private final TaskSpec tezTask;
private final Credentials credentials;
private final boolean credentialsChanged;
- public AMContainerTask(boolean shouldDie, TaskSpec tezTask,
+ public AMContainerTask(TaskSpec tezTask,
Map<String, LocalResource> additionalResources, Credentials credentials, boolean credentialsChanged) {
- this.shouldDie = shouldDie;
+ Preconditions.checkNotNull(tezTask, "TaskSpec cannot be null");
this.tezTask = tezTask;
this.additionalResources = additionalResources;
this.credentials = credentials;
this.credentialsChanged = credentialsChanged;
}
- public boolean shouldDie() {
- return this.shouldDie;
- }
-
public TaskSpec getTask() {
return this.tezTask;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
new file mode 100644
index 0000000..599a289
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -0,0 +1,182 @@
+/*
+ * 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.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.rm.container.AMContainer;
+import org.apache.tez.dag.app.rm.container.AMContainerMap;
+import org.apache.tez.dag.app.rm.container.AMContainerTask;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.junit.Test;
+
+public class TestTaskAttemptListenerImplTezDag {
+
+ @Test(timeout = 5000)
+ public void testGetTask() throws IOException {
+ ApplicationId appId = ApplicationId.newInstance(1000, 1);
+ AppContext appContext = mock(AppContext.class);
+ EventHandler eventHandler = mock(EventHandler.class);
+ DAG dag = mock(DAG.class);
+ AMContainerMap amContainerMap = mock(AMContainerMap.class);
+ Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
+ doReturn(eventHandler).when(appContext).getEventHandler();
+ doReturn(dag).when(appContext).getCurrentDAG();
+ doReturn(appAcls).when(appContext).getApplicationACLs();
+ doReturn(amContainerMap).when(appContext).getAllContainers();
+
+ TaskAttemptListenerImpTezDag taskAttemptListener =
+ new TaskAttemptListenerImplForTest(appContext, mock(TaskHeartbeatHandler.class),
+ mock(ContainerHeartbeatHandler.class), null);
+
+
+ TaskSpec taskSpec = mock(TaskSpec.class);
+ TezTaskAttemptID taskAttemptId = mock(TezTaskAttemptID.class);
+ doReturn(taskAttemptId).when(taskSpec).getTaskAttemptID();
+ AMContainerTask amContainerTask = new AMContainerTask(taskSpec, null, null, false);
+ ContainerTask containerTask = null;
+
+
+ ContainerId containerId1 = createContainerId(appId, 1);
+ doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
+ ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
+ containerTask = taskAttemptListener.getTask(containerContext1);
+ assertTrue(containerTask.shouldDie());
+
+
+ ContainerId containerId2 = createContainerId(appId, 2);
+ doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId2);
+ ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
+ taskAttemptListener.registerRunningContainer(containerId2);
+ containerTask = taskAttemptListener.getTask(containerContext2);
+ assertNull(containerTask);
+
+ // Valid task registered
+ taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
+ containerTask = taskAttemptListener.getTask(containerContext2);
+ assertFalse(containerTask.shouldDie());
+ assertEquals(taskSpec, containerTask.getTaskSpec());
+
+ // Task unregistered. Should respond to heartbeats
+ taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+ containerTask = taskAttemptListener.getTask(containerContext2);
+ assertNull(containerTask);
+
+ // Container unregistered. Should send a shouldDie = true
+ taskAttemptListener.unregisterRunningContainer(containerId2);
+ containerTask = taskAttemptListener.getTask(containerContext2);
+ assertTrue(containerTask.shouldDie());
+
+ ContainerId containerId3 = createContainerId(appId, 3);
+ ContainerContext containerContext3 = new ContainerContext(containerId3.toString());
+ taskAttemptListener.registerRunningContainer(containerId3);
+
+ // Register task to container3, followed by unregistering container 3 all together
+ TaskSpec taskSpec2 = mock(TaskSpec.class);
+ TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
+ doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
+ AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false);
+ taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
+ taskAttemptListener.unregisterRunningContainer(containerId3);
+ containerTask = taskAttemptListener.getTask(containerContext3);
+ assertTrue(containerTask.shouldDie());
+ }
+
+ @Test(timeout = 5000)
+ public void testGetTaskMultiplePulls() throws IOException {
+ ApplicationId appId = ApplicationId.newInstance(1000, 1);
+ AppContext appContext = mock(AppContext.class);
+ EventHandler eventHandler = mock(EventHandler.class);
+ DAG dag = mock(DAG.class);
+ AMContainerMap amContainerMap = mock(AMContainerMap.class);
+ Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
+ doReturn(eventHandler).when(appContext).getEventHandler();
+ doReturn(dag).when(appContext).getCurrentDAG();
+ doReturn(appAcls).when(appContext).getApplicationACLs();
+ doReturn(amContainerMap).when(appContext).getAllContainers();
+
+ TaskAttemptListenerImpTezDag taskAttemptListener =
+ new TaskAttemptListenerImplForTest(appContext, mock(TaskHeartbeatHandler.class),
+ mock(ContainerHeartbeatHandler.class), null);
+
+
+ TaskSpec taskSpec = mock(TaskSpec.class);
+ TezTaskAttemptID taskAttemptId = mock(TezTaskAttemptID.class);
+ doReturn(taskAttemptId).when(taskSpec).getTaskAttemptID();
+ AMContainerTask amContainerTask = new AMContainerTask(taskSpec, null, null, false);
+ ContainerTask containerTask = null;
+
+
+ ContainerId containerId1 = createContainerId(appId, 1);
+ doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
+ ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
+ taskAttemptListener.registerRunningContainer(containerId1);
+ containerTask = taskAttemptListener.getTask(containerContext1);
+ assertNull(containerTask);
+
+ // Register task
+ taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
+ containerTask = taskAttemptListener.getTask(containerContext1);
+ assertFalse(containerTask.shouldDie());
+ assertEquals(taskSpec, containerTask.getTaskSpec());
+
+ // Try pulling again - simulates re-use pull
+ containerTask = taskAttemptListener.getTask(containerContext1);
+ assertNull(containerTask);
+ }
+
+ private ContainerId createContainerId(ApplicationId applicationId, long containerIdLong) {
+ ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(applicationId, 1);
+ ContainerId containerId = ContainerId.newContainerId(appAttemptId, containerIdLong);
+ return containerId;
+ }
+
+ private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+
+ public TaskAttemptListenerImplForTest(AppContext context,
+ TaskHeartbeatHandler thh,
+ ContainerHeartbeatHandler chh,
+ JobTokenSecretManager jobTokenSecretManager) {
+ super(context, thh, chh, jobTokenSecretManager);
+ }
+
+ @Override
+ protected void startRpcServer() {
+ }
+
+ @Override
+ protected void stopRpcServer() {
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/fe39ede3/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 438c50d..22c0559 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
@@ -20,10 +20,11 @@ package org.apache.tez.dag.app.rm.container;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -103,40 +104,35 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.LAUNCHING);
// 1 Launch request.
wc.verifyCountAndGetOutgoingEvents(1);
+ verify(wc.tal).registerRunningContainer(wc.containerID);
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
// Assign task.
wc.assignTaskAttempt(wc.taskAttemptID);
wc.verifyState(AMContainerState.LAUNCHING);
wc.verifyNoOutgoingEvents();
- assertEquals(wc.taskAttemptID, wc.amContainer.getQueuedTaskAttempts()
- .get(0));
+ assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
// Container Launched
wc.containerLaunched();
- wc.verifyState(AMContainerState.IDLE);
- wc.verifyNoOutgoingEvents();
- assertEquals(wc.taskAttemptID, wc.amContainer.getQueuedTaskAttempts()
- .get(0));
- assertNull(wc.amContainer.getRunningTaskAttempt());
- verify(wc.tal).registerRunningContainer(wc.containerID);
- verify(wc.chh).register(wc.containerID);
-
- // Pull TA
- AMContainerTask pulledTask = wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.verifyNoOutgoingEvents();
- assertFalse(pulledTask.shouldDie());
- assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
- .getTaskAttemptID());
- assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
+ // Once for the previous NO_TASKS, one for the actual task.
+ verify(wc.chh).register(wc.containerID);
+ ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ assertEquals(1, argumentCaptor.getAllValues().size());
+ assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
+ // Attempt succeeded
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
- assertNull(wc.amContainer.getRunningTaskAttempt());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+ // Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
@@ -160,39 +156,98 @@ public class TestAMContainer {
wc.verifyState(AMContainerState.LAUNCHING);
// 1 Launch request.
wc.verifyCountAndGetOutgoingEvents(1);
+ verify(wc.tal).registerRunningContainer(wc.containerID);
// Container Launched
wc.containerLaunched();
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
- verify(wc.tal).registerRunningContainer(wc.containerID);
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
verify(wc.chh).register(wc.containerID);
// Assign task.
wc.assignTaskAttempt(wc.taskAttemptID);
+ wc.verifyState(AMContainerState.RUNNING);
+ wc.verifyNoOutgoingEvents();
+ assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
+ ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ assertEquals(1, argumentCaptor.getAllValues().size());
+ assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
+
+ wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
- assertEquals(wc.taskAttemptID, wc.amContainer.getQueuedTaskAttempts()
- .get(0));
- assertNull(wc.amContainer.getRunningTaskAttempt());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
+ verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
- // Pull TA
- AMContainerTask pulledTask = wc.pullTaskToRun();
+ wc.containerCompleted();
+ wc.verifyHistoryStopEvent();
+ wc.verifyState(AMContainerState.COMPLETED);
+ wc.verifyNoOutgoingEvents();
+ verify(wc.tal).unregisterRunningContainer(wc.containerID);
+ verify(wc.chh).unregister(wc.containerID);
+
+ assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
+ assertFalse(wc.amContainer.isInErrorState());
+ }
+
+ @Test (timeout=5000)
+ // Assign before launch.
+ public void tetMultipleSuccessfulTaskFlow() {
+ WrappedContainer wc = new WrappedContainer();
+
+ wc.verifyState(AMContainerState.ALLOCATED);
+
+ // Launch request.
+ wc.launchContainer();
+ wc.verifyState(AMContainerState.LAUNCHING);
+ // 1 Launch request.
+ wc.verifyCountAndGetOutgoingEvents(1);
+ verify(wc.tal).registerRunningContainer(wc.containerID);
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
+
+ // Assign task.
+ wc.assignTaskAttempt(wc.taskAttemptID);
+ wc.verifyState(AMContainerState.LAUNCHING);
+ wc.verifyNoOutgoingEvents();
+ assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
+
+ // Container Launched
+ wc.containerLaunched();
wc.verifyState(AMContainerState.RUNNING);
wc.verifyNoOutgoingEvents();
- assertFalse(pulledTask.shouldDie());
- assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
- .getTaskAttemptID());
- assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertEquals(wc.taskAttemptID, wc.amContainer.getCurrentTaskAttempt());
+ // Once for the previous NO_TASKS, one for the actual task.
+ verify(wc.chh).register(wc.containerID);
+ ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ assertEquals(1, argumentCaptor.getAllValues().size());
+ assertEquals(wc.taskAttemptID, argumentCaptor.getAllValues().get(0).getTask().getTaskAttemptID());
+ // Attempt succeeded
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.verifyState(AMContainerState.IDLE);
wc.verifyNoOutgoingEvents();
- assertNull(wc.amContainer.getRunningTaskAttempt());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
verify(wc.tal).unregisterTaskAttempt(wc.taskAttemptID);
+ TezTaskAttemptID taId2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
+ wc.assignTaskAttempt(taId2);
+ wc.verifyState(AMContainerState.RUNNING);
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ assertEquals(2, argumentCaptor.getAllValues().size());
+ assertEquals(taId2, argumentCaptor.getAllValues().get(1).getTask().getTaskAttemptID());
+
+ // Attempt succeeded
+ wc.taskAttemptSucceeded(taId2);
+ wc.verifyState(AMContainerState.IDLE);
+ wc.verifyNoOutgoingEvents();
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
+ verify(wc.tal).unregisterTaskAttempt(taId2);
+
+ // Container completed
wc.containerCompleted();
wc.verifyHistoryStopEvent();
wc.verifyState(AMContainerState.COMPLETED);
@@ -200,7 +255,7 @@ public class TestAMContainer {
verify(wc.tal).unregisterRunningContainer(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
- assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
+ assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
assertFalse(wc.amContainer.isInErrorState());
}
@@ -213,7 +268,6 @@ public class TestAMContainer {
wc.launchContainer();
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.stopRequest();
@@ -234,8 +288,7 @@ public class TestAMContainer {
verify(wc.tal).unregisterRunningContainer(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
- assertNull(wc.amContainer.getRunningTaskAttempt());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
assertFalse(wc.amContainer.isInErrorState());
}
@@ -249,7 +302,6 @@ public class TestAMContainer {
wc.launchContainer();
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.stopRequest();
@@ -273,22 +325,21 @@ public class TestAMContainer {
verify(wc.tal).unregisterRunningContainer(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
- assertNull(wc.amContainer.getRunningTaskAttempt());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
assertFalse(wc.amContainer.isInErrorState());
}
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
- public void testMultipleAllocationsAtIdle() {
+ public void testMultipleAllocationsWhileActive() {
WrappedContainer wc = new WrappedContainer();
List<Event> outgoingEvents;
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.verifyState(AMContainerState.IDLE);
+ wc.verifyState(AMContainerState.RUNNING);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
@@ -313,22 +364,20 @@ public class TestAMContainer {
TaskAttemptEventType.TA_CONTAINER_TERMINATED,
TaskAttemptEventType.TA_CONTAINER_TERMINATED);
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
}
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
- public void testAllocationAtRunning() {
+ public void testMultipleAllocationsAtLaunching() {
WrappedContainer wc = new WrappedContainer();
List<Event> outgoingEvents;
wc.launchContainer();
- wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
- wc.verifyState(AMContainerState.RUNNING);
+ wc.verifyState(AMContainerState.LAUNCHING);
+ verify(wc.tal).registerRunningContainer(wc.containerID);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
@@ -353,63 +402,56 @@ public class TestAMContainer {
TaskAttemptEventType.TA_CONTAINER_TERMINATED,
TaskAttemptEventType.TA_CONTAINER_TERMINATED);
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
}
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
- public void testMultipleAllocationsAtLaunching() {
+ public void testContainerTimedOutAtRunning() {
WrappedContainer wc = new WrappedContainer();
List<Event> outgoingEvents;
wc.launchContainer();
+ wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
- wc.verifyState(AMContainerState.LAUNCHING);
-
- TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
- wc.assignTaskAttempt(taID2);
+ wc.verifyState(AMContainerState.RUNNING);
+ wc.containerTimedOut();
wc.verifyState(AMContainerState.STOP_REQUESTED);
verify(wc.tal).unregisterRunningContainer(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
- // 1 for NM stop request. 2 TERMINATING to TaskAttempt.
- outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+ // 1 to TA, 1 for RM de-allocate.
+ outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
- NMCommunicatorEventType.CONTAINER_STOP_REQUEST,
TaskAttemptEventType.TA_CONTAINER_TERMINATING,
- TaskAttemptEventType.TA_CONTAINER_TERMINATING);
- assertTrue(wc.amContainer.isInErrorState());
+ NMCommunicatorEventType.CONTAINER_STOP_REQUEST);
+ // TODO Should this be an RM DE-ALLOCATE instead ?
- wc.nmStopSent();
wc.containerCompleted();
wc.verifyHistoryStopEvent();
- // 1 Inform scheduler. 2 TERMINATED to TaskAttempt.
- outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+ outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
- TaskAttemptEventType.TA_CONTAINER_TERMINATED,
TaskAttemptEventType.TA_CONTAINER_TERMINATED);
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
- assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
+ assertFalse(wc.amContainer.isInErrorState());
+
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
+ assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
}
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
- public void testContainerTimedOutAtRunning() {
+ public void testStopRequestedAtRunning() {
WrappedContainer wc = new WrappedContainer();
List<Event> outgoingEvents;
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
- wc.containerTimedOut();
+ wc.stopRequest();
wc.verifyState(AMContainerState.STOP_REQUESTED);
verify(wc.tal).unregisterRunningContainer(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
@@ -428,8 +470,7 @@ public class TestAMContainer {
assertFalse(wc.amContainer.isInErrorState());
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(1, wc.amContainer.getAllTaskAttempts().size());
}
@@ -442,6 +483,7 @@ public class TestAMContainer {
wc.launchContainer();
wc.assignTaskAttempt(wc.taskAttemptID);
wc.verifyState(AMContainerState.LAUNCHING);
+ verify(wc.tal).registerRunningContainer(wc.containerID);
wc.launchFailed();
wc.verifyState(AMContainerState.STOPPING);
verify(wc.tal).registerRunningContainer(wc.containerID);
@@ -454,7 +496,7 @@ public class TestAMContainer {
for (Event e : outgoingEvents) {
if (e.getType() == TaskAttemptEventType.TA_CONTAINER_TERMINATING) {
Assert.assertEquals(TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,
- ((TaskAttemptEventContainerTerminating)e).getTerminationCause());
+ ((TaskAttemptEventContainerTerminating)e).getTerminationCause());
}
}
@@ -462,7 +504,7 @@ public class TestAMContainer {
outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
verifyUnOrderedOutgoingEventTypes(outgoingEvents,
TaskAttemptEventType.TA_CONTAINER_TERMINATED);
-
+
// Valid transition. Container complete, but not with an error.
assertFalse(wc.amContainer.isInErrorState());
}
@@ -511,7 +553,7 @@ public class TestAMContainer {
assertFalse(wc.amContainer.isInErrorState());
}
-
+
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
public void testContainerCompletedAtLaunchingSpecificClusterError() {
@@ -520,7 +562,6 @@ public class TestAMContainer {
wc.launchContainer();
-
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
@@ -542,7 +583,7 @@ public class TestAMContainer {
assertFalse(wc.amContainer.isInErrorState());
}
-
+
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
public void testContainerCompletedAtLaunchingSpecificError() {
@@ -582,7 +623,6 @@ public class TestAMContainer {
wc.launchContainer();
- wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
wc.verifyState(AMContainerState.IDLE);
@@ -593,16 +633,10 @@ public class TestAMContainer {
verify(wc.chh).register(wc.containerID);
verify(wc.chh).unregister(wc.containerID);
- outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
- verifyUnOrderedOutgoingEventTypes(outgoingEvents,
- TaskAttemptEventType.TA_CONTAINER_TERMINATED);
+ wc.verifyCountAndGetOutgoingEvents(0);
assertFalse(wc.amContainer.isInErrorState());
- // Pending pull request. (Ideally, container should be dead at this point
- // and this event should not be generated. Network timeout on NM-RM heartbeat
- // can cause it to be genreated)
- wc.pullTaskToRun();
wc.verifyNoOutgoingEvents();
wc.verifyHistoryStopEvent();
@@ -619,7 +653,6 @@ public class TestAMContainer {
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.containerCompleted();
@@ -655,7 +688,6 @@ public class TestAMContainer {
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.containerCompleted(ContainerExitStatus.PREEMPTED, TaskAttemptTerminationCause.EXTERNAL_PREEMPTION);
@@ -693,7 +725,6 @@ public class TestAMContainer {
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.containerCompleted(ContainerExitStatus.INVALID, TaskAttemptTerminationCause.INTERNAL_PREEMPTION);
@@ -720,7 +751,7 @@ public class TestAMContainer {
assertFalse(wc.amContainer.isInErrorState());
}
-
+
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
public void testContainerDiskFailedAtRunning() {
@@ -731,7 +762,6 @@ public class TestAMContainer {
wc.assignTaskAttempt(wc.taskAttemptID);
wc.containerLaunched();
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.containerCompleted(ContainerExitStatus.DISKS_FAILED, TaskAttemptTerminationCause.NODE_DISK_ERROR);
@@ -758,7 +788,7 @@ public class TestAMContainer {
assertFalse(wc.amContainer.isInErrorState());
}
-
+
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
public void testTaskAssignedToCompletedContainer() {
@@ -768,7 +798,6 @@ public class TestAMContainer {
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.containerCompleted();
@@ -791,28 +820,16 @@ public class TestAMContainer {
assertTrue(wc.amContainer.isInErrorState());
}
- @Test (timeout=5000)
- public void testTaskPullAtLaunching() {
- WrappedContainer wc = new WrappedContainer();
-
- wc.launchContainer();
- AMContainerTask pulledTask = wc.pullTaskToRun();
- wc.verifyState(AMContainerState.LAUNCHING);
- wc.verifyNoOutgoingEvents();
- assertFalse(pulledTask.shouldDie());
- assertNull(pulledTask.getTask());
- }
-
@SuppressWarnings("rawtypes")
@Test (timeout=5000)
- public void testNodeFailedAtIdle() {
+ public void testNodeFailedAtRunning() {
WrappedContainer wc = new WrappedContainer();
List<Event> outgoingEvents;
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.verifyState(AMContainerState.IDLE);
+ wc.verifyState(AMContainerState.RUNNING);
wc.nodeFailed();
// Expecting a complete event from the RM
@@ -848,13 +865,11 @@ public class TestAMContainer {
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
wc.verifyState(AMContainerState.IDLE);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(taID2);
wc.verifyState(AMContainerState.IDLE);
@@ -880,8 +895,7 @@ public class TestAMContainer {
wc.verifyNoOutgoingEvents();
wc.verifyHistoryStopEvent();
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
}
@@ -894,12 +908,10 @@ public class TestAMContainer {
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
- wc.pullTaskToRun();
wc.verifyState(AMContainerState.RUNNING);
wc.nodeFailed();
@@ -926,8 +938,7 @@ public class TestAMContainer {
TaskAttemptEventType.TA_CONTAINER_TERMINATED);
assertFalse(wc.amContainer.isInErrorState());
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
}
@@ -940,12 +951,10 @@ public class TestAMContainer {
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(taID2);
wc.stopRequest();
wc.nmStopSent();
@@ -958,8 +967,7 @@ public class TestAMContainer {
TaskAttemptEventType.TA_NODE_FAILED,
TaskAttemptEventType.TA_NODE_FAILED);
- assertNull(wc.amContainer.getRunningTaskAttempt());
- assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
+ assertNull(wc.amContainer.getCurrentTaskAttempt());
assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
}
@@ -970,12 +978,10 @@ public class TestAMContainer {
wc.launchContainer();
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(wc.taskAttemptID);
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2);
- wc.pullTaskToRun();
wc.taskAttemptSucceeded(taID2);
wc.stopRequest();
wc.nmStopSent();
@@ -988,7 +994,7 @@ public class TestAMContainer {
wc.verifyNoOutgoingEvents();
wc.verifyHistoryStopEvent();
}
-
+
@Test (timeout=5000)
public void testLocalResourceAddition() {
WrappedContainer wc = new WrappedContainer();
@@ -1003,7 +1009,9 @@ public class TestAMContainer {
wc.launchContainer(initialResources, new Credentials());
wc.containerLaunched();
wc.assignTaskAttempt(wc.taskAttemptID);
- AMContainerTask task1 = wc.pullTaskToRun();
+ ArgumentCaptor<AMContainerTask> argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ AMContainerTask task1 = argumentCaptor.getAllValues().get(0);
assertEquals(0, task1.getAdditionalResources().size());
wc.taskAttemptSucceeded(wc.taskAttemptID);
@@ -1014,7 +1022,9 @@ public class TestAMContainer {
TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2);
wc.assignTaskAttempt(taID2, additionalResources, new Credentials());
- AMContainerTask task2 = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ AMContainerTask task2 = argumentCaptor.getAllValues().get(1);
Map<String, LocalResource> pullTaskAdditionalResources = task2.getAdditionalResources();
assertEquals(2, pullTaskAdditionalResources.size());
pullTaskAdditionalResources.remove(rsrc2);
@@ -1035,7 +1045,9 @@ public class TestAMContainer {
// task is not asked to re-localize again.
TezTaskAttemptID taID3 = TezTaskAttemptID.getInstance(wc.taskID, 3);
wc.assignTaskAttempt(taID3, new HashMap<String, LocalResource>(), new Credentials());
- AMContainerTask task3 = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ AMContainerTask task3 = argumentCaptor.getAllValues().get(2);
assertEquals(0, task3.getAdditionalResources().size());
wc.taskAttemptSucceeded(taID3);
@@ -1063,66 +1075,79 @@ public class TestAMContainer {
TezTaskAttemptID attempt22 = TezTaskAttemptID.getInstance(taskID2, 300);
TezTaskAttemptID attempt31 = TezTaskAttemptID.getInstance(taskID3, 200);
TezTaskAttemptID attempt32 = TezTaskAttemptID.getInstance(taskID3, 300);
-
+
Map<String, LocalResource> LRs = new HashMap<String, LocalResource>();
AMContainerTask fetchedTask = null;
-
+ ArgumentCaptor<AMContainerTask> argumentCaptor = null;
+
Token<TokenIdentifier> amGenToken = mock(Token.class);
Token<TokenIdentifier> token1 = mock(Token.class);
Token<TokenIdentifier> token3 = mock(Token.class);
-
+
Credentials containerCredentials = new Credentials();
TokenCache.setSessionToken(amGenToken, containerCredentials);
Text token1Name = new Text("tokenDag1");
Text token3Name = new Text("tokenDag3");
-
+
Credentials dag1Credentials = new Credentials();
dag1Credentials.addToken(new Text(token1Name), token1);
Credentials dag3Credentials = new Credentials();
dag3Credentials.addToken(new Text(token3Name), token3);
-
+
wc.launchContainer(new HashMap<String, LocalResource>(), containerCredentials);
wc.containerLaunched();
- wc.assignTaskAttempt(attempt11, LRs , dag1Credentials);
- fetchedTask = wc.pullTaskToRun();
+ wc.assignTaskAttempt(attempt11, LRs, dag1Credentials);
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(1)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(0);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNotNull(fetchedTask.getCredentials());
assertNotNull(fetchedTask.getCredentials().getToken(token1Name));
wc.taskAttemptSucceeded(attempt11);
-
+
wc.assignTaskAttempt(attempt12, LRs, dag1Credentials);
- fetchedTask = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(2)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(1);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
wc.taskAttemptSucceeded(attempt12);
-
+
// Move to running a second DAG, with no credentials.
wc.setNewDAGID(dagID2);
wc.assignTaskAttempt(attempt21, LRs, null);
- fetchedTask = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(3)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(2);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
wc.taskAttemptSucceeded(attempt21);
-
+
wc.assignTaskAttempt(attempt22, LRs, null);
- fetchedTask = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(4)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(3);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
wc.taskAttemptSucceeded(attempt22);
-
+
// Move to running a third DAG, with Credentials this time
wc.setNewDAGID(dagID3);
wc.assignTaskAttempt(attempt31, LRs , dag3Credentials);
- fetchedTask = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(5)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(4);
assertTrue(fetchedTask.haveCredentialsChanged());
assertNotNull(fetchedTask.getCredentials());
assertNotNull(fetchedTask.getCredentials().getToken(token3Name));
assertNull(fetchedTask.getCredentials().getToken(token1Name));
wc.taskAttemptSucceeded(attempt31);
-
+
wc.assignTaskAttempt(attempt32, LRs, dag1Credentials);
- fetchedTask = wc.pullTaskToRun();
+ argumentCaptor = ArgumentCaptor.forClass(AMContainerTask.class);
+ verify(wc.tal, times(6)).registerTaskAttempt(argumentCaptor.capture(), eq(wc.containerID));
+ fetchedTask = argumentCaptor.getAllValues().get(5);
assertFalse(fetchedTask.haveCredentialsChanged());
assertNull(fetchedTask.getCredentials());
wc.taskAttemptSucceeded(attempt32);
@@ -1261,15 +1286,11 @@ public class TestAMContainer {
public void assignTaskAttempt(TezTaskAttemptID taID,
Map<String, LocalResource> additionalResources, Credentials credentials) {
reset(eventHandler);
+ doReturn(taID).when(taskSpec).getTaskAttemptID();
amContainer.handle(new AMContainerEventAssignTA(containerID, taID, taskSpec,
additionalResources, credentials));
}
- public AMContainerTask pullTaskToRun() {
- reset(eventHandler);
- return amContainer.pullTaskContext();
- }
-
public void containerLaunched() {
reset(eventHandler);
amContainer.handle(new AMContainerEventLaunched(containerID));
[05/17] tez git commit: TEZ-2036. OneToOneEdgeManager should enforce
that source and destination tasks have same number (bikas)
Posted by ss...@apache.org.
TEZ-2036. OneToOneEdgeManager should enforce that source and destination tasks have same number (bikas)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/5f3a9a3e
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/5f3a9a3e
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/5f3a9a3e
Branch: refs/heads/TEZ-2003
Commit: 5f3a9a3edb51f029bb3754b67e3b91748c993474
Parents: 236b0e2
Author: Bikas Saha <bi...@apache.org>
Authored: Thu Feb 5 18:50:42 2015 -0800
Committer: Bikas Saha <bi...@apache.org>
Committed: Thu Feb 5 18:50:42 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 2 +
.../dag/app/dag/impl/OneToOneEdgeManager.java | 8 ++++
.../apache/tez/dag/app/dag/impl/TestEdge.java | 39 +++++++++++++++++++-
3 files changed, 47 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/5f3a9a3e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 079ea73..2ab6c0a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -178,6 +178,8 @@ TEZ-UI CHANGES (TEZ-8):
Release 0.5.4: Unreleased
ALL CHANGES:
+ TEZ-2036. OneToOneEdgeManager should enforce that source and destination
+ tasks have same number
TEZ-1895. Vertex reRunning should decrease successfulMembers of VertexGroupInfo.
TEZ-2020. For 1-1 edge vertex configured event may be sent incorrectly
TEZ-2015. VertexImpl.doneReconfiguringVertex() should check other criteria
http://git-wip-us.apache.org/repos/asf/tez/blob/5f3a9a3e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index ced481d..11a6483 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -27,6 +27,8 @@ import org.apache.tez.dag.api.EdgeManagerPluginContext;
import org.apache.tez.runtime.api.events.DataMovementEvent;
import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import com.google.common.base.Preconditions;
+
public class OneToOneEdgeManager extends EdgeManagerPlugin {
List<Integer> destinationInputIndices =
@@ -55,6 +57,12 @@ public class OneToOneEdgeManager extends EdgeManagerPlugin {
public void routeDataMovementEventToDestination(DataMovementEvent event,
int sourceTaskIndex, int sourceOutputIndex,
Map<Integer, List<Integer>> destinationTaskAndInputIndices) {
+ // by the time routing is initiated all task counts must be determined and stable
+ Preconditions.checkState(getContext().getSourceVertexNumTasks() == getContext()
+ .getDestinationVertexNumTasks(), "1-1 source and destination task counts must match."
+ + " Destination: " + getContext().getDestinationVertexName() + " tasks: "
+ + getContext().getDestinationVertexNumTasks() + " Source: "
+ + getContext().getSourceVertexName() + " tasks: " + getContext().getSourceVertexNumTasks());
destinationTaskAndInputIndices.put(sourceTaskIndex, destinationInputIndices);
}
http://git-wip-us.apache.org/repos/asf/tez/blob/5f3a9a3e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
index 31c8064..a607f1b 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
@@ -26,14 +26,18 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collection;
import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
import java.util.Map.Entry;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.api.EdgeManagerPluginContext;
import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
@@ -51,13 +55,45 @@ import org.apache.tez.runtime.api.events.DataMovementEvent;
import org.apache.tez.runtime.api.impl.EventMetaData;
import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Assert;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
+import com.google.common.collect.Maps;
+
public class TestEdge {
+ @Test (timeout = 5000)
+ public void testOneToOneEdgeManager() {
+ EdgeManagerPluginContext mockContext = mock(EdgeManagerPluginContext.class);
+ when(mockContext.getSourceVertexName()).thenReturn("Source");
+ when(mockContext.getDestinationVertexName()).thenReturn("Destination");
+ when(mockContext.getSourceVertexNumTasks()).thenReturn(3);
+ OneToOneEdgeManager manager = new OneToOneEdgeManager(mockContext);
+ manager.initialize();
+ Map<Integer, List<Integer>> destinationTaskAndInputIndices = Maps.newHashMap();
+ DataMovementEvent event = DataMovementEvent.create(1, null);
+
+ // fail when source and destination are inconsistent
+ when(mockContext.getDestinationVertexNumTasks()).thenReturn(4);
+ try {
+ manager.routeDataMovementEventToDestination(event, 1, 1, destinationTaskAndInputIndices);
+ Assert.fail();
+ } catch (IllegalStateException e) {
+ Assert.assertTrue(e.getMessage().contains("1-1 source and destination task counts must match"));
+ }
+
+ // now make it consistent
+ when(mockContext.getDestinationVertexNumTasks()).thenReturn(3);
+ manager.routeDataMovementEventToDestination(event, 1, 1, destinationTaskAndInputIndices);
+ Assert.assertEquals(1, destinationTaskAndInputIndices.size());
+ Assert.assertEquals(1, destinationTaskAndInputIndices.entrySet().iterator().next().getKey()
+ .intValue());
+ Assert.assertEquals(0, destinationTaskAndInputIndices.entrySet().iterator().next().getValue()
+ .get(0).intValue());
+ }
- @SuppressWarnings({ "rawtypes", "unchecked" })
+ @SuppressWarnings({ "rawtypes" })
@Test (timeout = 5000)
public void testCompositeEventHandling() throws AMUserCodeException {
EventHandler eventHandler = mock(EventHandler.class);
@@ -107,7 +143,6 @@ public class TestEdge {
verifyEvents(srcTAID, destTasks);
}
- @SuppressWarnings("rawtypes")
private void verifyEvents(TezTaskAttemptID srcTAID, LinkedHashMap<TezTaskID, Task> destTasks) {
int count = 0;
[06/17] tez git commit: TEZ-2043. Tez UI: add progress info from am
webservice to dag and vertex views. (Prakash Ramachandran via hitesh)
Posted by ss...@apache.org.
TEZ-2043. Tez UI: add progress info from am webservice to dag and vertex views. (Prakash Ramachandran via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/d51103db
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/d51103db
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/d51103db
Branch: refs/heads/TEZ-2003
Commit: d51103db3ae383b0e6e43577dd1c97f49f5f3bfc
Parents: 5f3a9a3
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Feb 6 09:28:18 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Feb 6 09:28:18 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
tez-ui/src/main/webapp/app/index.html | 1 +
tez-ui/src/main/webapp/app/scripts/app.js | 44 ++++++++++++++
.../app/scripts/controllers/dag_controller.js | 18 +++++-
.../scripts/controllers/dag_index_controller.js | 8 +++
.../app/scripts/controllers/dag_vertices.js | 46 +++++++++++++-
.../app/scripts/controllers/dags_controller.js | 23 ++++++-
.../controllers/tez-app-dags-controller.js | 48 ++++++++++++++-
.../scripts/controllers/vertex_controller.js | 21 ++++++-
.../controllers/vertex_index_controller.js | 8 +++
.../main/webapp/app/scripts/default-configs.js | 5 +-
.../main/webapp/app/scripts/helpers/number.js | 10 ++++
.../app/scripts/mixins/paginated_content.js | 15 ++++-
.../app/scripts/models/TimelineRestAdapter.js | 4 ++
.../src/main/webapp/app/scripts/models/dag.js | 23 +++++++
.../src/main/webapp/app/templates/dag/index.hbs | 63 ++++++++++----------
.../main/webapp/app/templates/vertex/index.hbs | 1 +
17 files changed, 295 insertions(+), 44 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 2ab6c0a..0732004 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/index.html
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/index.html b/tez-ui/src/main/webapp/app/index.html
index 73e991b..4fe7e3d 100644
--- a/tez-ui/src/main/webapp/app/index.html
+++ b/tez-ui/src/main/webapp/app/index.html
@@ -53,6 +53,7 @@
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-modal.min.js"></script>
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-nav.min.js"></script>
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-items-action-bar.min.js"></script>
+ <script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-badge.min.js"></script>
<script src="bower_components/antiscroll/antiscroll.js"></script>
<script src="bower_components/jquery-mousewheel/jquery.mousewheel.js"></script>
<script src="bower_components/ember-table/dist/ember-table.js"></script>
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/app.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/app.js b/tez-ui/src/main/webapp/app/scripts/app.js
index 5beb815..4fc461a 100644
--- a/tez-ui/src/main/webapp/app/scripts/app.js
+++ b/tez-ui/src/main/webapp/app/scripts/app.js
@@ -107,6 +107,50 @@ App.ready = function () {
});
}
});
+
+ App.AMInfoAdapter = DS.RESTAdapter.extend({
+ ajax: function(url, method, hash) {
+ hash = hash || {}; // hash may be undefined
+ if (hash && hash.data && hash.data.__app_id__) {
+ url = url.replace('__app_id__', hash.data.__app_id__);
+ delete hash.data['__app_id__'];
+ }
+ hash.crossDomain = true;
+ hash.xhrFields = {withCredentials: true};
+ return this._super(url, method, hash);
+ },
+ host: App.env.RMWebUrl,
+ namespace: App.Configs.restNamespace.aminfo,
+ });
+
+ App.DagProgressAdapter = App.AMInfoAdapter.extend({
+ buildURL: function(type, id, record) {
+ var url = this._super(type, undefined, record);
+ return url.replace('__app_id__', record.get('appId'))
+ .fmt(record.get('dagIdx'));
+ },
+ pathForType: function() {
+ return 'dagProgress?dagID=%@';
+ }
+ });
+
+ App.VertexProgressAdapter = App.AMInfoAdapter.extend({
+ findQuery: function(store, type, query) {
+ var record = query.metadata;
+ delete query.metadata;
+ return this.ajax(
+ this.buildURL(Ember.String.pluralize(type.typeKey),
+ record.vertexIds, Em.Object.create(record)), 'GET', { data: query});
+ },
+ buildURL: function(type, id, record) {
+ var url = this._super(type, undefined, record);
+ return url.replace('__app_id__', record.get('appId'))
+ .fmt(record.get('dagIdx'), id);
+ },
+ pathForType: function(typeName) {
+ return typeName + '?dagID=%@&vertexID=%@';
+ }
+ });
};
/* Order and include */
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
index 3375e33..1fdf053 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
@@ -24,7 +24,23 @@ App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
loading: true,
updateLoading: function() {
- this.set('loading', false);
+ var that = this;
+ var loaders = [];
+ if (this.get('status') === 'RUNNING') {
+ // update the progress info if available. this need not block the UI
+ var aminfoLoader = that.store.find('dagProgress', that.get('id'), {
+ appId: that.get('applicationId'),
+ dagIdx: that.get('idx')
+ }).then(function(dagProgressInfo) {
+ that.set('progress', dagProgressInfo.get('progress'));
+ }).catch(function (error) {
+ Em.Logger.error("Failed to fetch dagProgress")
+ });
+ loaders.push(aminfoLoader);
+ }
+ Em.RSVP.allSettled(loaders).then(function(){
+ that.set('loading', false);
+ });
}.observes('content'),
childDisplayViews: [
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
index a7eb3ea..fed28c7 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
@@ -24,6 +24,14 @@ App.DagIndexController = Em.ObjectController.extend({
return App.Helpers.misc.getStatusClassForEntity(this.get('model'));
}.property('id', 'status', 'counterGroups'),
+ progressStr: function() {
+ var pct;
+ if (Ember.typeOf(this.get('progress')) === 'number') {
+ pct = App.Helpers.number.fractionToPercentage(this.get('progress'));
+ }
+ return pct;
+ }.property('id', 'status', 'progress'),
+
totalTasks: function() {
return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'),
this.get('id'), 'org.apache.tez.common.counters.DAGCounter', 'TOTAL_LAUNCHED_TASKS')
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
index 7cc5fbf..c26f83e 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
@@ -44,6 +44,40 @@ App.DagVerticesController = Em.ObjectController.extend(App.PaginatedContentMixin
this.setFiltersAndLoadEntities(filters);
},
+ loadAdditional: function() {
+ var defer = Em.RSVP.defer();
+
+ var that = this;
+ vertices = this.get('entities');
+ var runningVerticesIdx = vertices
+ .filterBy('status', 'RUNNING')
+ .map(function(item) {
+ return item.get('id').split('_').splice(-1).pop();
+ });
+ if (runningVerticesIdx.length > 0) {
+ this.store.findQuery('vertexProgress', {
+ metadata: {
+ appId: that.get('applicationId'),
+ dagIdx: that.get('idx'),
+ vertexIds: runningVerticesIdx.join(',')
+ }
+ }).then(function(vertexProgressInfo) {
+ vertexProgressInfo.forEach(function(item) {
+ var model = vertices.findBy('id', item.get('id')) || Em.Object.create();
+ model.set('progress', item.get('progress'));
+ });
+ }).catch(function(error) {
+ Em.Logger.debug("failed to fetch vertex progress")
+ }).finally(function(){
+ defer.resolve();
+ })
+ } else {
+ defer.resolve();
+ }
+
+ return defer.promise;
+ },
+
actions : {
filterUpdated: function(filterID, value) {
// any validations required goes here.
@@ -129,12 +163,18 @@ App.DagVerticesController = Em.ObjectController.extend(App.PaginatedContentMixin
template: Em.Handlebars.compile(
'<span class="ember-table-content"> \
<i {{bind-attr class=":task-status view.cellContent.statusIcon"}}></i>\
- {{view.cellContent.status}}</span>')
+ {{view.cellContent.status}}\
+ {{#if view.cellContent.progress}} {{bs-badge content=view.cellContent.progress}}{{/if}}</span>')
}),
getCellContent: function(row) {
+ var pct;
+ if (Ember.typeOf(row.get('progress')) === 'number') {
+ pct = App.Helpers.number.fractionToPercentage(row.get('progress'));
+ }
return {
status: row.get('status'),
- statusIcon: App.Helpers.misc.getStatusClassForEntity(row)
+ statusIcon: App.Helpers.misc.getStatusClassForEntity(row),
+ progress: pct
};
}
},
@@ -180,4 +220,4 @@ App.DagVerticesController = Em.ObjectController.extend(App.PaginatedContentMixin
);
}.property(),
-});
\ No newline at end of file
+});
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
index 55f9d26..9d1a243 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
@@ -87,6 +87,19 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
dag.set('tezApp', app);
});
loaders.push(fetcher);
+ if (dag.get('status') === 'RUNNING') {
+ amInfoFetcher = store.find('dagProgress', dag.get('id'), {
+ appId: dag.get('applicationId'),
+ dagIdx: dag.get('idx')
+ })
+ .then(function(dagProgressInfo) {
+ dag.set('progress', dagProgressInfo.get('progress'));
+ })
+ .catch(function(error) {
+ Em.Logger.error('Failed to fetch dagProgress' + error);
+ });
+ loaders.push(amInfoFetcher);
+ }
}
});
Em.RSVP.allSettled(loaders).then(function(){
@@ -154,12 +167,18 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
template: Em.Handlebars.compile(
'<span class="ember-table-content"> \
<i {{bind-attr class=":task-status view.cellContent.statusIcon"}}></i>\
- {{view.cellContent.status}}</span>')
+ {{view.cellContent.status}}\
+ {{#if view.cellContent.progress}} {{bs-badge content=view.cellContent.progress}}{{/if}}</span>')
}),
getCellContent: function(row) {
+ var pct;
+ if (Ember.typeOf(row.get('progress')) === 'number') {
+ pct = App.Helpers.number.fractionToPercentage(row.get('progress'));
+ }
return {
status: row.get('status'),
- statusIcon: App.Helpers.misc.getStatusClassForEntity(row)
+ statusIcon: App.Helpers.misc.getStatusClassForEntity(row),
+ progress: pct
};
}
},
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
index b9054b1..438ec29 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
@@ -47,6 +47,44 @@ App.TezAppDagsController = Em.ObjectController.extend(App.PaginatedContentMixin,
this.setFiltersAndLoadEntities(filters);
},
+ loadEntities: function() {
+ var that = this;
+ var childEntityType = this.get('childEntityType');
+ var defaultErrMsg = 'Error while loading %@. could not connect to %@'
+ .fmt(childEntityType, App.env.timelineBaseUrl);
+
+
+ that.set('loading', true);
+
+ this.get('store').unloadAll(childEntityType);
+ this.get('store').findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
+ that.set('entities', entities);
+
+ var loaders = [];
+ entities.forEach(function (dag) {
+ if (dag.get('status') === 'RUNNING') {
+ amInfoFetcher = that.store.find('dagProgress', dag.get('id'), {
+ appId: dag.get('applicationId'),
+ dagIdx: dag.get('idx')
+ }).then(function(dagProgressInfo) {
+ dag.set('progress', dagProgressInfo.get('progress'));
+ }).catch(function(error) {
+ Em.Logger.error('Failed to fetch dagProgress' + error);
+ });
+ loaders.push(amInfoFetcher);
+ }
+ });
+ Em.RSVP.allSettled(loaders).then(function(){
+ that.set('loading', false);
+ });
+ }).catch(function(error){
+ Em.Logger.error(error);
+ var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+ var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+ App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+ });
+ },
+
actions : {
filterUpdated: function(filterID, value) {
// any validations required goes here.
@@ -97,12 +135,18 @@ App.TezAppDagsController = Em.ObjectController.extend(App.PaginatedContentMixin,
template: Em.Handlebars.compile(
'<span class="ember-table-content"> \
<i {{bind-attr class=":task-status view.cellContent.statusIcon"}}></i>\
- {{view.cellContent.status}}</span>')
+ {{view.cellContent.status}}\
+ {{#if view.cellContent.progress}} {{bs-badge content=view.cellContent.progress}}{{/if}}</span>')
}),
getCellContent: function(row) {
+ var pct;
+ if (Ember.typeOf(row.get('progress')) === 'number') {
+ pct = App.Helpers.number.fractionToPercentage(row.get('progress'));
+ }
return {
status: row.get('status'),
- statusIcon: App.Helpers.misc.getStatusClassForEntity(row)
+ statusIcon: App.Helpers.misc.getStatusClassForEntity(row),
+ progress: pct
};
}
},
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
index 1c20b3e..e5e74e5 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
@@ -24,7 +24,26 @@ App.VertexController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
loading: true,
updateLoading: function() {
- this.set('loading', false);
+ var loaders = [],
+ that = this;
+
+ if (this.get('status') == 'RUNNING') {
+ var vertexIdx = that.get('id').split('_').splice(-1).pop();
+ var progressLoader = this.store.find('vertexProgress', vertexIdx, {
+ appId: that.get('applicationId'),
+ dagIdx: that.get('dagIdx')
+ }).then(function(vertexProgressInfo) {
+ if (vertexProgressInfo) {
+ that.set('progress', vertexProgressInfo.get('progress'));
+ }
+ }).catch(function(error) {
+ Em.Logger.error("Failed to fetch vertexProgress" + error)
+ });
+ loaders.push(progressLoader);
+ }
+ Em.RSVP.allSettled(loaders).then(function(){
+ that.set('loading', false);
+ });
}.observes('content'),
childDisplayViews: [
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
index a7b16c3..8c53db2 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
@@ -24,6 +24,14 @@ App.VertexIndexController = Em.ObjectController.extend({
return App.Helpers.misc.getStatusClassForEntity(this.get('model'));
}.property('id', 'status', 'counterGroups'),
+ progressStr: function() {
+ var pct;
+ if (Ember.typeOf(this.get('progress')) === 'number') {
+ pct = App.Helpers.number.fractionToPercentage(this.get('progress'));
+ }
+ return pct;
+ }.property('id', 'status', 'progress'),
+
hasFailedTasks: function() {
return this.get('failedTasks') > 0;
}.property('id', 'counterGroups'),
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/default-configs.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/default-configs.js b/tez-ui/src/main/webapp/app/scripts/default-configs.js
index eba6a7e..e438bad 100644
--- a/tez-ui/src/main/webapp/app/scripts/default-configs.js
+++ b/tez-ui/src/main/webapp/app/scripts/default-configs.js
@@ -35,12 +35,13 @@ $.extend(true, App.Configs, {
version: "0.7.0",
timelineBaseUrl: getDefaultTimelineUrl(),
- RMWebUrl: getDefaultRMWebUrl(),
+ RMWebUrl: getDefaultRMWebUrl()
},
restNamespace: {
timeline: 'ws/v1/timeline',
- applicationHistory: 'ws/v1/applicationhistory'
+ applicationHistory: 'ws/v1/applicationhistory',
+ aminfo: 'proxy/__app_id__/ws/v1/tez'
},
tables: {
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/helpers/number.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/number.js b/tez-ui/src/main/webapp/app/scripts/helpers/number.js
index c72edb3..912e8f5 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/number.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/number.js
@@ -110,6 +110,16 @@ App.Helpers.number = {
*/
isValidInt: function(value) {
return value % 1 == 0;
+ },
+
+ /**
+ * converts fraction to percentage.
+ * @param {number} fraction assumes < 1
+ * @return {float} fixed decimal point formatted percentage
+ */
+ fractionToPercentage: function(number, decimal) {
+ decimal = decimal || 2;
+ return parseFloat((number * 100).toFixed(decimal)) + ' %';
}
};
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js b/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
index 568d526..7ce5f5e 100644
--- a/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
+++ b/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
@@ -71,8 +71,19 @@ App.PaginatedContentMixin = Em.Mixin.create({
this.get('store').unloadAll(childEntityType);
this.get('store').findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
that.set('entities', entities);
- that.set('loading', false);
- }).catch(function(jqXHR){
+ var loaders = [];
+ try {
+ var loader = Em.tryInvoke(that, 'loadAdditional');
+ if (!!loader) {
+ loaders.push(loader);
+ }
+ } catch(error) {
+ Em.Logger.error("Exception invoking additional load", error);
+ }
+ Em.RSVP.allSettled(loaders).then(function(){
+ that.set('loading', false);
+ });
+ }).catch(function(error){
Em.Logger.error(error);
var err = App.Helpers.misc.formatError(error, defaultErrMsg);
var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
index a0943e2..d135174 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
@@ -279,6 +279,7 @@ var timelineJsonToVertexMap = {
id: 'entity',
name: 'otherinfo.vertexName',
dagID: 'primaryfilters.TEZ_DAG_ID.0',
+ applicationId: 'primaryfilters.applicationId.0',
processorClassName: 'processorClassName',
counterGroups: 'counterGroups',
inputs: 'inputs',
@@ -545,3 +546,6 @@ App.TezAppSerializer = App.TimelineSerializer.extend({
return Em.JsonMapper.map(hash, timelineJsonToTezAppMap);
},
});
+
+App.VertexProgressSerializer = App.DagProgressSerializer = DS.RESTSerializer.extend({});
+
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/scripts/models/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/dag.js b/tez-ui/src/main/webapp/app/scripts/models/dag.js
index cc87f5d..e050037 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/dag.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/dag.js
@@ -17,6 +17,10 @@
App.Dag = App.AbstractEntity.extend({
+ idx: function() {
+ return this.get('id').split('_').splice(-1).pop();
+ }.property('id'),
+
submittedTime: DS.attr('number'),
// start time of the entity
@@ -93,9 +97,16 @@ App.Edge = DS.Model.extend({
App.Vertex = DS.Model.extend({
name: DS.attr('string'),
+ vertexIdx: function() {
+ return this.get('id').split('_').splice(-1).pop();
+ }.property('id'),
dag: DS.belongsTo('dag'),
dagID: DS.attr('string'),
+ applicationId: DS.attr('string'),
+ dagIdx: function() {
+ return this.get('dagID').split('_').splice(-1).pop();
+ }.property('dagID'),
/**
* State of this vertex. Should be one of constants defined in
@@ -323,6 +334,18 @@ App.Task = App.AbstractEntity.extend({
counterGroups: DS.hasMany('counterGroup', { inverse: 'parent' })
});
+App.DagProgress = DS.Model.extend({
+ progress: DS.attr('number'),
+ appId: DS.attr('string'),
+ dagIdx: DS.attr('number')
+});
+
+App.VertexProgress = DS.Model.extend({
+ progress: DS.attr('number'),
+ appId: DS.attr('string'),
+ dagIdx: DS.attr('string')
+});
+
App.KVDatum = DS.Model.extend({
key: DS.attr('string'),
value: DS.attr('string'),
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/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 0d38686..7e9dbcb 100644
--- a/tez-ui/src/main/webapp/app/templates/dag/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
@@ -17,42 +17,43 @@
}}
<div class='type-table fill-full margin-small-horizontal'>
- <div class='align-left'>
- <table class='detail-list'>
- <tbody>
- <tr>
- <td>{{t 'common.status'}}</td>
- <td>
- {{status}}
- <i {{bind-attr class=':task-status taskIconStatus'}}></i>
- </td>
+ <div class='align-left'>
+ <table class='detail-list'>
+ <tbody>
+ <tr>
+ <td>{{t 'common.status'}}</td>
+ <td>
+ {{status}}
+ <i {{bind-attr class=':task-status taskIconStatus'}}></i>
+ {{#if progressStr}} {{bs-badge content=progressStr}}{{/if}}
+ </td>
<td>
{{#if hasFailedTasks}}
<a href='{{unbound failedTasksLink}}'>FailedTasks</a>
{{/if}}
</td>
- </tr>
- </tbody>
- </table>
- </div>
- <div class='align-right'>
- <table class='detail-list'>
- <tbody>
- <tr>
- <td>{{t 'common.time.start'}}</td>
- <td>{{formatUnixTimestamp startTime}}</td>
- </tr>
- <tr>
- <td>{{t 'common.time.end'}}</td>
- <td>{{formatUnixTimestamp endTime}}</td>
- </tr>
- <tr>
- <td>{{t 'common.time.duration'}}</td>
- <td>{{formatDuration startTime endTime}}</td>
- </tr>
- </tbody>
- </table>
- </div>
+ </tr>
+ </tbody>
+ </table>
+ </div>
+ <div class='align-right'>
+ <table class='detail-list'>
+ <tbody>
+ <tr>
+ <td>{{t 'common.time.start'}}</td>
+ <td>{{formatUnixTimestamp startTime}}</td>
+ </tr>
+ <tr>
+ <td>{{t 'common.time.end'}}</td>
+ <td>{{formatUnixTimestamp endTime}}</td>
+ </tr>
+ <tr>
+ <td>{{t 'common.time.duration'}}</td>
+ <td>{{formatDuration startTime endTime}}</td>
+ </tr>
+ </tbody>
+ </table>
+ </div>
</div>
{{#if diagnostics}}
http://git-wip-us.apache.org/repos/asf/tez/blob/d51103db/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
index cbb0c44..bc2e7ef 100644
--- a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
@@ -32,6 +32,7 @@
<td>
<i {{bind-attr class=':task-status iconStatus'}}></i>
{{status}}
+ {{#if progressStr}} {{bs-badge content=progressStr}}{{/if}}
</td>
</tr>
<tr>
[08/17] tez git commit: TEZ-900. Confusing message for incorrect
queue for some tez examples. (Chang Li via hitesh)
Posted by ss...@apache.org.
TEZ-900. Confusing message for incorrect queue for some tez examples. (Chang Li via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/2b5eeea5
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/2b5eeea5
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/2b5eeea5
Branch: refs/heads/TEZ-2003
Commit: 2b5eeea52b63ee10aa92ce1d7cd7ac88aff6a730
Parents: 2b12f4c
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Feb 6 14:35:38 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Feb 6 14:35:38 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/tez/client/TezYarnClient.java | 8 ++++-
.../org/apache/tez/client/TestTezClient.java | 8 +++++
.../java/org/apache/tez/test/TestTezJobs.java | 35 ++++++++++++++++++++
4 files changed, 51 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/2b5eeea5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ab46d64..584f49a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -180,6 +180,7 @@ TEZ-UI CHANGES (TEZ-8):
Release 0.5.4: Unreleased
ALL CHANGES:
+ TEZ-900. Confusing message for incorrect queue for some tez examples.
TEZ-2036. OneToOneEdgeManager should enforce that source and destination
tasks have same number
TEZ-1895. Vertex reRunning should decrease successfulMembers of VertexGroupInfo.
http://git-wip-us.apache.org/repos/asf/tez/blob/2b5eeea5/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java
index d58a0bb..eae1909 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -66,7 +67,12 @@ public class TezYarnClient extends FrameworkClient {
@Override
public ApplicationId submitApplication(ApplicationSubmissionContext appSubmissionContext) throws YarnException, IOException {
- return yarnClient.submitApplication(appSubmissionContext);
+ ApplicationId appId= yarnClient.submitApplication(appSubmissionContext);
+ ApplicationReport appReport = getApplicationReport(appId);
+ if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED){
+ throw new IOException("Failed to submit application to YARN: " + appReport.getDiagnostics());
+ }
+ return appId;
}
@Override
http://git-wip-us.apache.org/repos/asf/tez/blob/2b5eeea5/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 f4d4112..14d4d7f 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
@@ -141,6 +141,8 @@ public class TestTezClient {
TezClientForTest client = configure(lrs, isSession);
ArgumentCaptor<ApplicationSubmissionContext> captor = ArgumentCaptor.forClass(ApplicationSubmissionContext.class);
+ when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState())
+ .thenReturn(YarnApplicationState.RUNNING);
client.start();
verify(client.mockYarnClient, times(1)).init((Configuration)any());
verify(client.mockYarnClient, times(1)).start();
@@ -197,6 +199,8 @@ public class TestTezClient {
when(client.mockYarnClient.createApplication().getNewApplicationResponse().getApplicationId())
.thenReturn(appId2);
+ when(client.mockYarnClient.getApplicationReport(appId2).getYarnApplicationState())
+ .thenReturn(YarnApplicationState.RUNNING);
dag = DAG.create("DAG").addVertex(
Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1, Resource.newInstance(1, 1)));
dagClient = client.submitDAG(dag);
@@ -269,6 +273,8 @@ public class TestTezClient {
public void testMultipleSubmissionsJob(boolean isSession) throws Exception {
TezClientForTest client1 = configure(new HashMap<String, LocalResource>(), isSession);
+ when(client1.mockYarnClient.getApplicationReport(client1.mockAppId).getYarnApplicationState())
+ .thenReturn(YarnApplicationState.RUNNING);
client1.start();
String mockLR1Name = "LR1";
@@ -287,6 +293,8 @@ public class TestTezClient {
client1.submitDAG(dag);
TezClientForTest client2 = configure();
+ when(client2.mockYarnClient.getApplicationReport(client2.mockAppId).getYarnApplicationState())
+ .thenReturn(YarnApplicationState.RUNNING);
client2.start();
// verify resubmission of same dag to new client (simulates submission error resulting in the
http://git-wip-us.apache.org/repos/asf/tez/blob/2b5eeea5/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
index adf27e8..b9cb491 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
@@ -672,6 +672,41 @@ public class TestTezJobs {
}
+ @Test(timeout = 60000)
+ public void testInvalidQueueSubmission() throws Exception {
+
+ TezConfiguration tezConf = new TezConfiguration(mrrTezCluster.getConfig());
+ YarnClient yarnClient = YarnClient.createYarnClient();
+ try {
+
+ yarnClient.init(mrrTezCluster.getConfig());
+ yarnClient.start();
+
+ SimpleSessionExample job = new SimpleSessionExample();
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false);
+ tezConf.set(TezConfiguration.TEZ_QUEUE_NAME, "nonexistent");
+
+ String[] inputPaths = new String[1];
+ String[] outputPaths = new String[1];
+ String inputDirStr = "/tmp/owc-input";
+ inputPaths[0] = inputDirStr;
+ Path inputDir = new Path(inputDirStr);
+ remoteFs.mkdirs(inputDir);
+ String outputDirStr = "/tmp/owc-output";
+ outputPaths[0] = outputDirStr;
+ job.run(tezConf, new String[] { StringUtils.join(",", inputPaths),
+ StringUtils.join(",", outputPaths), "2" }, null);
+ fail("Job submission should have thrown an exception");
+ } catch(IOException e) {
+ Assert.assertTrue(e.getMessage().contains("Failed to submit application to YARN"));
+ } finally {
+ if (yarnClient != null) {
+ yarnClient.stop();
+ }
+ }
+
+ }
+
@Test (timeout=60000)
public void testVertexOrder() throws Exception {
TezConfiguration tezConf = new TezConfiguration(mrrTezCluster.getConfig());
[17/17] tez git commit: TEZ-2019. Temporarily allow the scheduler and
launcher to be specified via configuration. (sseth)
Posted by ss...@apache.org.
TEZ-2019. Temporarily allow the scheduler and launcher to be specified
via configuration. (sseth)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/6e2f5491
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/6e2f5491
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/6e2f5491
Branch: refs/heads/TEZ-2003
Commit: 6e2f549103c7d4416c94569b9230d210e8f77e93
Parents: fe39ede
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Jan 30 16:02:32 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 10 13:51:24 2015 -0800
----------------------------------------------------------------------
TEZ-2003-CHANGES.txt | 4 +++
.../apache/tez/dag/api/TezConfiguration.java | 3 ++
.../org/apache/tez/dag/app/DAGAppMaster.java | 30 ++++++++++++++++-
.../dag/app/rm/TaskSchedulerEventHandler.java | 34 ++++++++++++++++++--
.../org/apache/tez/runtime/task/TezChild.java | 3 +-
5 files changed, 70 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/6e2f5491/TEZ-2003-CHANGES.txt
----------------------------------------------------------------------
diff --git a/TEZ-2003-CHANGES.txt b/TEZ-2003-CHANGES.txt
new file mode 100644
index 0000000..1822fcb
--- /dev/null
+++ b/TEZ-2003-CHANGES.txt
@@ -0,0 +1,4 @@
+ALL CHANGES:
+ TEZ-2019. Temporarily allow the scheduler and launcher to be specified via configuration.
+
+INCOMPATIBLE CHANGES:
http://git-wip-us.apache.org/repos/asf/tez/blob/6e2f5491/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 bff5c6a..155db98 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
@@ -1009,4 +1009,7 @@ public class TezConfiguration extends Configuration {
public static final String TEZ_AM_WEBSERVICE_ENABLE = TEZ_AM_PREFIX
+ "tez-ui.webservice.enable";
public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
+
+ public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
+ public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
}
http://git-wip-us.apache.org/repos/asf/tez/blob/6e2f5491/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 5aca3cf..c7dc79d 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
@@ -27,6 +27,8 @@ import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintWriter;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
@@ -91,6 +93,7 @@ import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.tez.common.AsyncDispatcher;
+import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezConverterUtils;
import org.apache.tez.common.TezUtilsInternal;
@@ -947,9 +950,34 @@ public class DAGAppMaster extends AbstractService {
protected ContainerLauncher
createContainerLauncher(final AppContext context) throws UnknownHostException {
if(isLocal){
+ LOG.info("Creating LocalContainerLauncher");
return new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
} else {
- return new ContainerLauncherImpl(context);
+ // TODO: Temporary reflection with specific parameters until a clean interface is defined.
+ String containerLauncherClassName = getConfig().get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
+ if (containerLauncherClassName == null) {
+ LOG.info("Creating Default Container Launcher");
+ return new ContainerLauncherImpl(context);
+ } else {
+ LOG.info("Creating container launcher : " + containerLauncherClassName);
+ Class<? extends ContainerLauncher> containerLauncherClazz = (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+ containerLauncherClassName);
+ try {
+ Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+ .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+ ctor.setAccessible(true);
+ ContainerLauncher instance = ctor.newInstance(context, getConfig(), taskAttemptListener);
+ return instance;
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/6e2f5491/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 616690c..46b9b50 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -18,6 +18,8 @@
package org.apache.tez.dag.app.rm;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.List;
@@ -42,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.TaskLocationHint;
@@ -329,12 +332,39 @@ public class TaskSchedulerEventHandler extends AbstractService
boolean isLocal = getConfig().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
TezConfiguration.TEZ_LOCAL_MODE_DEFAULT);
if (isLocal) {
+ LOG.info("Using TaskScheduler: LocalTaskSchedulerService");
return new LocalTaskSchedulerService(this, this.containerSignatureMatcher,
host, port, trackingUrl, appContext);
}
else {
- return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
- host, port, trackingUrl, appContext);
+ String schedulerClassName = getConfig().get(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS);
+ if (schedulerClassName == null) {
+ LOG.info("Using TaskScheduler: YarnTaskSchedulerService");
+ return new YarnTaskSchedulerService(this, this.containerSignatureMatcher,
+ host, port, trackingUrl, appContext);
+ } else {
+ LOG.info("Using custom TaskScheduler: " + schedulerClassName);
+ // TODO Temporary reflection with specific parameters. Remove once there is a clean interface.
+ Class<? extends TaskSchedulerService> taskSchedulerClazz =
+ (Class<? extends TaskSchedulerService>) ReflectionUtils.getClazz(schedulerClassName);
+ try {
+ Constructor<? extends TaskSchedulerService> ctor = taskSchedulerClazz
+ .getConstructor(TaskSchedulerAppCallback.class, AppContext.class, String.class,
+ Integer.class, String.class, Configuration.class);
+ ctor.setAccessible(true);
+ TaskSchedulerService taskSchedulerService =
+ ctor.newInstance(this, appContext, host, port, trackingUrl, getConfig());
+ return taskSchedulerService;
+ } catch (NoSuchMethodException e) {
+ throw new TezUncheckedException(e);
+ } catch (InvocationTargetException e) {
+ throw new TezUncheckedException(e);
+ } catch (InstantiationException e) {
+ throw new TezUncheckedException(e);
+ } catch (IllegalAccessException e) {
+ throw new TezUncheckedException(e);
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/6e2f5491/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index d537846..6164e52 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -356,7 +356,8 @@ public class TezChild {
DefaultMetricsSystem.shutdown();
if (!isLocal) {
RPC.stopProxy(umbilical);
- LogManager.shutdown();
+ // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
+// LogManager.shutdown();
}
}
}
[10/17] tez git commit: TEZ-1893. Verify invalid -1 parallelism in
DAG.verify() (zjffdu)
Posted by ss...@apache.org.
TEZ-1893. Verify invalid -1 parallelism in DAG.verify() (zjffdu)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/3da9566d
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/3da9566d
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/3da9566d
Branch: refs/heads/TEZ-2003
Commit: 3da9566de60f6be2d03465b4f4cd2724e7da8c4f
Parents: 20bf31c
Author: Jeff Zhang <zj...@apache.org>
Authored: Mon Feb 9 09:46:50 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Mon Feb 9 09:46:50 2015 +0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../main/java/org/apache/tez/dag/api/DAG.java | 42 +++++++++
.../apache/tez/client/TestTezClientUtils.java | 4 +-
.../org/apache/tez/dag/api/TestDAGVerify.java | 98 +++++++++++++++++++-
4 files changed, 142 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/3da9566d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df94b54..a1b42a8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -181,6 +181,7 @@ TEZ-UI CHANGES (TEZ-8):
Release 0.5.4: Unreleased
ALL CHANGES:
+ TEZ-1893. Verify invalid -1 parallelism in DAG.verify().
TEZ-900. Confusing message for incorrect queue for some tez examples.
TEZ-2036. OneToOneEdgeManager should enforce that source and destination
tasks have same number
http://git-wip-us.apache.org/repos/asf/tez/blob/3da9566d/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 c6a4f4a..7be5ba4 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
@@ -378,6 +378,48 @@ public class DAG {
}
}
}
+
+ // check the vertices with -1 parallelism, currently only 3 cases are allowed to has -1 parallelism.
+ // It is OK not using topological order to check vertices here.
+ // 1. has input initializers
+ // 2. 1-1 uninited sources
+ // 3. has custom vertex manager
+ for (Vertex vertex : vertices.values()) {
+ if (vertex.getParallelism() == -1) {
+ boolean hasInputInititlaizer = false;
+ if (vertex.getDataSources()!= null && !vertex.getDataSources().isEmpty()) {
+ for (DataSourceDescriptor ds : vertex.getDataSources()) {
+ if (ds.getInputInitializerDescriptor() != null) {
+ hasInputInititlaizer = true;
+ break;
+ }
+ }
+ }
+ if (hasInputInititlaizer) {
+ continue;
+ }
+
+ boolean has1to1UninitedSources = false;
+ if (vertex.getInputVertices()!= null && !vertex.getInputVertices().isEmpty()) {
+ for (Vertex srcVertex : vertex.getInputVertices()) {
+ if (srcVertex.getParallelism() == -1) {
+ has1to1UninitedSources = true;
+ break;
+ }
+ }
+ }
+ if (has1to1UninitedSources) {
+ continue;
+ }
+
+ if (vertex.getVertexManagerPlugin() != null) {
+ continue;
+ }
+ throw new IllegalStateException(vertex.getName() +
+ " has -1 tasks but does not have input initializers, " +
+ "1-1 uninited sources or custom vertex manager to set it at runtime");
+ }
+ }
}
// AnnotatedVertex is used by verify()
http://git-wip-us.apache.org/repos/asf/tez/blob/3da9566d/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 8cc55ca..468361b 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
@@ -203,7 +203,7 @@ public class TestTezClientUtils {
ApplicationId appId = ApplicationId.newInstance(1000, 1);
DAG dag = DAG.create("testdag");
- dag.addVertex(Vertex.create("testVertex", ProcessorDescriptor.create("processorClassname"))
+ dag.addVertex(Vertex.create("testVertex", ProcessorDescriptor.create("processorClassname"), 1)
.setTaskLaunchCmdOpts("initialLaunchOpts"));
AMConfiguration amConf =
new AMConfiguration(tezConf, new HashMap<String, LocalResource>(), new Credentials());
@@ -239,7 +239,7 @@ public class TestTezClientUtils {
ApplicationId appId = ApplicationId.newInstance(1000, 1);
DAG dag = DAG.create("testdag");
- dag.addVertex(Vertex.create("testVertex", ProcessorDescriptor.create("processorClassname"))
+ dag.addVertex(Vertex.create("testVertex", ProcessorDescriptor.create("processorClassname"), 1)
.setTaskLaunchCmdOpts("initialLaunchOpts"));
AMConfiguration amConf =
new AMConfiguration(tezConf, new HashMap<String, LocalResource>(), new Credentials());
http://git-wip-us.apache.org/repos/asf/tez/blob/3da9566d/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 4bf594f..74e780c 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
@@ -50,7 +50,9 @@ public class TestDAGVerify {
private final String dummyProcessorClassName = TestDAGVerify.class.getName();
private final String dummyInputClassName = TestDAGVerify.class.getName();
+ private final String dummyInputInitClassName = TestDAGVerify.class.getName();
private final String dummyOutputClassName = TestDAGVerify.class.getName();
+ private final String dummyVMPluginClassName = TestDAGVerify.class.getName();
private final int dummyTaskCount = 2;
private final Resource dummyTaskResource = Resource.newInstance(1, 1);
@@ -190,6 +192,9 @@ public class TestDAGVerify {
Vertex v1 = Vertex.create("v1",
ProcessorDescriptor.create(dummyProcessorClassName),
-1, dummyTaskResource);
+ DataSourceDescriptor dsDesc = DataSourceDescriptor.create(InputDescriptor.create(dummyInputClassName),
+ InputInitializerDescriptor.create(dummyInputInitClassName), null);
+ v1.addDataSource("input_1", dsDesc);
Vertex v2 = Vertex.create("v2",
ProcessorDescriptor.create("MapProcessor"),
-1, dummyTaskResource);
@@ -957,7 +962,7 @@ public class TestDAGVerify {
taskLocationHints.add(TaskLocationHint.createTaskLocationHint(hosts, null));
VertexLocationHint vLoc = VertexLocationHint.create(taskLocationHints);
DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"),
- null, dummyTaskCount, null, vLoc, lrs2);
+ InputInitializerDescriptor.create(dummyInputInitClassName), dummyTaskCount, null, vLoc, lrs2);
v1.addDataSource("i1", ds);
DAG dag = DAG.create("testDag");
@@ -1056,4 +1061,95 @@ public class TestDAGVerify {
Assert.assertTrue(foundModifyAcls);
}
+ // v1 has input initializer
+ @Test(timeout = 5000)
+ public void testDAGInvalidParallelism1() {
+ DAG dag = DAG.create("testDAG");
+ Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName));
+ dag.addVertex(v1);
+ try {
+ dag.verify();
+ Assert.fail();
+ } catch (Exception e) {
+ Assert.assertEquals(
+ "v1 has -1 tasks but does not have input initializers, 1-1 uninited sources or custom vertex manager to set it at runtime",
+ e.getMessage());
+ }
+
+ DataSourceDescriptor dsDesc = DataSourceDescriptor.create(InputDescriptor.create(dummyInputClassName),
+ InputInitializerDescriptor.create(dummyInputInitClassName), null);
+ v1.addDataSource("input_1", dsDesc);
+ dag.verify();
+ }
+
+ // v1 has custom vertex manager
+ @Test(timeout = 5000)
+ public void testDAGInvalidParallelism2() {
+ DAG dag = DAG.create("testDAG");
+ Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName));
+ dag.addVertex(v1);
+ try {
+ dag.verify();
+ Assert.fail();
+ } catch (Exception e) {
+ Assert.assertEquals(
+ "v1 has -1 tasks but does not have input initializers, 1-1 uninited sources or custom vertex manager to set it at runtime",
+ e.getMessage());
+ }
+
+ v1.setVertexManagerPlugin(VertexManagerPluginDescriptor.create(dummyVMPluginClassName));
+ dag.verify();
+ }
+
+ // v1 has 1-1 united source vertex v0 which has input initializer
+ @Test(timeout = 5000)
+ public void testDAGInvalidParallelism3() {
+ DAG dag = DAG.create("testDAG");
+ Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName));
+ dag.addVertex(v1);
+ try {
+ dag.verify();
+ Assert.fail();
+ } catch (Exception e) {
+ Assert.assertEquals(
+ "v1 has -1 tasks but does not have input initializers, 1-1 uninited sources or custom vertex manager to set it at runtime",
+ e.getMessage());
+ }
+
+ Vertex v0 = Vertex.create("v0", ProcessorDescriptor.create(dummyProcessorClassName));
+ DataSourceDescriptor dsDesc = DataSourceDescriptor.create(InputDescriptor.create(dummyInputClassName),
+ InputInitializerDescriptor.create(dummyInputInitClassName), null);
+ v0.addDataSource("input", dsDesc);
+ dag.addVertex(v0);
+ dag.addEdge(Edge.create(v0, v1, EdgeProperty.create(DataMovementType.ONE_TO_ONE,
+ DataSourceType.PERSISTED,SchedulingType.SEQUENTIAL,
+ OutputDescriptor.create(dummyOutputClassName),
+ InputDescriptor.create(dummyInputClassName))));
+ dag.verify();
+ }
+
+ // v1 has an 1-1 united parent v0 which has custom vertex manager
+ @Test//(timeout = 5000)
+ public void testDAGInvalidParallelism4() {
+ DAG dag = DAG.create("testDAG");
+ Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName));
+ dag.addVertex(v1);
+ try {
+ dag.verify();
+ Assert.fail();
+ } catch (Exception e) {
+ Assert.assertEquals(
+ "v1 has -1 tasks but does not have input initializers, 1-1 uninited sources or custom vertex manager to set it at runtime",
+ e.getMessage());
+ }
+
+ Vertex v0 = Vertex.create("v2", ProcessorDescriptor.create(dummyProcessorClassName));
+ v0.setVertexManagerPlugin(VertexManagerPluginDescriptor.create(dummyVMPluginClassName));
+ dag.addVertex(v0);
+ dag.addEdge(Edge.create(v0, v1, EdgeProperty.create(DataMovementType.ONE_TO_ONE,
+ DataSourceType.PERSISTED,SchedulingType.SEQUENTIAL,
+ OutputDescriptor.create(dummyOutputClassName),
+ InputDescriptor.create(dummyInputClassName))));
+ dag.verify();
+ }
}
[09/17] tez git commit: TEZ-2056. Tez UI: fix VertexID filter,
show only tez configs by default, fix appattemptid.
Posted by ss...@apache.org.
TEZ-2056. Tez UI: fix VertexID filter,show only tez configs by default,fix appattemptid.
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/20bf31cd
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/20bf31cd
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/20bf31cd
Branch: refs/heads/TEZ-2003
Commit: 20bf31cd560ff0f46d325cfdb70acb963bcbae1e
Parents: 2b5eeea
Author: Hitesh Shah <hi...@apache.org>
Authored: Sun Feb 8 11:50:20 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Sun Feb 8 11:50:20 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
tez-ui/src/main/webapp/app/index.html | 1 +
.../webapp/app/scripts/components/kv-table.js | 32 ++++++++++++++++++--
.../webapp/app/scripts/controllers/dag_tasks.js | 20 +++++++-----
.../app/scripts/models/TimelineRestAdapter.js | 7 ++++-
tez-ui/src/main/webapp/app/styles/main.less | 6 ++++
.../app/templates/components/kv-table.hbs | 10 +++++-
.../webapp/app/templates/partials/configs.hbs | 2 +-
8 files changed, 67 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 584f49a..df94b54 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/index.html
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/index.html b/tez-ui/src/main/webapp/app/index.html
index 4fe7e3d..b2d1ac3 100644
--- a/tez-ui/src/main/webapp/app/index.html
+++ b/tez-ui/src/main/webapp/app/index.html
@@ -54,6 +54,7 @@
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-nav.min.js"></script>
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-items-action-bar.min.js"></script>
<script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-badge.min.js"></script>
+ <script src="bower_components/ember-addons.bs_for_ember/dist/js/bs-label.min.js"></script>
<script src="bower_components/antiscroll/antiscroll.js"></script>
<script src="bower_components/jquery-mousewheel/jquery.mousewheel.js"></script>
<script src="bower_components/ember-table/dist/ember-table.js"></script>
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/scripts/components/kv-table.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/kv-table.js b/tez-ui/src/main/webapp/app/scripts/components/kv-table.js
index 02beeea..5e433ea 100644
--- a/tez-ui/src/main/webapp/app/scripts/components/kv-table.js
+++ b/tez-ui/src/main/webapp/app/scripts/components/kv-table.js
@@ -19,13 +19,41 @@
App.KvTableComponent = Em.Component.extend({
layoutName: 'components/kv-table',
filterExp: null,
+ showAllButtonClass: '',
+ errorMsgClass: '',
+
+ actions: {
+ showAllButtonClicked: function() {
+ this.set('filterExp', null);
+ }
+ },
+
+ showError: function(show) {
+ this.set('errorMsgClass', show ? '' : 'no-display');
+ },
filteredKVs: function() {
+ var filterExp = this.get('filterExp');
var kvList = this.get('data') || [],
filteredKvs = [],
- filterStringRegex = new RegExp(this.filterExp, 'i');
+ filterStringRegex;
+
+ if (filterExp) {
+ this.set('showAllButtonClass', '');
+ } else {
+ this.set('showAllButtonClass', 'hidden');
+ }
+
+ try {
+ filterStringRegex = new RegExp(filterExp, 'i');
+ } catch(e) {
+ this.showError(true);
+ Em.Logger.debug("Invalid regex " + e);
+ return;
+ }
- if (Em.isEmpty(this.filterExp)) {
+ this.showError(false);
+ if (Em.isEmpty(filterExp)) {
return kvList;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index 019d29a..98958ae 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -32,11 +32,15 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
vertex_id_filter: null,
loadData: function() {
+ var primaryFilter;
+ if (!!this.vertex_id_filter) {
+ primaryFilter = { TEZ_VERTEX_ID : this.vertex_id_filter };
+ } else {
+ primaryFilter = { TEZ_DAG_ID : this.get('controllers.dag.id') };
+ }
+
var filters = {
- primary: {
- TEZ_DAG_ID: this.get('controllers.dag.id'),
- TEZ_VERTEX_ID: this.vertex_id_filter,
- },
+ primary: primaryFilter,
secondary: {
status: this.status_filter
}
@@ -49,14 +53,16 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
store = this.get('store'),
fetcher;
childEntityType = this.get('childEntityType');
- var defaultErrMsg = 'Error while loading tasks. could not connect to %@'.fmt(App.env.timelineBaseUrl);
+ var defaultErrMsg = 'Error while loading tasks. could not connect to %@'
+ .fmt(App.env.timelineBaseUrl);
store.unloadAll(childEntityType);
store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
that.set('entities', entities);
var pivotLoaders = [];
entities.forEach(function (task) {
- var taskAttemptId = task.get('successfulAttemptId') || task.get('attempts.lastObject');
+ var taskAttemptId = task.get('successfulAttemptId') ||
+ task.get('attempts.lastObject');
if (!!taskAttemptId) {
// Pivot attempt selection logic
fetcher = store.find('taskAttempt', taskAttemptId);
@@ -207,4 +213,4 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
);
}.property(),
-});
\ No newline at end of file
+});
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
index d135174..6e84648 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/TimelineRestAdapter.js
@@ -461,7 +461,12 @@ App.OutputSerializer = App.TimelineSerializer.extend({
var timelineJsonToAppDetailMap = {
id: 'appId',
- attemptId: 'currentAppAttemptId',
+ attemptId: {
+ custom: function(source) {
+ // while an attempt is in progress the attempt id contains a '-'
+ return (Em.get(source, 'currentAppAttemptId') || '').replace('-','');
+ }
+ },
name: 'name',
queue: 'queue',
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/styles/main.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/main.less b/tez-ui/src/main/webapp/app/styles/main.less
index 92536f8..1681f3d 100644
--- a/tez-ui/src/main/webapp/app/styles/main.less
+++ b/tez-ui/src/main/webapp/app/styles/main.less
@@ -410,6 +410,12 @@ div.indent {
font-weight: bold;
}
+ .input-wrapper {
+ display: block;
+ overflow: hidden;
+ padding: 4px 10px 4px 4px;
+ }
+
input[type=search] {
width: 100%;
}
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/templates/components/kv-table.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/kv-table.hbs b/tez-ui/src/main/webapp/app/templates/components/kv-table.hbs
index ae2ad17..bc359d8 100644
--- a/tez-ui/src/main/webapp/app/templates/components/kv-table.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/kv-table.hbs
@@ -24,7 +24,15 @@
</tr>
<tr>
<td class='filter' colspan='2'>
- {{input size='60' type='search' results='1' placeholder='Search...' valueBinding='filterExp'}}
+ <div>
+ {{bs-button title="showAll" size="small" class="align-right" clicked="showAllButtonClicked" classBinding="showAllButtonClass"}}
+ <span class="input-wrapper">
+ {{input size='60' type='search' results='1' placeholder='Search...' valueBinding='filterExp'}}
+ </span>
+ </div>
+ <div>
+ {{bs-label content="Invalid regex" type="warning" classBinding="errorMsgClass"}}
+ </div>
</td>
</tr>
<tbody>
http://git-wip-us.apache.org/repos/asf/tez/blob/20bf31cd/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/partials/configs.hbs b/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
index a639352..0e00ca5 100644
--- a/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
+++ b/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
@@ -18,7 +18,7 @@
{{#if configs}}
<div class='table-container'>
- {{kv-table-component data=configs}}
+ {{kv-table-component data=configs filterExp="^tez."}}
</div>
{{else}}
<div class='margin-small-horizontal'>
[13/17] tez git commit: TEZ-2063. Tez UI: Flaky log url in tasks
table. (Sreenath Somarajapuram via hitesh)
Posted by ss...@apache.org.
TEZ-2063. Tez UI: Flaky log url in tasks table. (Sreenath Somarajapuram via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/4ce0b747
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/4ce0b747
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/4ce0b747
Branch: refs/heads/TEZ-2003
Commit: 4ce0b747782555527a1c1bcf69009931f9e877a2
Parents: 12c31ab
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue Feb 10 12:38:15 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue Feb 10 12:38:15 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js | 2 +-
.../main/webapp/app/scripts/controllers/vertex_tasks_controller.js | 2 +-
3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/4ce0b747/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ab9250b..cb4c468 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/4ce0b747/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index 98958ae..3be59f0 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -58,7 +58,6 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
store.unloadAll(childEntityType);
store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
- that.set('entities', entities);
var pivotLoaders = [];
entities.forEach(function (task) {
var taskAttemptId = task.get('successfulAttemptId') ||
@@ -73,6 +72,7 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
}
});
Em.RSVP.allSettled(pivotLoaders).then(function(){
+ that.set('entities', entities);
that.set('loading', false);
});
}).catch(function(error){
http://git-wip-us.apache.org/repos/asf/tez/blob/4ce0b747/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
index 4241a2b..1edb834 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
@@ -50,7 +50,6 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
store.unloadAll(childEntityType);
store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
- that.set('entities', entities);
var pivotLoaders = [];
entities.forEach(function (task) {
var taskAttemptId = task.get('successfulAttemptId') || task.get('attempts.lastObject');
@@ -64,6 +63,7 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
}
});
Em.RSVP.allSettled(pivotLoaders).then(function(){
+ that.set('entities', entities);
that.set('loading', false);
});
}).catch(function(jqXHR){
[07/17] tez git commit: TEZ-2052. Tez UI: log view fixes,
show version from build,
better handling of ats url config. (Sreenath Somarajapuram via hitesh)
Posted by ss...@apache.org.
TEZ-2052. Tez UI: log view fixes, show version from build, better handling of ats url config. (Sreenath Somarajapuram via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/2b12f4cb
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/2b12f4cb
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/2b12f4cb
Branch: refs/heads/TEZ-2003
Commit: 2b12f4cba60e043eaa725c54f7bcfb390460e24a
Parents: d51103d
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Feb 6 09:30:15 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Feb 6 09:30:15 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
tez-ui/pom.xml | 12 ++++++++++++
tez-ui/src/main/webapp/app/scripts/app.js | 7 +++++++
tez-ui/src/main/webapp/app/scripts/configs.js | 1 +
.../controllers/dag-task-attempts-controller.js | 4 ++--
.../webapp/app/scripts/controllers/dag_tasks.js | 4 ++--
.../controllers/task_task_attempts_controller.js | 4 ++--
.../controllers/vertex_task_attempts_controller.js | 4 ++--
.../scripts/controllers/vertex_tasks_controller.js | 4 ++--
tez-ui/src/main/webapp/app/scripts/helpers/misc.js | 17 +++++++++++++++++
.../src/main/webapp/app/templates/application.hbs | 4 ++--
tez-ui/src/main/webapp/app/templates/tez-app.hbs | 2 +-
12 files changed, 51 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0732004..ab46d64 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml
index ed49f88..db9673f 100644
--- a/tez-ui/pom.xml
+++ b/tez-ui/pom.xml
@@ -130,10 +130,22 @@
<configuration>
<webXml>${basedir}/src/main/webapp/WEB-INF/web.xml</webXml>
<warSourceDirectory>${webappDir}/dist</warSourceDirectory>
+ <warSourceExcludes>**/configs.js</warSourceExcludes>
<webResources>
<resource>
+ <filtering>false</filtering>
<directory>${basedir}/src/main/resources/</directory>
</resource>
+ <resource>
+ <filtering>false</filtering>
+ <directory>${webappDir}/dist</directory>
+ <exclude>**/configs.js</exclude>
+ </resource>
+ <resource>
+ <filtering>true</filtering>
+ <directory>${webappDir}/dist</directory>
+ <include>**/configs.js</include>
+ </resource>
</webResources>
</configuration>
</plugin>
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/app.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/app.js b/tez-ui/src/main/webapp/app/scripts/app.js
index 4fc461a..69d6407 100644
--- a/tez-ui/src/main/webapp/app/scripts/app.js
+++ b/tez-ui/src/main/webapp/app/scripts/app.js
@@ -34,7 +34,14 @@ var App = window.App = Em.Application.createWithMixins(Bootstrap, {
},
setConfigs: function (configs) {
+ if(configs.envDefaults.version == "${version}") {
+ delete configs.envDefaults.version;
+ }
App.Helpers.misc.merge(App.Configs, configs);
+ $.extend(App.env, {
+ timelineBaseUrl: App.Helpers.misc.normalizePath(App.env.timelineBaseUrl),
+ RMWebUrl: App.Helpers.misc.normalizePath(App.env.RMWebUrl)
+ });
App.advanceReadiness();
}
});
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/configs.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/configs.js b/tez-ui/src/main/webapp/app/scripts/configs.js
index b0397d8..1e2782f 100644
--- a/tez-ui/src/main/webapp/app/scripts/configs.js
+++ b/tez-ui/src/main/webapp/app/scripts/configs.js
@@ -20,6 +20,7 @@ App.setConfigs({
/* Environment configurations */
envDefaults: {
+ version: "${version}",
/*
* By default TEZ UI looks for timeline server at http://localhost:8188, uncomment and change
* the following value for pointing to a different domain.
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
index eaa533e..d473d9a 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
@@ -161,9 +161,9 @@ App.DagTaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentM
{{#unless view.cellContent}}\
Not Available\
{{else}}\
- <a href="//{{unbound view.cellContent}}">View</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}">View</a>\
\
- <a href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
{{/unless}}\
</span>')
}),
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index 41bd101..019d29a 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -180,9 +180,9 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
{{#unless view.cellContent}}\
Not Available\
{{else}}\
- <a href="//{{unbound view.cellContent}}">View</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}">View</a>\
\
- <a href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
{{/unless}}\
</span>')
}),
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
index 399124b..aacefe3 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
@@ -162,9 +162,9 @@ App.TaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentMixi
{{#unless view.cellContent}}\
Not Available\
{{else}}\
- <a href="//{{unbound view.cellContent}}">View</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}">View</a>\
\
- <a href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
{{/unless}}\
</span>')
}),
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
index a525ba4..4c38f51 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
@@ -162,9 +162,9 @@ App.VertexTaskAttemptsController = Em.ObjectController.extend(App.PaginatedConte
{{#unless view.cellContent}}\
Not Available\
{{else}}\
- <a href="//{{unbound view.cellContent}}">View</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}">View</a>\
\
- <a href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
{{/unless}}\
</span>')
}),
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
index 44f160e..4241a2b 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
@@ -170,9 +170,9 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
{{#unless view.cellContent}}\
Not Available\
{{else}}\
- <a href="//{{unbound view.cellContent}}">View</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}">View</a>\
\
- <a href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
+ <a target="_blank" href="//{{unbound view.cellContent}}?start=0" download target="_blank" type="application/octet-stream">Download</a>\
{{/unless}}\
</span>')
}),
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/misc.js b/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
index f115379..1bbb677 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
@@ -161,6 +161,23 @@ App.Helpers.misc = {
};
},
+ /**
+ * Normalize path
+ * @param path {String}
+ * @return normalized path {String}
+ */
+ normalizePath: function (path) {
+ if(path && path.charAt(path.length - 1) == '/') {
+ path = path.slice(0, -1);
+ }
+ return path;
+ },
+
+ /**
+ * Merge content of obj2 into obj2, array elements will be concated.
+ * @param obj1 {Object}
+ * @param obj2 {Object}
+ */
merge: function objectMerge(obj1, obj2) {
$.each(obj2, function (key, val) {
if(Array.isArray(obj1[key]) && Array.isArray(val)) {
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/templates/application.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/application.hbs b/tez-ui/src/main/webapp/app/templates/application.hbs
index 10d939e..b51818a 100644
--- a/tez-ui/src/main/webapp/app/templates/application.hbs
+++ b/tez-ui/src/main/webapp/app/templates/application.hbs
@@ -22,10 +22,10 @@
<div class="navbar navbar-static-top">
<div class="navbar-inner">
<div class="main-container">
- <a class="logo" href="/">
+ {{#link-to 'application' class="logo"}}
<img src="img/apache-tez-logo-transparent.png" width="70px"/>
<span>{{unbound App.env.version}}</span>
- </a>
+ {{/link-to}}
</div>
</div>
</div>
http://git-wip-us.apache.org/repos/asf/tez/blob/2b12f4cb/tez-ui/src/main/webapp/app/templates/tez-app.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/tez-app.hbs b/tez-ui/src/main/webapp/app/templates/tez-app.hbs
index 5ab983c..011fde7 100644
--- a/tez-ui/src/main/webapp/app/templates/tez-app.hbs
+++ b/tez-ui/src/main/webapp/app/templates/tez-app.hbs
@@ -28,7 +28,7 @@
<tbody>
<tr>
<td>YARN RM Tracking URL</td>
- <td><a {{bind-attr href=rmTrackingURL}}>{{unbound appId}}</a></td>
+ <td><a target="_blank" {{bind-attr href=rmTrackingURL}}>{{unbound appId}}</a></td>
</tr>
<tr>
<td>Attempt Id</td>
[04/17] tez git commit: TEZ-1895. Update CHANGES.txt (move TEZ-1895
to 0.5.4)
Posted by ss...@apache.org.
TEZ-1895. Update CHANGES.txt (move TEZ-1895 to 0.5.4)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/236b0e28
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/236b0e28
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/236b0e28
Branch: refs/heads/TEZ-2003
Commit: 236b0e28839938a737cac9aec0646d1d7c328cd4
Parents: f4667a6
Author: Jeff Zhang <zj...@apache.org>
Authored: Fri Feb 6 10:27:37 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Fri Feb 6 10:27:37 2015 +0800
----------------------------------------------------------------------
CHANGES.txt | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/236b0e28/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0d02ae5..079ea73 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,7 +8,6 @@ INCOMPATIBLE CHANGES
ALL CHANGES:
TEZ-2023. Refactor logIndividualFetchComplete() to be common for both shuffle-schedulers.
- TEZ-1895. Vertex reRunning should decrease successfulMembers of VertexGroupInfo.
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.
@@ -179,6 +178,7 @@ TEZ-UI CHANGES (TEZ-8):
Release 0.5.4: Unreleased
ALL CHANGES:
+ TEZ-1895. Vertex reRunning should decrease successfulMembers of VertexGroupInfo.
TEZ-2020. For 1-1 edge vertex configured event may be sent incorrectly
TEZ-2015. VertexImpl.doneReconfiguringVertex() should check other criteria
before sending notification
[12/17] tez git commit: TEZ-2062. Tez UI: Showing 50 elements not
working properly. (Sreenath Somarajapuram via hitesh)
Posted by ss...@apache.org.
TEZ-2062. Tez UI: Showing 50 elements not working properly. (Sreenath Somarajapuram via hitesh)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/12c31ab5
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/12c31ab5
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/12c31ab5
Branch: refs/heads/TEZ-2003
Commit: 12c31ab5e5a7a19dafa635e69e67f829540a0457
Parents: 28941dc
Author: Hitesh Shah <hi...@apache.org>
Authored: Mon Feb 9 16:09:22 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Mon Feb 9 16:09:22 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../main/webapp/app/scripts/components/extended-table.js | 8 ++------
tez-ui/src/main/webapp/app/styles/main.less | 3 +++
tez-ui/src/main/webapp/app/templates/dags.hbs | 10 +++-------
4 files changed, 9 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/12c31ab5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 40668e1..ab9250b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,7 @@ Release 0.6.1: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/12c31ab5/tez-ui/src/main/webapp/app/scripts/components/extended-table.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/extended-table.js b/tez-ui/src/main/webapp/app/scripts/components/extended-table.js
index e16e03c..0999176 100644
--- a/tez-ui/src/main/webapp/app/scripts/components/extended-table.js
+++ b/tez-ui/src/main/webapp/app/scripts/components/extended-table.js
@@ -207,7 +207,7 @@ App.ExTable.TableComponent = Ember.Table.EmberTableComponent.extend({
return Ember.run.next(this, function() {
return Ember.run.once(this, this.updateLayout);
});
- }, 'columns.length'),
+ }, 'columns.length', '_tableContentHeight'),
_filterHeight: function() {
var minHeight = this.get('minFilterHeight');
@@ -226,11 +226,7 @@ App.ExTable.TableComponent = Ember.Table.EmberTableComponent.extend({
height = this.get('_height');
contentHeight = this.get('_tableContentHeight') + this.get('_headerHeight') + this.get('_footerHeight')
+ this.get('_filterHeight');
- if (contentHeight < height) {
- return contentHeight;
- } else {
- return height;
- }
+ return height && contentHeight;
}.property('_height', '_tableContentHeight', '_headerHeight', '_footerHeight', '_filterHeight'),
_bodyHeight: function() {
http://git-wip-us.apache.org/repos/asf/tez/blob/12c31ab5/tez-ui/src/main/webapp/app/styles/main.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/main.less b/tez-ui/src/main/webapp/app/styles/main.less
index 1681f3d..a5d1892 100644
--- a/tez-ui/src/main/webapp/app/styles/main.less
+++ b/tez-ui/src/main/webapp/app/styles/main.less
@@ -432,6 +432,9 @@ div.indent {
}
}
}
+.ember-table-cell:last-child {
+ border: none;
+}
.inline-display {
display: inline;
http://git-wip-us.apache.org/repos/asf/tez/blob/12c31ab5/tez-ui/src/main/webapp/app/templates/dags.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dags.hbs b/tez-ui/src/main/webapp/app/templates/dags.hbs
index 69c5cf8..abe5ea5 100644
--- a/tez-ui/src/main/webapp/app/templates/dags.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dags.hbs
@@ -20,10 +20,6 @@
<li class="active">All Dags</li>
</ul>
-{{#unless loading}}
- {{partial 'partials/table-controls'}}
- </div>
- {{partial 'partials/table'}}
-{{else}}
- {{partial 'partials/loading-spinner'}}
-{{/unless}}
+<div class='margin-small-vertical'>
+ {{partial 'common/table-with-spinner'}}
+</div>
\ No newline at end of file
[15/17] tez git commit: TEZ-1914. VertexManager logic should not run
on the central dispatcher (bikas)
Posted by ss...@apache.org.
TEZ-1914. VertexManager logic should not run on the central dispatcher (bikas)
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/f0354689
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/f0354689
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/f0354689
Branch: refs/heads/TEZ-2003
Commit: f03546896ed13bb605e8f39e738d4221de04bd22
Parents: 4805530
Author: Bikas Saha <bi...@apache.org>
Authored: Tue Feb 10 13:37:43 2015 -0800
Committer: Bikas Saha <bi...@apache.org>
Committed: Tue Feb 10 13:37:43 2015 -0800
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/tez/common/AsyncDispatcher.java | 4 +-
.../java/org/apache/tez/dag/app/AppContext.java | 4 +
.../org/apache/tez/dag/app/DAGAppMaster.java | 21 ++
.../tez/dag/app/dag/event/CallableEvent.java | 42 +++
.../dag/app/dag/event/CallableEventType.java | 25 ++
.../event/VertexEventInputDataInformation.java | 40 +++
.../tez/dag/app/dag/event/VertexEventType.java | 2 +
.../apache/tez/dag/app/dag/impl/TaskImpl.java | 4 +-
.../apache/tez/dag/app/dag/impl/VertexImpl.java | 85 +++--
.../tez/dag/app/dag/impl/VertexManager.java | 311 +++++++++++++++----
.../app/dag/impl/CallableEventDispatcher.java | 37 +++
.../tez/dag/app/dag/impl/TestDAGImpl.java | 35 ++-
.../tez/dag/app/dag/impl/TestVertexImpl.java | 99 ++++--
.../tez/dag/app/dag/impl/TestVertexManager.java | 82 +++--
.../vertexmanager/InputReadyVertexManager.java | 2 +-
.../TestInputReadyVertexManager.java | 6 -
17 files changed, 646 insertions(+), 154 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4da24a7..d617bee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ Release 0.7.0: Unreleased
INCOMPATIBLE CHANGES
ALL CHANGES:
+ 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.
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
index c23d669..253db23 100644
--- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
+++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
@@ -29,7 +29,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.yarn.event.Dispatcher;
@@ -87,7 +86,7 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
this.eventDispatchers = Maps.newHashMap();
}
- Runnable createThread() {
+ public Runnable createThread() {
return new Runnable() {
@Override
public void run() {
@@ -122,6 +121,7 @@ public class AsyncDispatcher extends CompositeService implements Dispatcher {
@Override
protected void serviceInit(Configuration conf) throws Exception {
+ // TODO TEZ-2049 remove YARN reference
this.exitOnDispatchException =
conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/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 f8086d0..5564809 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
@@ -37,6 +37,8 @@ import org.apache.tez.common.security.ACLManager;
import org.apache.tez.dag.history.HistoryEventHandler;
import org.apache.tez.dag.records.TezDAGID;
+import com.google.common.util.concurrent.ListeningExecutorService;
+
/**
* Context interface for sharing information across components in Tez DAG
@@ -63,6 +65,8 @@ public interface AppContext {
String getUser();
DAG getCurrentDAG();
+
+ ListeningExecutorService getExecService();
void setDAG(DAG dag);
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/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 c7e1e83..5aca3cf 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
@@ -47,6 +47,8 @@ import java.util.Random;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
@@ -163,6 +165,9 @@ import org.codehaus.jettison.json.JSONException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* The Tez DAG Application Master.
@@ -254,6 +259,10 @@ public class DAGAppMaster extends AbstractService {
private Path currentRecoveryDataDir;
private Path tezSystemStagingDir;
private FileSystem recoveryFS;
+
+ private ExecutorService rawExecutor;
+ private ListeningExecutorService execService;
+
/**
* set of already executed dag names.
*/
@@ -483,6 +492,10 @@ public class DAGAppMaster extends AbstractService {
}
}
+ rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true)
+ .setNameFormat("App Shared Pool - " + "#%d").build());
+ execService = MoreExecutors.listeningDecorator(rawExecutor);
+
initServices(conf);
super.serviceInit(conf);
@@ -1261,6 +1274,11 @@ public class DAGAppMaster extends AbstractService {
rLock.unlock();
}
}
+
+ @Override
+ public ListeningExecutorService getExecService() {
+ return execService;
+ }
@Override
public Set<String> getAllDAGIDs() {
@@ -1677,6 +1695,7 @@ public class DAGAppMaster extends AbstractService {
if (this.dagSubmissionTimer != null) {
this.dagSubmissionTimer.cancel();
}
+
stopServices();
// Given pre-emption, we should delete tez scratch dir only if unregister is
@@ -1708,6 +1727,8 @@ public class DAGAppMaster extends AbstractService {
}
}
+ execService.shutdownNow();
+
super.serviceStop();
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
new file mode 100644
index 0000000..e148fe8
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEvent.java
@@ -0,0 +1,42 @@
+/**
+* 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.dag.event;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+import com.google.common.util.concurrent.FutureCallback;
+
+public abstract class CallableEvent extends AbstractEvent<CallableEventType> implements
+ Callable<Void> {
+ private final FutureCallback<Void> callback;
+
+ public CallableEvent(FutureCallback<Void> callback) {
+ super(CallableEventType.CALLABLE);
+ this.callback = callback;
+ }
+
+ public FutureCallback<Void> getCallback() {
+ return callback;
+ }
+
+ @Override
+ public abstract Void call() throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEventType.java
new file mode 100644
index 0000000..e9e93b9
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/CallableEventType.java
@@ -0,0 +1,25 @@
+/**
+* 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.dag.event;
+
+public enum CallableEventType {
+
+ CALLABLE,
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventInputDataInformation.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventInputDataInformation.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventInputDataInformation.java
new file mode 100644
index 0000000..6b5cad5
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventInputDataInformation.java
@@ -0,0 +1,40 @@
+/**
+* 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.dag.event;
+
+import java.util.List;
+
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+public class VertexEventInputDataInformation extends VertexEvent {
+
+ private final List<TezEvent> events;
+
+ public VertexEventInputDataInformation(TezVertexID vertexId, List<TezEvent> events) {
+ super(vertexId, VertexEventType.V_INPUT_DATA_INFORMATION);
+ this.events = events;
+ }
+
+ public List<TezEvent> getEvents() {
+ return events;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
index 5eb4929..aa202a4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
@@ -49,6 +49,8 @@ public enum VertexEventType {
//Producer: VertexInputInitializer
V_ROOT_INPUT_INITIALIZED,
V_ROOT_INPUT_FAILED,
+
+ V_INPUT_DATA_INFORMATION,
// Recover Event, Producer:DAG
V_RECOVER,
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
index 149033c..aba20cf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -252,15 +252,15 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
.addTransition(TaskStateInternal.FAILED, TaskStateInternal.FAILED,
EnumSet.of(
TaskEventType.T_TERMINATE,
+ TaskEventType.T_SCHEDULE,
TaskEventType.T_ADD_SPEC_ATTEMPT))
// Transitions from KILLED state
.addTransition(TaskStateInternal.KILLED, TaskStateInternal.KILLED,
EnumSet.of(
TaskEventType.T_TERMINATE,
+ TaskEventType.T_SCHEDULE,
TaskEventType.T_ADD_SPEC_ATTEMPT))
- .addTransition(TaskStateInternal.FAILED, TaskStateInternal.FAILED,
- TaskEventType.T_SCHEDULE)
// create the topology tables
.installTopology();
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/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 865b182..05c3cc1 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
@@ -40,6 +40,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.annotation.Nullable;
import com.google.common.base.Strings;
+
import org.apache.commons.lang.StringUtils;
import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.commons.logging.Log;
@@ -114,6 +115,7 @@ import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
import org.apache.tez.dag.app.dag.event.TaskEventTermination;
import org.apache.tez.dag.app.dag.event.TaskEventType;
import org.apache.tez.dag.app.dag.event.VertexEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation;
import org.apache.tez.dag.app.dag.event.VertexEventManagerUserCodeError;
import org.apache.tez.dag.app.dag.event.VertexEventNullEdgeInitialized;
import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
@@ -342,6 +344,11 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexEventType.V_ROOT_INPUT_INITIALIZED,
new RootInputInitializedTransition())
.addTransition(VertexState.INITIALIZING,
+ EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
+ VertexState.FAILED),
+ VertexEventType.V_INPUT_DATA_INFORMATION,
+ new InputDataInformationTransition())
+ .addTransition(VertexState.INITIALIZING,
EnumSet.of(VertexState.INITED, VertexState.FAILED),
VertexEventType.V_READY_TO_INIT,
new VertexInitializedTransition())
@@ -861,7 +868,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
@Override
public int getTotalTasks() {
- return numTasks;
+ readLock.lock();
+ try {
+ return numTasks;
+ } finally {
+ readLock.unlock();
+ }
}
@Override
@@ -2175,7 +2187,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
.getVertexManagerPlugin());
LOG.info("Setting user vertex manager plugin: "
+ pluginDesc.getClassName() + " on vertex: " + getLogIdentifier());
- vertexManager = new VertexManager(pluginDesc, this, appContext, stateChangeNotifier);
+ vertexManager = new VertexManager(pluginDesc, dagUgi, this, appContext, stateChangeNotifier);
} else {
// Intended order of picking a vertex manager
// If there is an InputInitializer then we use the RootInputVertexManager. May be fixed by TEZ-703
@@ -2188,26 +2200,26 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
+ logIdentifier);
vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(RootInputVertexManager.class.getName()),
- this, appContext, stateChangeNotifier);
+ dagUgi, this, appContext, stateChangeNotifier);
} else if (hasOneToOne && !hasCustom) {
LOG.info("Setting vertexManager to InputReadyVertexManager for "
+ logIdentifier);
vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(InputReadyVertexManager.class.getName()),
- this, appContext, stateChangeNotifier);
+ dagUgi, this, appContext, stateChangeNotifier);
} else if (hasBipartite && !hasCustom) {
LOG.info("Setting vertexManager to ShuffleVertexManager for "
+ logIdentifier);
// shuffle vertex manager needs a conf payload
vertexManager = new VertexManager(ShuffleVertexManager.createConfigBuilder(conf).build(),
- this, appContext, stateChangeNotifier);
+ dagUgi, this, appContext, stateChangeNotifier);
} else {
// schedule all tasks upon vertex start. Default behavior.
LOG.info("Setting vertexManager to ImmediateStartVertexManager for "
+ logIdentifier);
vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(ImmediateStartVertexManager.class.getName()),
- this, appContext, stateChangeNotifier);
+ dagUgi, this, appContext, stateChangeNotifier);
}
}
}
@@ -3063,14 +3075,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
VertexState state = vertex.getState();
if (state == VertexState.INITIALIZING) {
try {
- List<TezEvent> inputInfoEvents =
- vertex.vertexManager.onRootVertexInitialized(
- liInitEvent.getInputName(),
- vertex.getAdditionalInputs().get(liInitEvent.getInputName())
- .getIODescriptor(), liInitEvent.getEvents());
- if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
- VertexImpl.handleRoutedTezEvents(vertex, inputInfoEvents, false);
- }
+ vertex.vertexManager.onRootVertexInitialized(liInitEvent.getInputName(), vertex
+ .getAdditionalInputs().get(liInitEvent.getInputName()).getIODescriptor(),
+ liInitEvent.getEvents());
} catch (AMUserCodeException e) {
String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
LOG.error(msg, e);
@@ -3087,10 +3094,35 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
vertex.rootInputInitializerManager.shutdown();
vertex.rootInputInitializerManager = null;
}
+
+ // the return of these events from the VM will complete initialization and move into
+ // INITED state if possible via InputDataInformationTransition
+
+ return vertex.getState();
+ }
+ }
+
+ public static class InputDataInformationTransition implements
+ MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
+
+ @Override
+ public VertexState transition(VertexImpl vertex, VertexEvent event) {
+ VertexEventInputDataInformation iEvent = (VertexEventInputDataInformation) event;
+ List<TezEvent> inputInfoEvents = iEvent.getEvents();
+ try {
+ if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
+ VertexImpl.handleRoutedTezEvents(vertex, inputInfoEvents, false);
+ }
+ } catch (AMUserCodeException e) {
+ String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
+ LOG.error(msg, e);
+ vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE, msg + ","
+ + ExceptionUtils.getStackTrace(e.getCause()));
+ return VertexState.FAILED;
+ }
// done. check if we need to do the initialization
- if (vertex.getState() == VertexState.INITIALIZING &&
- vertex.initWaitsForRootInitializers) {
+ if (vertex.getState() == VertexState.INITIALIZING && vertex.initWaitsForRootInitializers) {
if (vertex.numInitializedInputs == vertex.inputsWithInitializers.size()) {
// set the wait flag to false if all initializers are done
vertex.initWaitsForRootInitializers = false;
@@ -4021,7 +4053,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
@Override
public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
getAdditionalInputs() {
- return this.rootInputDescriptors;
+ readLock.lock();
+ try {
+ return this.rootInputDescriptors;
+ } finally {
+ readLock.unlock();
+ }
}
@Nullable
@@ -4059,7 +4096,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
@Override
public Map<Vertex, Edge> getInputVertices() {
- return Collections.unmodifiableMap(this.sourceVertices);
+ readLock.lock();
+ try {
+ return Collections.unmodifiableMap(this.sourceVertices);
+ } finally {
+ readLock.unlock();
+ }
}
@Override
@@ -4092,7 +4134,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
}
public Resource getTaskResource() {
- return taskResource;
+ readLock.lock();
+ try {
+ return taskResource;
+ } finally {
+ readLock.unlock();
+ }
}
@VisibleForTesting
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index da86151..af92348 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -20,6 +20,8 @@ package org.apache.tez.dag.app.dag.impl;
import static com.google.common.base.Preconditions.checkNotNull;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@@ -33,6 +35,7 @@ import javax.annotation.Nullable;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.tez.common.ReflectionUtils;
@@ -54,6 +57,8 @@ import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.app.dag.Task;
import org.apache.tez.dag.app.dag.TaskAttempt;
import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation;
import org.apache.tez.dag.app.dag.event.VertexEventManagerUserCodeError;
import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source;
import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
@@ -62,7 +67,6 @@ import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.InputSpecUpdate;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.events.VertexManagerEvent;
import org.apache.tez.runtime.api.impl.EventMetaData;
import org.apache.tez.runtime.api.impl.TezEvent;
import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
@@ -72,18 +76,30 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+@SuppressWarnings("unchecked")
public class VertexManager {
- VertexManagerPluginDescriptor pluginDesc;
- VertexManagerPlugin plugin;
- Vertex managedVertex;
- VertexManagerPluginContextImpl pluginContext;
- UserPayload payload = null;
- AppContext appContext;
- BlockingQueue<TezEvent> rootInputInitEventQueue;
- StateChangeNotifier stateChangeNotifier;
+ final VertexManagerPluginDescriptor pluginDesc;
+ final UserGroupInformation dagUgi;
+ final VertexManagerPlugin plugin;
+ final Vertex managedVertex;
+ final VertexManagerPluginContextImpl pluginContext;
+ final UserPayload payload;
+ final AppContext appContext;
+ final BlockingQueue<TezEvent> rootInputInitEventQueue;
+ final StateChangeNotifier stateChangeNotifier;
+
+ private final ListeningExecutorService execService;
+ private final LinkedBlockingQueue<VertexManagerEvent> eventQueue;
+ private final AtomicBoolean eventInFlight;
+ private final AtomicBoolean pluginFailed;
private static final Log LOG = LogFactory.getLog(VertexManager.class);
+ private final VertexManagerCallback VM_CALLBACK = new VertexManagerCallback();
class VertexManagerPluginContextImpl implements VertexManagerPluginContext, VertexStateUpdateListener {
@@ -97,6 +113,9 @@ public class VertexManager {
if (isComplete()) {
throw new TezUncheckedException("Cannot invoke context methods after reporting done");
}
+ if (pluginFailed.get()) {
+ throw new TezUncheckedException("Cannot invoke context methods after throwing an exception");
+ }
}
@Override
@@ -233,6 +252,7 @@ public class VertexManager {
return appContext.getTaskScheduler().getNumClusterNodes();
}
+ // TODO TEZ-2048. Remove this API
@Override
public synchronized Container getTaskContainer(String vertexName, Integer taskIndex) {
checkAndThrowIfDone();
@@ -287,41 +307,36 @@ public class VertexManager {
managedVertex.doneReconfiguringVertex();
}
- @SuppressWarnings("unchecked")
@Override
public synchronized void onStateUpdated(VertexStateUpdate event) {
- if (isComplete()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Dropping state update for vertex=" + event.getVertexName() + ", state=" +
- event.getVertexState() +
- " since vertexmanager for " + managedVertex.getLogIdentifier() + " is complete.");
- }
- } else {
- try {
- plugin.onVertexStateUpdated(event);
- } catch (Exception e) {
- // state change must be triggered via an event transition
- appContext.getEventHandler().handle(
- new VertexEventManagerUserCodeError(managedVertex.getVertexId(),
- new AMUserCodeException(Source.VertexManager, e)));
- }
- }
+ enqueueAndScheduleNextEvent(new VertexManagerEventOnVertexStateUpdate(event));
}
}
- public VertexManager(VertexManagerPluginDescriptor pluginDesc,
+ public VertexManager(VertexManagerPluginDescriptor pluginDesc, UserGroupInformation dagUgi,
Vertex managedVertex, AppContext appContext, StateChangeNotifier stateChangeNotifier) {
checkNotNull(pluginDesc, "pluginDesc is null");
checkNotNull(managedVertex, "managedVertex is null");
checkNotNull(appContext, "appContext is null");
checkNotNull(stateChangeNotifier, "notifier is null");
this.pluginDesc = pluginDesc;
+ this.dagUgi = dagUgi;
this.managedVertex = managedVertex;
this.appContext = appContext;
this.stateChangeNotifier = stateChangeNotifier;
// don't specify the size of rootInputInitEventQueue, otherwise it will fail when addAll
this.rootInputInitEventQueue = new LinkedBlockingQueue<TezEvent>();
+
+ pluginContext = new VertexManagerPluginContextImpl();
+ Preconditions.checkArgument(pluginDesc != null);
+ plugin = ReflectionUtils.createClazzInstance(pluginDesc.getClassName(),
+ new Class[] { VertexManagerPluginContext.class }, new Object[] { pluginContext });
+ payload = pluginDesc.getUserPayload();
+ execService = appContext.getExecService();
+ eventQueue = new LinkedBlockingQueue<VertexManagerEvent>();
+ eventInFlight = new AtomicBoolean(false);
+ pluginFailed = new AtomicBoolean(false);
}
public VertexManagerPlugin getPlugin() {
@@ -329,20 +344,57 @@ public class VertexManager {
}
public void initialize() throws AMUserCodeException {
- pluginContext = new VertexManagerPluginContextImpl();
- if (pluginDesc != null) {
- plugin = ReflectionUtils.createClazzInstance(pluginDesc.getClassName(),
- new Class[]{VertexManagerPluginContext.class}, new Object[]{pluginContext});
- payload = pluginDesc.getUserPayload();
- }
try {
if (!pluginContext.isComplete()) {
- plugin.initialize();
+ // TODO TEZ-2066 tracks moving this async.
+ synchronized (VertexManager.this) {
+ plugin.initialize();
+ }
}
} catch (Exception e) {
throw new AMUserCodeException(Source.VertexManager, e);
}
}
+
+ private boolean pluginInvocationAllowed(String msg) {
+ if (pluginFailed.get()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(msg + " . Manager failed. Vertex=" + managedVertex.getLogIdentifier());
+ }
+ return false;
+ }
+ if (pluginContext.isComplete()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(msg+ " . Manager complete. Not scheduling event. Vertex=" + managedVertex.getLogIdentifier());
+ }
+ return false;
+ }
+ return true;
+ }
+
+ private void enqueueAndScheduleNextEvent(VertexManagerEvent e) {
+ if (!pluginInvocationAllowed("Dropping event")) {
+ return;
+ }
+ eventQueue.add(e);
+ tryScheduleNextEvent();
+ }
+
+ private void tryScheduleNextEvent() {
+ if (!pluginInvocationAllowed("Not scheduling")) {
+ return;
+ }
+ if (eventQueue.isEmpty()) {
+ return;
+ }
+ if (eventInFlight.compareAndSet(false, true)) {
+ // no event was in flight
+ VertexManagerEvent e = eventQueue.poll();
+ Preconditions.checkState(e != null);
+ ListenableFuture<Void> future = execService.submit(e);
+ Futures.addCallback(future, e.getCallback());
+ }
+ }
public void onVertexStarted(List<TezTaskAttemptID> completions) throws AMUserCodeException {
Map<String, List<Integer>> pluginCompletionsMap = Maps.newHashMap();
@@ -360,53 +412,180 @@ public class VertexManager {
taskIdList.add(taskId);
}
}
- try {
- if (!pluginContext.isComplete()) {
- plugin.onVertexStarted(pluginCompletionsMap);
- }
- } catch (Exception e) {
- throw new AMUserCodeException(Source.VertexManager, e);
- }
+ enqueueAndScheduleNextEvent(new VertexManagerEventOnVertexStarted(pluginCompletionsMap));
}
public void onSourceTaskCompleted(TezTaskID tezTaskId) throws AMUserCodeException {
Integer taskId = Integer.valueOf(tezTaskId.getId());
String vertexName =
appContext.getCurrentDAG().getVertex(tezTaskId.getVertexID()).getName();
- try {
- if (!pluginContext.isComplete()) {
- plugin.onSourceTaskCompleted(vertexName, taskId);
- }
- } catch (Exception e) {
- throw new AMUserCodeException(Source.VertexManager, e);
+ enqueueAndScheduleNextEvent(new VertexManagerEventSourceTaskCompleted(taskId, vertexName));
+ }
+
+ public void onVertexManagerEventReceived(
+ org.apache.tez.runtime.api.events.VertexManagerEvent vmEvent) throws AMUserCodeException {
+ enqueueAndScheduleNextEvent(new VertexManagerEventReceived(vmEvent));
+ }
+
+ public void onRootVertexInitialized(String inputName,
+ InputDescriptor inputDescriptor, List<Event> events) throws AMUserCodeException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("vertex:" + managedVertex.getLogIdentifier() + "; enqueueing onRootVertexInitialized"
+ + " on input:" + inputName + ", current task events size is " + rootInputInitEventQueue.size());
}
+ enqueueAndScheduleNextEvent(new VertexManagerEventRootInputInitialized(inputName,
+ inputDescriptor, events));
}
- public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) throws AMUserCodeException {
- try {
- if (!pluginContext.isComplete()) {
- plugin.onVertexManagerEventReceived(vmEvent);
+ private class VertexManagerCallback implements FutureCallback<Void> {
+
+ @Override
+ public void onFailure(Throwable t) {
+ // stop further event processing
+ pluginFailed.set(true);
+ eventQueue.clear();
+ // catch real root cause of failure, it would throw UndeclaredThrowableException
+ // if using UGI.doAs
+ if (t instanceof UndeclaredThrowableException) {
+ t = t.getCause();
}
- } catch (Exception e) {
- throw new AMUserCodeException(Source.VertexManager, e);
+ Preconditions.checkState(appContext != null);
+ Preconditions.checkState(managedVertex != null);
+ // state change must be triggered via an event transition
+ appContext.getEventHandler().handle(
+ new VertexEventManagerUserCodeError(managedVertex.getVertexId(),
+ new AMUserCodeException(Source.VertexManager, t)));
+ // enqueue no further events due to user code error
+ }
+
+ @Override
+ public void onSuccess(Void result) {
+ Preconditions.checkState(eventInFlight.get());
+ eventInFlight.set(false);
+ tryScheduleNextEvent();
}
}
+
+ private class VertexManagerRootInputInitializedCallback extends VertexManagerCallback {
- public List<TezEvent> onRootVertexInitialized(String inputName,
- InputDescriptor inputDescriptor, List<Event> events) throws AMUserCodeException {
- try {
- if (!pluginContext.isComplete()) {
- plugin.onRootVertexInitialized(inputName, inputDescriptor, events);
+ @Override
+ public void onSuccess(Void result) {
+ super.onSuccess(result);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("vertex:" + managedVertex.getLogIdentifier()
+ + "; after call of VertexManagerPlugin.onRootVertexInitialized" + " on input:"
+ + ", current task events size is " + rootInputInitEventQueue.size());
}
- } catch (Exception e) {
- throw new AMUserCodeException(Source.VertexManager, e);
+ List<TezEvent> resultEvents = new ArrayList<TezEvent>();
+ rootInputInitEventQueue.drainTo(resultEvents);
+ appContext.getEventHandler().handle(
+ new VertexEventInputDataInformation(managedVertex.getVertexId(), resultEvents));
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("vertex:" + managedVertex.getLogIdentifier() + "; after call of VertexManagerPlugin.onRootVertexInitialized"
- + " on input:" + inputName + ", current task events size is " + rootInputInitEventQueue.size());
+ }
+
+ class VertexManagerEventOnVertexStateUpdate extends VertexManagerEvent {
+ private final VertexStateUpdate event;
+
+ public VertexManagerEventOnVertexStateUpdate(VertexStateUpdate event) {
+ this.event = event;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ plugin.onVertexStateUpdated(event);
+ }
+
+ }
+
+ class VertexManagerEventOnVertexStarted extends VertexManagerEvent {
+ private final Map<String, List<Integer>> pluginCompletionsMap;
+
+ public VertexManagerEventOnVertexStarted(Map<String, List<Integer>> pluginCompletionsMap) {
+ this.pluginCompletionsMap = pluginCompletionsMap;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ plugin.onVertexStarted(pluginCompletionsMap);
+ }
+
+ }
+
+ class VertexManagerEventSourceTaskCompleted extends VertexManagerEvent {
+ private final Integer taskId;
+ private final String vertexName;
+
+ public VertexManagerEventSourceTaskCompleted(Integer taskId, String vertexName) {
+ this.taskId = taskId;
+ this.vertexName = vertexName;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ plugin.onSourceTaskCompleted(vertexName, taskId);
}
- List<TezEvent> resultEvents = new ArrayList<TezEvent>();
- rootInputInitEventQueue.drainTo(resultEvents);
- return resultEvents;
+
+ }
+
+ class VertexManagerEventReceived extends VertexManagerEvent {
+ private final org.apache.tez.runtime.api.events.VertexManagerEvent vmEvent;
+
+ public VertexManagerEventReceived(org.apache.tez.runtime.api.events.VertexManagerEvent vmEvent) {
+ this.vmEvent = vmEvent;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ plugin.onVertexManagerEventReceived(vmEvent);
+ }
+
+ }
+
+ class VertexManagerEventRootInputInitialized extends VertexManagerEvent {
+ private final String inputName;
+ private final InputDescriptor inputDescriptor;
+ private final List<Event> events;
+
+ public VertexManagerEventRootInputInitialized(String inputName,
+ InputDescriptor inputDescriptor, List<Event> events) {
+ super(new VertexManagerRootInputInitializedCallback());
+ this.inputName = inputName;
+ this.inputDescriptor = inputDescriptor;
+ this.events = events;
+ }
+
+ @Override
+ public void invoke() throws Exception {
+ plugin.onRootVertexInitialized(inputName, inputDescriptor, events);
+ }
+
+ }
+
+ abstract class VertexManagerEvent extends CallableEvent {
+ public VertexManagerEvent() {
+ this(VM_CALLBACK);
+ }
+ public VertexManagerEvent(VertexManagerCallback callback) {
+ super(callback);
+ }
+
+ @Override
+ public Void call() throws Exception {
+ final VertexManager manager = VertexManager.this;
+ manager.dagUgi.doAs(new PrivilegedExceptionAction<Void>() {
+ @Override
+ public Void run() throws Exception {
+ synchronized (manager) {
+ if (manager.pluginInvocationAllowed("Not invoking")) {
+ invoke();
+ }
+ }
+ return null;
+ }
+ });
+ return null;
+ }
+
+ public abstract void invoke() throws Exception;
}
}
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/CallableEventDispatcher.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/CallableEventDispatcher.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/CallableEventDispatcher.java
new file mode 100644
index 0000000..a81bd68
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/CallableEventDispatcher.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.dag.impl;
+
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+
+public class CallableEventDispatcher implements EventHandler<CallableEvent> {
+
+ @Override
+ public void handle(CallableEvent event) {
+ try {
+ event.call();
+ event.getCallback().onSuccess(null);
+ } catch (Exception e) {
+ event.getCallback().onFailure(e);
+ }
+ }
+
+ }
+
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index cae9059..599f01e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -18,6 +18,7 @@
package org.apache.tez.dag.app.dag.impl;
+import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@@ -26,9 +27,10 @@ import static org.mockito.Mockito.verify;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
@@ -37,6 +39,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -58,7 +61,6 @@ import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
-import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.oldrecords.TaskState;
import org.apache.tez.dag.api.records.DAGProtos;
@@ -85,6 +87,8 @@ import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.VertexState;
import org.apache.tez.dag.app.dag.VertexTerminationCause;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.CallableEventType;
import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
import org.apache.tez.dag.app.dag.event.DAGEvent;
@@ -125,8 +129,13 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.protobuf.ByteString;
public class TestDAGImpl {
@@ -136,6 +145,7 @@ public class TestDAGImpl {
private TezDAGID dagId;
private static Configuration conf;
private DrainDispatcher dispatcher;
+ private ListeningExecutorService execService;
private Credentials fsTokens;
private AppContext appContext;
private ACLManager aclManager;
@@ -724,6 +734,7 @@ public class TestDAGImpl {
MockDNSToSwitchMapping.initializeMockRackResolver();
}
+ @SuppressWarnings({ "unchecked", "rawtypes" })
@Before
public void setup() {
conf = new Configuration();
@@ -736,6 +747,19 @@ public class TestDAGImpl {
dispatcher = new DrainDispatcher();
fsTokens = new Credentials();
appContext = mock(AppContext.class);
+ execService = mock(ListeningExecutorService.class);
+ final ListenableFuture<Void> mockFuture = mock(ListenableFuture.class);
+
+ Mockito.doAnswer(new Answer() {
+ public ListenableFuture<Void> answer(InvocationOnMock invocation) {
+ Object[] args = invocation.getArguments();
+ CallableEvent e = (CallableEvent) args[0];
+ dispatcher.getEventHandler().handle(e);
+ return mockFuture;
+ }})
+ .when(execService).submit((Callable<Void>) any());
+
+ doReturn(execService).when(appContext).getExecService();
historyEventHandler = mock(HistoryEventHandler.class);
aclManager = new ACLManager("amUser");
doReturn(conf).when(appContext).getAMConf();
@@ -750,6 +774,7 @@ public class TestDAGImpl {
doReturn(dag).when(appContext).getCurrentDAG();
mrrAppContext = mock(AppContext.class);
doReturn(aclManager).when(mrrAppContext).getAMACLManager();
+ doReturn(execService).when(mrrAppContext).getExecService();
mrrDagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 2);
mrrDagPlan = createTestMRRDAGPlan();
mrrDag = new DAGImpl(mrrDagId, conf, mrrDagPlan,
@@ -763,6 +788,7 @@ public class TestDAGImpl {
doReturn(historyEventHandler).when(mrrAppContext).getHistoryHandler();
groupAppContext = mock(AppContext.class);
doReturn(aclManager).when(groupAppContext).getAMACLManager();
+ doReturn(execService).when(groupAppContext).getExecService();
groupDagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 3);
groupDagPlan = createGroupDAGPlan();
groupDag = new DAGImpl(groupDagId, conf, groupDagPlan,
@@ -778,6 +804,7 @@ public class TestDAGImpl {
// reset totalCommitCounter to 0
TotalCountingOutputCommitter.totalCommitCounter = 0;
+ dispatcher.register(CallableEventType.class, new CallableEventDispatcher());
taskEventDispatcher = new TaskEventDispatcher();
dispatcher.register(TaskEventType.class, taskEventDispatcher);
taskAttemptEventDispatcher = new TaskAttemptEventDispatcher();
@@ -797,6 +824,7 @@ public class TestDAGImpl {
public void teardown() {
dispatcher.await();
dispatcher.stop();
+ execService.shutdownNow();
dagPlan = null;
dag = null;
}
@@ -817,6 +845,7 @@ public class TestDAGImpl {
dispatcher.getEventHandler(), taskAttemptListener,
fsTokens, clock, "user", thh, dagWithCustomEdgeAppContext);
doReturn(conf).when(dagWithCustomEdgeAppContext).getAMConf();
+ doReturn(execService).when(dagWithCustomEdgeAppContext).getExecService();
doReturn(dagWithCustomEdge).when(dagWithCustomEdgeAppContext).getCurrentDAG();
doReturn(appAttemptId).when(dagWithCustomEdgeAppContext).getApplicationAttemptId();
doReturn(appAttemptId.getApplicationId()).when(dagWithCustomEdgeAppContext).getApplicationID();
@@ -838,7 +867,7 @@ public class TestDAGImpl {
dispatcher.await();
Assert.assertEquals(DAGState.RUNNING, impl.getState());
}
-
+
@Test(timeout = 5000)
public void testDAGInit() {
initDAG(dag);
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index 83a3a8a..2c6f5e0 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -38,6 +38,7 @@ import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
@@ -116,6 +117,8 @@ import org.apache.tez.dag.app.dag.Vertex;
import org.apache.tez.dag.app.dag.VertexState;
import org.apache.tez.dag.app.dag.VertexStateUpdateListener;
import org.apache.tez.dag.app.dag.VertexTerminationCause;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.CallableEventType;
import org.apache.tez.dag.app.dag.event.DAGEvent;
import org.apache.tez.dag.app.dag.event.DAGEventType;
import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
@@ -176,10 +179,13 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
+import org.mockito.Mockito;
import org.mockito.internal.util.collections.Sets;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -187,6 +193,7 @@ import org.mockito.stubbing.Answer;
public class TestVertexImpl {
private static final Log LOG = LogFactory.getLog(TestVertexImpl.class);
+ private ListeningExecutorService execService;
private boolean useCustomInitializer = false;
private InputInitializer customInitializer = null;
@@ -2119,6 +2126,7 @@ public class TestVertexImpl {
anyInt());
}
+ @SuppressWarnings({ "unchecked", "rawtypes" })
public void setupPostDagCreation() throws AMUserCodeException {
String dagName = "dag0";
dispatcher = new DrainDispatcher();
@@ -2138,6 +2146,19 @@ public class TestVertexImpl {
doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
doReturn(appAttemptId.getApplicationId()).when(appContext).getApplicationID();
doReturn(dag).when(appContext).getCurrentDAG();
+ execService = mock(ListeningExecutorService.class);
+ final ListenableFuture<Void> mockFuture = mock(ListenableFuture.class);
+
+ Mockito.doAnswer(new Answer() {
+ public ListenableFuture<Void> answer(InvocationOnMock invocation) {
+ Object[] args = invocation.getArguments();
+ CallableEvent e = (CallableEvent) args[0];
+ dispatcher.getEventHandler().handle(e);
+ return mockFuture;
+ }})
+ .when(execService).submit((Callable<Void>) any());
+
+ doReturn(execService).when(appContext).getExecService();
doReturn(conf).when(appContext).getAMConf();
doReturn(new Credentials()).when(dag).getCredentials();
doReturn(DAGPlan.getDefaultInstance()).when(dag).getJobPlan();
@@ -2191,6 +2212,7 @@ public class TestVertexImpl {
edge.initialize();
}
+ dispatcher.register(CallableEventType.class, new CallableEventDispatcher());
taskAttemptEventDispatcher = new TaskAttemptEventDispatcher();
dispatcher.register(TaskAttemptEventType.class, taskAttemptEventDispatcher);
taskEventDispatcher = new TaskEventDispatcher();
@@ -2224,6 +2246,7 @@ public class TestVertexImpl {
dispatcher.await();
dispatcher.stop();
}
+ execService.shutdownNow();
dispatcher = null;
vertexEventDispatcher = null;
dagEventDispatcher = null;
@@ -2247,7 +2270,6 @@ public class TestVertexImpl {
}
}
-
@SuppressWarnings("unchecked")
private void initVertex(VertexImpl v) {
Assert.assertEquals(VertexState.NEW, v.getState());
@@ -2382,6 +2404,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexState.INITED, v2.getState());
Assert.assertEquals(0, listener.events.size()); // configured event not sent
startVertex(v1, true);
+ dispatcher.await();
Assert.assertEquals(VertexState.RUNNING, v2.getState());
Assert.assertEquals(1, listener.events.size()); // configured event sent after VM
Assert.assertEquals("vertex2", listener.events.get(0).getVertexName());
@@ -2821,12 +2844,13 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexTaskAttemptProcessorFailure() {
+ public void testVertexTaskAttemptProcessorFailure() throws Exception {
initAllVertices(VertexState.INITED);
VertexImpl v = vertices.get("vertex1");
startVertex(v);
+ dispatcher.await();
TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next();
ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2));
@@ -2856,12 +2880,13 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexTaskAttemptInputFailure() {
+ public void testVertexTaskAttemptInputFailure() throws Exception {
initAllVertices(VertexState.INITED);
VertexImpl v = vertices.get("vertex1");
startVertex(v);
+ dispatcher.await();
TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next();
ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2));
@@ -2892,12 +2917,13 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexTaskAttemptOutputFailure() {
+ public void testVertexTaskAttemptOutputFailure() throws Exception {
initAllVertices(VertexState.INITED);
VertexImpl v = vertices.get("vertex1");
startVertex(v);
+ dispatcher.await();
TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next();
ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2));
@@ -3355,7 +3381,7 @@ public class TestVertexImpl {
@Test(timeout = 5000)
- public void testVertexWithOneToOneSplit() throws AMUserCodeException {
+ public void testVertexWithOneToOneSplit() throws Exception {
// create a diamond shaped dag with 1-1 edges.
// split the source and remaining vertices should split equally
// vertex with 2 incoming splits from the same source should split once
@@ -3386,7 +3412,7 @@ public class TestVertexImpl {
RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
List<TaskLocationHint> v1Hints = createTaskLocationHints(numTasks);
initializerManager1.completeInputInitialization(0, numTasks, v1Hints);
-
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v1.getState());
Assert.assertEquals(numTasks, v1.getTotalTasks());
Assert.assertEquals(RootInputVertexManager.class.getName(), v1
@@ -3437,10 +3463,10 @@ public class TestVertexImpl {
// fudge vertex manager so that tasks dont start running
v1.vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(VertexManagerPluginForTest.class.getName()),
- v1, appContext, mock(StateChangeNotifier.class));
+ UserGroupInformation.getCurrentUser(), v1, appContext, mock(StateChangeNotifier.class));
v1.vertexManager.initialize();
startVertex(v1);
-
+ dispatcher.await();
Assert.assertEquals(numTasks, vertices.get("vertex2").getTotalTasks());
Assert.assertEquals(numTasks, vertices.get("vertex3").getTotalTasks());
Assert.assertEquals(numTasks, vertices.get("vertex4").getTotalTasks());
@@ -3476,7 +3502,7 @@ public class TestVertexImpl {
// fudge vertex manager so that tasks dont start running
v1.vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(VertexManagerPluginForTest.class.getName()),
- v1, appContext, mock(StateChangeNotifier.class));
+ UserGroupInformation.getCurrentUser(), v1, appContext, mock(StateChangeNotifier.class));
v1.vertexManager.initialize();
Assert.assertEquals(numTasks, vertices.get("vertex2").getTotalTasks());
@@ -3519,7 +3545,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexWithInitializerFailure() throws AMUserCodeException {
+ public void testVertexWithInitializerFailure() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer");
@@ -3548,7 +3574,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexState.INITIALIZING, v2.getState());
RootInputInitializerManagerControlled initializerManager2 = v2.getRootInputInitializerManager();
initializerManager2.failInputInitialization();
-
+ dispatcher.await();
Assert.assertEquals(VertexState.FAILED, v2.getState());
Assert.assertEquals(RootInputVertexManager.class.getName(), v2
.getVertexManager().getPlugin().getClass().getName());
@@ -4400,7 +4426,7 @@ public class TestVertexImpl {
}
@Test(timeout = 5000)
- public void testVertexWithMultipleInitializers1() throws AMUserCodeException {
+ public void testVertexWithMultipleInitializers1() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer");
@@ -4419,15 +4445,17 @@ public class TestVertexImpl {
// Complete initializer which sets parallelism first
initializerManager1.completeInputInitialization(0, 5, v1Hints);
+ dispatcher.await();
Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
// Complete second initializer
initializerManager1.completeInputInitialization(1);
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v1.getState());
}
@Test(timeout = 5000)
- public void testVertexWithMultipleInitializers2() throws AMUserCodeException {
+ public void testVertexWithMultipleInitializers2() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer");
@@ -4446,16 +4474,18 @@ public class TestVertexImpl {
// Complete initializer which does not set parallelism
initializerManager1.completeInputInitialization(1);
+ dispatcher.await();
Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
// Complete second initializer which sets parallelism
initializerManager1.completeInputInitialization(0, 5, v1Hints);
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v1.getState());
}
@SuppressWarnings("unchecked")
@Test(timeout = 500000)
- public void testVertexWithInitializerSuccess() throws AMUserCodeException {
+ public void testVertexWithInitializerSuccess() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer");
@@ -4470,7 +4500,7 @@ public class TestVertexImpl {
RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
List<TaskLocationHint> v1Hints = createTaskLocationHints(5);
initializerManager1.completeInputInitialization(0, 5, v1Hints);
-
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v1.getState());
Assert.assertEquals(5, v1.getTotalTasks());
// task events get buffered
@@ -4510,7 +4540,7 @@ public class TestVertexImpl {
RootInputInitializerManagerControlled initializerManager2 = v2.getRootInputInitializerManager();
List<TaskLocationHint> v2Hints = createTaskLocationHints(10);
initializerManager2.completeInputInitialization(0, 10, v2Hints);
-
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v2.getState());
Assert.assertEquals(10, v2.getTotalTasks());
// task events get buffered
@@ -4530,7 +4560,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexWithInputDistributor() throws AMUserCodeException {
+ public void testVertexWithInputDistributor() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithInputDistributor("TestInputInitializer");
@@ -4547,6 +4577,7 @@ public class TestVertexImpl {
RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
byte[] payload = new byte[0];
initializerManager1.completeInputDistribution(payload);
+ dispatcher.await();
// edge is still null so its initializing
Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
Assert.assertEquals(true, initializerManager1.hasShutDown);
@@ -4565,7 +4596,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexRootInputSpecUpdateAll() throws AMUserCodeException {
+ public void testVertexRootInputSpecUpdateAll() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer");
@@ -4580,7 +4611,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
RootInputInitializerManagerControlled initializerManager1 = v3.getRootInputInitializerManager();
initializerManager1.completeInputInitialization();
-
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v3.getState());
Assert.assertEquals(expectedNumTasks, v3.getTotalTasks());
Assert.assertEquals(RootInputSpecUpdaterVertexManager.class.getName(), v3.getVertexManager()
@@ -4596,7 +4627,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testVertexRootInputSpecUpdatePerTask() throws AMUserCodeException {
+ public void testVertexRootInputSpecUpdatePerTask() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer");
@@ -4611,7 +4642,7 @@ public class TestVertexImpl {
Assert.assertEquals(VertexState.INITIALIZING, v4.getState());
RootInputInitializerManagerControlled initializerManager1 = v4.getRootInputInitializerManager();
initializerManager1.completeInputInitialization();
-
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v4.getState());
Assert.assertEquals(expectedNumTasks, v4.getTotalTasks());
Assert.assertEquals(RootInputSpecUpdaterVertexManager.class.getName(), v4.getVertexManager()
@@ -4969,7 +5000,7 @@ public class TestVertexImpl {
// fudge the vm so we can do custom stuff
vB.vertexManager = new VertexManager(
VertexManagerPluginDescriptor.create(VertexManagerPluginForTest.class.getName()),
- vB, appContext, mock(StateChangeNotifier.class));
+ UserGroupInformation.getCurrentUser(), vB, appContext, mock(StateChangeNotifier.class));
vB.vertexReconfigurationPlanned();
@@ -5093,7 +5124,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromVM_OnRootVertexInitialized() throws AMUserCodeException {
+ public void testExceptionFromVM_OnRootVertexInitialized() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnRootVertexInitialized);
@@ -5118,7 +5149,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromVM_OnVertexStarted() throws AMUserCodeException {
+ public void testExceptionFromVM_OnVertexStarted() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexStarted);
@@ -5136,7 +5167,7 @@ public class TestVertexImpl {
dispatcher.getEventHandler().handle(new VertexEvent(v1.getVertexId(),
VertexEventType.V_START));
dispatcher.await();
-
+
Assert.assertEquals(VertexManagerWithException.class, v1.vertexManager.getPlugin().getClass());
Assert.assertEquals(VertexState.FAILED, v1.getState());
String diagnostics = StringUtils.join(v1.getDiagnostics(), ",");
@@ -5146,7 +5177,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromVM_OnSourceTaskCompleted() throws AMUserCodeException {
+ public void testExceptionFromVM_OnSourceTaskCompleted() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnSourceTaskCompleted);
@@ -5183,7 +5214,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromVM_OnVertexManagerEventReceived() throws AMUserCodeException {
+ public void testExceptionFromVM_OnVertexManagerEventReceived() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexManagerEventReceived);
@@ -5210,7 +5241,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromVM_OnVertexManagerVertexStateUpdated() throws AMUserCodeException {
+ public void testExceptionFromVM_OnVertexManagerVertexStateUpdated() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithVMException("TestVMStateUpdate", VMExceptionLocation.OnVertexManagerVertexStateUpdated);
@@ -5228,7 +5259,7 @@ public class TestVertexImpl {
dispatcher.await();
Assert.assertEquals(VertexState.INITED, v2.getState());
startVertex(v1, false);
-
+ dispatcher.await();
Assert.assertEquals(VertexState.FAILED, v2.getState());
String diagnostics = StringUtils.join(v2.getDiagnostics(), ",");
assertTrue(diagnostics.contains(VMExceptionLocation.OnVertexManagerVertexStateUpdated.name()));
@@ -5261,7 +5292,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromII_InitFailedAfterInitialized() throws AMUserCodeException {
+ public void testExceptionFromII_InitFailedAfterInitialized() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithIIException();
@@ -5272,6 +5303,7 @@ public class TestVertexImpl {
initVertex(v1);
RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
initializerManager1.completeInputInitialization(0);
+ dispatcher.await();
Assert.assertEquals(VertexState.INITED, v1.getState());
String errorMsg = "ErrorWhenInitFailureAtInited";
dispatcher.getEventHandler().handle(new VertexEventRootInputFailed(v1.getVertexId(), "input1",
@@ -5285,7 +5317,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromII_InitFailedAfterRunning() throws AMUserCodeException {
+ public void testExceptionFromII_InitFailedAfterRunning() throws Exception {
useCustomInitializer = true;
setupPreDagCreation();
dagPlan = createDAGPlanWithIIException();
@@ -5296,6 +5328,7 @@ public class TestVertexImpl {
initVertex(v1);
RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
initializerManager1.completeInputInitialization(0);
+ dispatcher.await();
startVertex(v1);
Assert.assertEquals(VertexState.RUNNING, v1.getState());
String errorMsg = "ErrorWhenInitFailureAtRunning";
@@ -5310,7 +5343,7 @@ public class TestVertexImpl {
@SuppressWarnings("unchecked")
@Test(timeout = 5000)
- public void testExceptionFromII_HandleInputInitializerEvent() throws AMUserCodeException, InterruptedException {
+ public void testExceptionFromII_HandleInputInitializerEvent() throws Exception {
useCustomInitializer = true;
customInitializer = new EventHandlingRootInputInitializer(null, IIExceptionLocation.HandleInputInitializerEvent);
EventHandlingRootInputInitializer initializer =
@@ -5351,7 +5384,7 @@ public class TestVertexImpl {
dispatcher.getEventHandler()
.handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
dispatcher.await();
-
+
// it would cause v2 fail as its II throw exception in handleInputInitializerEvent
String diagnostics = StringUtils.join(v2.getDiagnostics(), ",");
assertTrue(diagnostics.contains(IIExceptionLocation.HandleInputInitializerEvent.name()));
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
index 73dc5eb..81cb42a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
@@ -23,6 +23,8 @@ import static org.mockito.Matchers.any;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
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;
import java.util.HashMap;
@@ -31,7 +33,10 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.Callable;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.VertexManagerPlugin;
import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -39,38 +44,73 @@ import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
import org.apache.tez.dag.app.AppContext;
import org.apache.tez.dag.app.dag.StateChangeNotifier;
import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation;
import org.apache.tez.runtime.api.Event;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
import org.apache.tez.runtime.api.events.VertexManagerEvent;
import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Before;
import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+@SuppressWarnings({ "rawtypes", "unchecked" })
public class TestVertexManager {
-
- @Test(timeout = 5000)
- public void testOnRootVertexInitialized() throws Exception {
- Vertex mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
- AppContext mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ AppContext mockAppContext;
+ ListeningExecutorService execService;
+ Vertex mockVertex;
+ EventHandler mockHandler;
+ ArgumentCaptor<VertexEventInputDataInformation> requestCaptor;
+
+ @Before
+ public void setup() {
+ mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+ execService = mock(ListeningExecutorService.class);
+ final ListenableFuture<Void> mockFuture = mock(ListenableFuture.class);
+ Mockito.doAnswer(new Answer() {
+ public ListenableFuture<Void> answer(InvocationOnMock invocation) {
+ Object[] args = invocation.getArguments();
+ CallableEvent e = (CallableEvent) args[0];
+ new CallableEventDispatcher().handle(e);
+ return mockFuture;
+ }})
+ .when(execService).submit((Callable<Void>) any());
+ doReturn(execService).when(mockAppContext).getExecService();
+ mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
doReturn("vertex1").when(mockVertex).getName();
+ mockHandler = mock(EventHandler.class);
+ when(mockAppContext.getEventHandler()).thenReturn(mockHandler);
when(
mockAppContext.getCurrentDAG().getVertex(any(String.class))
.getTotalTasks()).thenReturn(1);
+ requestCaptor = ArgumentCaptor.forClass(VertexEventInputDataInformation.class);
+ }
+
+ @Test(timeout = 5000)
+ public void testOnRootVertexInitialized() throws Exception {
VertexManager vm =
new VertexManager(
VertexManagerPluginDescriptor.create(RootInputVertexManager.class
- .getName()), mockVertex, mockAppContext, mock(StateChangeNotifier.class));
+ .getName()), UserGroupInformation.getCurrentUser(),
+ mockVertex, mockAppContext, mock(StateChangeNotifier.class));
vm.initialize();
InputDescriptor id1 = mock(InputDescriptor.class);
List<Event> events1 = new LinkedList<Event>();
InputDataInformationEvent diEvent1 =
InputDataInformationEvent.createWithSerializedPayload(0, null);
events1.add(diEvent1);
- List<TezEvent> tezEvents1 =
- vm.onRootVertexInitialized("input1", id1, events1);
+ vm.onRootVertexInitialized("input1", id1, events1);
+ verify(mockHandler, times(1)).handle(requestCaptor.capture());
+ List<TezEvent> tezEvents1 = requestCaptor.getValue().getEvents();
assertEquals(1, tezEvents1.size());
assertEquals(diEvent1, tezEvents1.get(0).getEvent());
@@ -79,8 +119,9 @@ public class TestVertexManager {
InputDataInformationEvent diEvent2 =
InputDataInformationEvent.createWithSerializedPayload(0, null);
events2.add(diEvent2);
- List<TezEvent> tezEvents2 =
- vm.onRootVertexInitialized("input1", id2, events2);
+ vm.onRootVertexInitialized("input1", id2, events2);
+ verify(mockHandler, times(2)).handle(requestCaptor.capture());
+ List<TezEvent> tezEvents2 = requestCaptor.getValue().getEvents();
assertEquals(tezEvents2.size(), 1);
assertEquals(diEvent2, tezEvents2.get(0).getEvent());
}
@@ -92,17 +133,11 @@ public class TestVertexManager {
*/
@Test(timeout = 5000)
public void testOnRootVertexInitialized2() throws Exception {
- Vertex mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
- AppContext mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
- doReturn("vertex1").when(mockVertex).getName();
- when(
- mockAppContext.getCurrentDAG().getVertex(any(String.class))
- .getTotalTasks()).thenReturn(1);
-
VertexManager vm =
new VertexManager(
VertexManagerPluginDescriptor.create(CustomVertexManager.class
- .getName()), mockVertex, mockAppContext, mock(StateChangeNotifier.class));
+ .getName()), UserGroupInformation.getCurrentUser(),
+ mockVertex, mockAppContext, mock(StateChangeNotifier.class));
vm.initialize();
InputDescriptor id1 = mock(InputDescriptor.class);
List<Event> events1 = new LinkedList<Event>();
@@ -111,17 +146,20 @@ public class TestVertexManager {
events1.add(diEvent1);
// do not call context.addRootInputEvents, just cache the TezEvent
- List<TezEvent> tezEventsAfterInput1 = vm.onRootVertexInitialized("input1", id1, events1);
+ vm.onRootVertexInitialized("input1", id1, events1);
+ verify(mockHandler, times(1)).handle(requestCaptor.capture());
+ List<TezEvent> tezEventsAfterInput1 = requestCaptor.getValue().getEvents();
assertEquals(0, tezEventsAfterInput1.size());
-
+
InputDescriptor id2 = mock(InputDescriptor.class);
List<Event> events2 = new LinkedList<Event>();
InputDataInformationEvent diEvent2 =
InputDataInformationEvent.createWithSerializedPayload(0, null);
events2.add(diEvent2);
// call context.addRootInputEvents(input1), context.addRootInputEvents(input2)
- List<TezEvent> tezEventsAfterInput2 =
- vm.onRootVertexInitialized("input2", id2, events2);
+ vm.onRootVertexInitialized("input2", id2, events2);
+ verify(mockHandler, times(2)).handle(requestCaptor.capture());
+ List<TezEvent> tezEventsAfterInput2 = requestCaptor.getValue().getEvents();
assertEquals(2, tezEventsAfterInput2.size());
// also verify the EventMetaData
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java
index e2e9dd3..bbe9dcf 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/InputReadyVertexManager.java
@@ -170,7 +170,7 @@ public class InputReadyVertexManager extends VertexManagerPlugin {
public synchronized void onVertexStateUpdated(VertexStateUpdate stateUpdate) throws Exception {
numConfiguredSources++;
int target = getContext().getInputVertexEdgeProperties().size();
- LOG.info("For vertex: " + getContext().getVertexName() + "Received configured signal from: "
+ LOG.info("For vertex: " + getContext().getVertexName() + " Received configured signal from: "
+ stateUpdate.getVertexName() + " numConfiguredSources: " + numConfiguredSources
+ " needed: " + target);
Preconditions.checkState(numConfiguredSources <= target, "Vertex: " + getContext().getVertexName());
http://git-wip-us.apache.org/repos/asf/tez/blob/f0354689/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java
index 8de747d..411ea71 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java
@@ -243,12 +243,6 @@ public class TestInputReadyVertexManager {
when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(3);
when(mockContext.getVertexNumTasks(mockSrcVertexId2)).thenReturn(3);
when(mockContext.getVertexNumTasks(mockSrcVertexId3)).thenReturn(3);
- when(mockContext.getTaskContainer(mockSrcVertexId2, 0)).thenReturn(mockContainer2);
- when(mockContext.getTaskContainer(mockSrcVertexId2, 1)).thenReturn(mockContainer2);
- when(mockContext.getTaskContainer(mockSrcVertexId2, 2)).thenReturn(mockContainer2);
- when(mockContext.getTaskContainer(mockSrcVertexId3, 0)).thenReturn(mockContainer3);
- when(mockContext.getTaskContainer(mockSrcVertexId3, 1)).thenReturn(mockContainer3);
- when(mockContext.getTaskContainer(mockSrcVertexId3, 2)).thenReturn(mockContainer3);
mockInputVertices.put(mockSrcVertexId1, eProp1);
mockInputVertices.put(mockSrcVertexId2, eProp2);
mockInputVertices.put(mockSrcVertexId3, eProp3);