Author: edwardyoon
Date: Thu Oct 13 00:15:43 2011
New Revision: 1182627

URL: http://svn.apache.org/viewvc?rev=1182627&view=rev
Log:
Add tasks num of Job to web UI

Modified:
    incubator/hama/trunk/CHANGES.txt
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPMaster.java
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java
    
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobStatus.java
    
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/TaskInProgress.java
    
incubator/hama/trunk/core/src/main/java/org/apache/hama/util/BSPServletUtil.java
    incubator/hama/trunk/core/src/main/webapp/bspmaster/bspjob.jsp

Modified: incubator/hama/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/CHANGES.txt?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- incubator/hama/trunk/CHANGES.txt (original)
+++ incubator/hama/trunk/CHANGES.txt Thu Oct 13 00:15:43 2011
@@ -4,6 +4,7 @@ Release 0.4 - Unreleased
 
   NEW FEATURES
 
+   HAMA-449: Add tasks num of Job to web UI (edwardyoon)
    HAMA-428: Create a separate maven module and add basic structure for the 
Graph (edwardyoon)
    HAMA-398: Add CheckPointer and saving messages for future fault-tolerant 
systems (ChiaHung Lin via edwardyoon)
 

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPMaster.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPMaster.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPMaster.java 
(original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPMaster.java 
Thu Oct 13 00:15:43 2011
@@ -733,6 +733,8 @@ public class BSPMaster implements JobSub
       JobStatus status = jip.getStatus();
 
       status.setStartTime(jip.getStartTime());
+      status.setNumOfTasks(jip.getNumOfTasks());
+      
       // Sets the user name
       status.setUsername(jip.getProfile().getUser());
       status.setName(jip.getJobName());

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java 
(original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java 
Thu Oct 13 00:15:43 2011
@@ -227,8 +227,8 @@ public class BSPPeer implements Watcher,
     BSPMessageSerializer msgSerializer = null;
     if (this.conf.getBoolean("bsp.checkpoint.enabled", false)) {
       msgSerializer = new BSPMessageSerializer(conf.getInt(
-          "bsp.checkpoint.port",
-          Integer.parseInt(CheckpointRunner.DEFAULT_PORT)));
+          "bsp.checkpoint.port", Integer
+              .valueOf(CheckpointRunner.DEFAULT_PORT)));
     }
     this.messageSerializer = msgSerializer;
   }
@@ -237,8 +237,8 @@ public class BSPPeer implements Watcher,
     try {
       if (LOG.isDebugEnabled())
         LOG.debug("reinitialize(): " + getPeerName());
-      this.server = RPC.getServer(this, peerAddress.getHostName(),
-          peerAddress.getPort(), conf);
+      this.server = RPC.getServer(this, peerAddress.getHostName(), peerAddress
+          .getPort(), conf);
       server.start();
       LOG.info(" BSPPeer address:" + peerAddress.getHostName() + " port:"
           + peerAddress.getPort());
@@ -389,15 +389,8 @@ public class BSPPeer implements Watcher,
     public void process(WatchedEvent event) {
       this.complete = true;
       synchronized (mutex) {
-        LOG.info(">>>>>>>>>>>>>>> at superstep " + getSuperstepCount()
+        LOG.debug(">>>>>>>>>>>>>>> at superstep " + getSuperstepCount()
             + " taskid:" + taskid.toString() + " is notified.");
-        /*
-         * try { Stat s = zk.exists(pathToSuperstepZnode+"/ready", false);
-         * if(null != s) { zk.delete(pathToSuperstepZnode+"/ready", 0); } }
-         * catch(KeeperException.NoNodeException nne) {
-         * LOG.warn("Ignore because znode may be deleted.", nne); }
-         * catch(Exception e) { throw new RuntimeException(e); }
-         */
         mutex.notifyAll();
       }
     }
@@ -464,8 +457,9 @@ public class BSPPeer implements Watcher,
         + taskid.getJobID().toString() + "/" + getSuperstepCount();
     while (true) {
       List<String> znodes = zk.getChildren(pathToSuperstepZnode, false);
-      LOG.info("leaveBarrier() !!! checking znodes cotnains /ready node or 
not: at superstep:"
-          + getSuperstepCount() + " znode:" + znodes);
+      LOG
+          .info("leaveBarrier() !!! checking znodes contnains /ready node or 
not: at superstep:"
+              + getSuperstepCount() + " znode:" + znodes);
       if (znodes.contains("ready")) {
         znodes.remove("ready");
       }
@@ -597,20 +591,23 @@ public class BSPPeer implements Watcher,
   }
 
   protected BSPPeerInterface getBSPPeerConnection(InetSocketAddress addr)
-      throws NullPointerException {
+      throws NullPointerException, IOException {
     BSPPeerInterface peer;
     synchronized (this.peers) {
       peer = peers.get(addr);
 
-      if (peer == null) {
-        try {
-          peer = (BSPPeerInterface) RPC.getProxy(BSPPeerInterface.class,
-              BSPPeerInterface.versionID, addr, this.conf);
-        } catch (IOException e) {
-          LOG.error(e);
+      int retries = 0;
+      while (peer != null) {
+        peer = (BSPPeerInterface) RPC.getProxy(BSPPeerInterface.class,
+            BSPPeerInterface.versionID, addr, this.conf);
+
+        retries++;
+        if (retries > 10) {
+          umbilical.fatalError(taskid, addr + " doesn't repond.");
         }
-        this.peers.put(addr, peer);
       }
+
+      this.peers.put(addr, peer);
     }
 
     return peer;
@@ -630,8 +627,8 @@ public class BSPPeer implements Watcher,
           "Peername must consist of exactly ONE \":\"! Given peername was: "
               + peerName);
     }
-    return new InetSocketAddress(peerAddrParts[0],
-        Integer.parseInt(peerAddrParts[1]));
+    return new InetSocketAddress(peerAddrParts[0], Integer
+        .valueOf(peerAddrParts[1]));
   }
 
   @Override

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java 
(original)
+++ 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobInProgress.java 
Thu Oct 13 00:15:43 2011
@@ -133,6 +133,10 @@ class JobInProgress {
     return finishTime;
   }
 
+  public int getNumOfTasks() {
+    return tasks.length;  
+  }
+  
   /**
    * @return the number of desired tasks.
    */
@@ -181,7 +185,7 @@ class JobInProgress {
       LOG.debug("numBSPTasks: " + numBSPTasks);
     }
 
-    // adjust number of map tasks to actual number of splits
+    // adjust number of BSP tasks to actual number of splits
     this.tasks = new TaskInProgress[numBSPTasks];
     for (int i = 0; i < numBSPTasks; i++) {
       tasks[i] = new TaskInProgress(getJobID(), this.jobFile.toString(),

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobStatus.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobStatus.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobStatus.java 
(original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/JobStatus.java 
Thu Oct 13 00:15:43 2011
@@ -97,7 +97,8 @@ public class JobStatus implements Writab
   private String user;
   private long superstepCount;
   private String name;
-
+  private int tasks;
+  
   private long finishTime;
 
   public JobStatus() {
@@ -172,6 +173,14 @@ public class JobStatus implements Writab
   public synchronized void setRunState(int state) {
     this.runState = state;
   }
+  
+  public synchronized void setNumOfTasks(int tasks) {
+    this.tasks = tasks;
+  }
+  
+  public synchronized int getNumOfTasks() {
+    return tasks;
+  }
 
   public synchronized long getSuperstepCount() {
     return superstepCount;

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/TaskInProgress.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/TaskInProgress.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/TaskInProgress.java 
(original)
+++ 
incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/TaskInProgress.java 
Thu Oct 13 00:15:43 2011
@@ -87,7 +87,7 @@ class TaskInProgress {
     this.jobFile = jobFile;
     this.partition = partition;
 
-    this.id = new TaskID(jobId, partition);
+    init(jobId);
   }
 
   public TaskInProgress(BSPJobID jobId, String jobFile, BSPMaster master,
@@ -99,9 +99,14 @@ class TaskInProgress {
     this.setConf(conf);
     this.partition = partition;
 
-    this.id = new TaskID(jobId, partition);
+    init(jobId);
   }
 
+  private void init(BSPJobID jobId2) {
+    this.id = new TaskID(jobId, partition);
+    this.startTime = System.currentTimeMillis();
+  }
+  
   /**
    * Return a Task that can be sent to a GroomServer for execution.
    */

Modified: 
incubator/hama/trunk/core/src/main/java/org/apache/hama/util/BSPServletUtil.java
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/util/BSPServletUtil.java?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- 
incubator/hama/trunk/core/src/main/java/org/apache/hama/util/BSPServletUtil.java
 (original)
+++ 
incubator/hama/trunk/core/src/main/java/org/apache/hama/util/BSPServletUtil.java
 Thu Oct 13 00:15:43 2011
@@ -61,7 +61,8 @@ public class BSPServletUtil extends Serv
     if (jobs.length > 0) {
       sb.append("<table border=\"1\" cellpadding=\"5\" cellspacing=\"0\">\n");
       sb.append("<tr><th>Jobid</th>" + "<th>User</th>" + "<th>Name</th>"
-          + "<th>SuperStep</th>" + "<th>Starttime</th>" + "</tr>\n");
+          + "<th>SuperSteps</th>" + "<th>Tasks</th>" + "<th>Starttime</th>"
+          + "</tr>\n");
       for (JobStatus status : jobs) {
         sb.append("<tr><td><a href=\"bspjob.jsp?jobid=" + status.getJobID()
             + "\">");
@@ -71,7 +72,9 @@ public class BSPServletUtil extends Serv
         sb.append("</td><td>");
         sb.append(status.getName());
         sb.append("</td><td>");
-        sb.append(status.progress());
+        sb.append(status.getSuperstepCount());
+        sb.append("</td><td>");
+        sb.append(status.getNumOfTasks());
         sb.append("</td><td>");
         sb.append(new Date(status.getStartTime()));
         sb.append("</td></tr>\n");
@@ -89,13 +92,14 @@ public class BSPServletUtil extends Serv
     StringBuilder sb = new StringBuilder();
     sb.append("<center>\n");
     sb.append("<table border=\"2\" cellpadding=\"5\" cellspacing=\"2\">\n");
-    sb.append("<tr><td align=\"center\" colspan=\"6\"><b>Groom 
Servers</b></td></tr>\n");
-    sb.append("<tr><td><b>Name</b></td>"
-        + "<td><b>Host</b></td>"
-        + "<td><b># maximum tasks</b></td><td><b># current running 
tasks</b></td>" +
-                       "<td><b># current failures</b></td>" +
-                       "<td><b>Last seen</b></td>" +
-                       "</tr>\n");
+    sb
+        .append("<tr><td align=\"center\" colspan=\"6\"><b>Groom 
Servers</b></td></tr>\n");
+    sb
+        .append("<tr><td><b>Name</b></td>"
+            + "<td><b>Host</b></td>"
+            + "<td><b># maximum tasks</b></td><td><b># current running 
tasks</b></td>"
+            + "<td><b># current failures</b></td>"
+            + "<td><b>Last seen</b></td>" + "</tr>\n");
     for (Entry<String, GroomServerStatus> entry : status
         .getActiveGroomServerStatus().entrySet()) {
       sb.append("<tr><td>");

Modified: incubator/hama/trunk/core/src/main/webapp/bspmaster/bspjob.jsp
URL: 
http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/webapp/bspmaster/bspjob.jsp?rev=1182627&r1=1182626&r2=1182627&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/webapp/bspmaster/bspjob.jsp (original)
+++ incubator/hama/trunk/core/src/main/webapp/bspmaster/bspjob.jsp Thu Oct 13 
00:15:43 2011
@@ -37,11 +37,12 @@
   <b>State: </b>  <%=state.toString() %> 
   
   <br/> <br/>
-  <table border="1" cellpadding="5" cellspacing="0">
+  <table border="1" cellpadding="6" cellspacing="0">
     <tr>
       <th>Name</th>
       <th>User</th>
-      <th>SuperStep</th>
+      <th>SuperSteps</th>
+      <th>Tasks</th>
       <th>StartTime</th>
       <th>FinishTime</th>
     </tr>
@@ -50,6 +51,7 @@
       <td><%=status.getName() %></td>
       <td><%=status.getUsername() %></td>
       <td><%=status.getSuperstepCount() %></td>
+      <td><%=status.getNumOfTasks() %></td>
       <td><%=new Date(status.getStartTime()).toString() %></td>
       <td>
         <% if(status.getFinishTime() != 0L) {out.write(new 
Date(status.getFinishTime()).toString());} %>


Reply via email to