Browse Source

AMBARI-1163. During agent registration and heartbeat, send information about various hadoop artifacts back to Ambari. (Nate Cole via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/incubator/ambari/trunk@1433560 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 12 years ago
parent
commit
a90f0521be
20 changed files with 416 additions and 40 deletions
  1. 16 1
      CHANGES.txt
  2. 12 3
      ambari-agent/src/main/python/ambari_agent/Heartbeat.py
  3. 115 0
      ambari-agent/src/main/python/ambari_agent/HostInfo.py
  4. 1 0
      ambari-agent/src/main/python/ambari_agent/NetUtil.py
  5. 9 1
      ambari-agent/src/main/python/ambari_agent/Register.py
  6. 1 1
      ambari-agent/src/test/python/TestHeartbeat.py
  7. 2 1
      ambari-agent/src/test/python/TestRegistration.py
  8. 122 0
      ambari-server/src/main/java/org/apache/ambari/server/agent/AgentEnv.java
  9. 9 0
      ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeat.java
  10. 4 4
      ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
  11. 18 9
      ambari-server/src/main/java/org/apache/ambari/server/agent/Register.java
  12. 23 6
      ambari-server/src/main/java/org/apache/ambari/server/controller/HostResponse.java
  13. 4 0
      ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java
  14. 11 0
      ambari-server/src/main/java/org/apache/ambari/server/state/Host.java
  15. 12 1
      ambari-server/src/main/java/org/apache/ambari/server/state/host/HostHealthyHeartbeatEvent.java
  16. 22 2
      ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java
  17. 6 3
      ambari-server/src/main/java/org/apache/ambari/server/state/host/HostRegistrationRequestEvent.java
  18. 1 0
      ambari-server/src/main/resources/properties.json
  19. 21 5
      ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java
  20. 7 3
      ambari-server/src/test/java/org/apache/ambari/server/state/host/HostTest.java

+ 16 - 1
CHANGES.txt

@@ -1,12 +1,27 @@
 Ambari   Change log
 
+
+Trunk:
+
+ INCOMPATIBLE CHANGES 
+
+ NEW FEATURES:
+
+ AMBARI-1163. During agent registration and heartbeat, send information about
+ various hadoop artifacts back to Ambari. (Nate Cole via mahadev)
+
+ BUG FIXES:
+
+
+
+AMBARI-1.2.0 branch:
+
 Notes:
  - Committers should be listed using their login and non-committers
 should be listed by their full name.
  - Please keep the file to a max of 80 characters wide.
  - Put latest commits first in each section.
 
-  Merging AMBARI-666 to trunk.
 
  INCOMPATIBLE CHANGES
  

+ 12 - 3
ambari-agent/src/main/python/ambari_agent/Heartbeat.py

@@ -23,11 +23,13 @@ import logging
 from Hardware import Hardware
 from ActionQueue import ActionQueue
 from ServerStatus import ServerStatus
+import NetUtil
 import AmbariConfig
 import socket
 import time
 import traceback
 from pprint import pprint, pformat
+from HostInfo import HostInfo
 
 logger = logging.getLogger()
 
@@ -38,7 +40,7 @@ class Heartbeat:
     self.actionQueue = actionQueue
     self.reports = []
 
-  def build(self, id='-1'):
+  def build(self, id='-1', state_interval=-1):
     global clusterId, clusterDefinitionRevision, firstContact
     timestamp = int(time.time()*1000)
     queueResult = self.actionQueue.result()
@@ -52,6 +54,13 @@ class Heartbeat:
                   'hostname'          : socket.getfqdn(),
                   'nodeStatus'        : nodeStatus
                 }
+    if (int(id) >= 0) and (int(id) % state_interval) == 0:
+      hostInfo = HostInfo()
+      nodeInfo = { }
+      # for now, just do the same work as registration
+      hostInfo.register(nodeInfo)
+      heartbeat['agentEnv'] = nodeInfo
+
     if len(queueResult) != 0:
       heartbeat['reports'] = queueResult['reports']
       heartbeat['componentStatus'] = queueResult['componentStatus']
@@ -60,9 +69,9 @@ class Heartbeat:
     return heartbeat
 
 def main(argv=None):
-  actionQueue = ActionQueue()
+  actionQueue = ActionQueue(AmbariConfig.config)
   heartbeat = Heartbeat(actionQueue)
-  print json.dumps(heartbeat.build())
+  print json.dumps(heartbeat.build('3',3))
 
 if __name__ == '__main__':
   main()

+ 115 - 0
ambari-agent/src/main/python/ambari_agent/HostInfo.py

@@ -0,0 +1,115 @@
+#!/usr/bin/env python2.6
+
+'''
+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.
+'''
+
+import os
+import glob
+import pwd
+
+class HostInfo:
+
+  def dirType(self, path):
+    if not os.path.exists(path):
+      return 'not_exist'
+    elif os.path.islink(path):
+      return 'sym_link'
+    elif os.path.isdir(path):
+      return 'directory'
+    return 'unknown'
+
+  def hadoopDir(self):
+    return self.dirType('/etc/hadoop')
+
+  def hadoopVarRunCount(self):
+    if not os.path.exists('/var/run/hadoop'):
+      return 0
+    pids = glob.glob('/var/run/hadoop/*/*.pid')
+    return len(pids)
+
+  def hadoopVarLogCount(self):
+    if not os.path.exists('/var/log/hadoop'):
+      return 0
+    logs = glob.glob('/var/log/hadoop/*/*.log')
+    return len(logs)
+
+  def dirHelper(self, dict, name, prefix):
+    dict[name] = self.dirType(os.path.join(prefix, name))
+
+  def register(self, dict):
+    dict['varLogHadoopLogCount'] = self.hadoopVarLogCount()
+    dict['varRunHadoopPidCount'] = self.hadoopVarRunCount()
+
+    etcDirs = { }
+    self.dirHelper(etcDirs, 'hadoop', '/etc')
+    etcDirs['hadoop_conf'] = self.dirType('/etc/hadoop/conf')
+    self.dirHelper(etcDirs, 'hbase', '/etc')
+    self.dirHelper(etcDirs, 'hcatalog', '/etc')
+    self.dirHelper(etcDirs, 'hive', '/etc')
+    self.dirHelper(etcDirs, 'oozie', '/etc')
+    self.dirHelper(etcDirs, 'sqoop', '/etc')
+    self.dirHelper(etcDirs, 'ganglia', '/etc')
+    self.dirHelper(etcDirs, 'nagios', '/etc')
+    dict['etcDirs'] = etcDirs
+    
+    varRunDirs = { }
+    self.dirHelper(varRunDirs, 'hadoop', '/var/run')
+    self.dirHelper(varRunDirs, 'zookeeper', '/var/run')
+    self.dirHelper(varRunDirs, 'hbase', '/var/run')
+    self.dirHelper(varRunDirs, 'templeton', '/var/run')
+    self.dirHelper(varRunDirs, 'oozie', '/var/run')
+    dict['varRunDirs'] = varRunDirs
+
+    varLogDirs = { }
+    self.dirHelper(varLogDirs, 'hadoop', '/var/log')
+    self.dirHelper(varLogDirs, 'zookeeper', '/var/log')
+    self.dirHelper(varLogDirs, 'hbase', '/var/log')
+    self.dirHelper(varLogDirs, 'hive', '/var/log')
+    self.dirHelper(varLogDirs, 'templeton', '/var/log')
+    self.dirHelper(varLogDirs, 'nagios', '/var/log')
+    dict['varLogDirs'] = varLogDirs
+
+    java = []
+    self.hadoopJava(java)
+    dict['hadoopJavaProcs'] = java
+    
+  def hadoopJava(self, list):
+    try:
+      pids = [pid for pid in os.listdir('/proc') if pid.isdigit()]
+      for pid in pids:
+        cmd = open(os.path.join('/proc', pid, 'cmdline'), 'rb').read()
+        if 'java' in cmd and 'hadoop' in cmd:
+          for line in open(os.path.join('/proc', pid, 'status')):
+            if line.startswith('Uid:'):
+              uid = int(line.split()[1])
+              dict = { }
+              dict['user'] = pwd.getpwuid(uid).pw_name
+              dict['pid'] = int(pid)
+              list.append(dict)
+    except:
+      pass
+    pass
+
+def main(argv=None):
+  h = HostInfo()
+  struct = { }
+  h.register(struct)
+  print struct
+
+if __name__ == '__main__':
+  main()

+ 1 - 0
ambari-agent/src/main/python/ambari_agent/NetUtil.py

@@ -29,6 +29,7 @@ class NetUtil:
   CONNECT_SERVER_RETRY_INTERVAL_SEC = 10
   HEARTBEAT_IDDLE_INTERVAL_SEC = 10
   HEARTBEAT_NOT_IDDLE_INTERVAL_SEC = 5
+  HEARTBEAT_STATE_INTERVAL = 6 # default one per minute
 
   # Url within server to request during status check. This url
   # should return HTTP code 200

+ 9 - 1
ambari-agent/src/main/python/ambari_agent/Register.py

@@ -27,6 +27,7 @@ import socket
 import time
 import urllib2
 import subprocess
+from HostInfo import HostInfo
 
 
 firstContact = True
@@ -38,7 +39,7 @@ class Register:
 
   def pfqdn(self):
     try:
-      handle = urllib2.urlopen('http://169.254.169.254/latest/meta-data/public-hostname', '', 3)
+      handle = urllib2.urlopen('http://169.254.169.254/latest/meta-data/public-hostname', '', 2)
       str = handle.read()
       handle.close()
       return str
@@ -48,12 +49,19 @@ class Register:
   def build(self, id='-1'):
     global clusterId, clusterDefinitionRevision, firstContact
     timestamp = int(time.time()*1000)
+   
+    hostInfo = HostInfo() 
+    agentEnv = { }
+    hostInfo.register(agentEnv)
+    
     register = { 'responseId'        : int(id),
                   'timestamp'         : timestamp,
                   'hostname'          : socket.getfqdn(),
                   'publicHostname'    : self.pfqdn(),
                   'hardwareProfile'   : self.hardware.get(),
+                  'agentEnv'          : agentEnv
                 }
+    print str(time.time())
     return register
 
 def doExec(vals, key, command, preLF=False):

+ 1 - 1
ambari-agent/src/test/python/TestHeartbeat.py

@@ -47,7 +47,7 @@ class TestHeartbeat(TestCase):
     self.assertEquals(len(result['nodeStatus']), 2)
     self.assertEquals(result['nodeStatus']['cause'], "NONE")
     self.assertEquals(result['nodeStatus']['status'], "HEALTHY")
-    self.assertEquals(len(result), 6)
+    self.assertEquals(len(result), 7)
     self.assertEquals(not heartbeat.reports, True, "Heartbeat should not contain task in progress")
 
 

+ 2 - 1
ambari-agent/src/test/python/TestRegistration.py

@@ -34,4 +34,5 @@ class TestRegistration(TestCase):
     self.assertEquals(data['publicHostname'] != "", True, "publicHostname should not be empty")
     self.assertEquals(data['responseId'], 1)
     self.assertEquals(data['timestamp'] > 1353678475465L, True, "timestamp should not be empty")
-    self.assertEquals(len(data), 5)
+    self.assertEquals(len(data['agentEnv']) > 0, True, "agentEnv should not be empty")
+    self.assertEquals(len(data), 6)

+ 122 - 0
ambari-server/src/main/java/org/apache/ambari/server/agent/AgentEnv.java

@@ -0,0 +1,122 @@
+/**
+ * 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.ambari.server.agent;
+
+import java.util.Map;
+
+/**
+ * Agent environment data.
+ */
+public class AgentEnv {
+  /**
+   * Various directories in /etc
+   */
+  private Map<String, String> etcDirs = null;
+  /**
+   * Various directories in /var/run
+   */
+  private Map<String, String> varRunDirs = null;
+  /**
+   * Various directories in /var/log
+   */
+  private Map<String, String> varLogDirs = null;
+
+  /**
+   * Java processes with the word "hadoop" in them, users and pids
+   */
+  private JavaProc[] hadoopJavaProcs = null;
+  /**
+   * Number of pid files found in /var/run/hadoop
+   */
+  private int varRunHadoopPidCount = 0;
+  /**
+   * Number of log files found in /var/log/hadoop
+   */
+  private int varLogHadoopLogCount = 0;
+
+  
+  public Map<String, String> getEtcDirs() {
+    return etcDirs;
+  }
+  
+  public void setEtcDirs(Map<String, String> dirs) {
+    etcDirs = dirs;
+  }
+  
+  public Map<String, String> getVarRunDirs() {
+    return varRunDirs;
+  }
+  
+  public void setVarRunDirs(Map<String, String> dirs) {
+    varRunDirs = dirs;
+  }
+  
+  public Map<String, String> getVarLogDirs() {
+    return varLogDirs;
+  }
+  
+  public void setVarLogDirs(Map<String, String> dirs) {
+    varLogDirs = dirs;
+  }
+  
+  public void setVarRunHadoopPidCount(int count) {
+    varRunHadoopPidCount = count;
+  }
+  
+  public int getVarRunHadoopPidCount() {
+    return varRunHadoopPidCount;
+  }
+  
+  public void setVarLogHadoopLogCount(int count) {
+    varLogHadoopLogCount = count;
+  }
+  
+  public int getVarLogHadoopLogCount() {
+    return varLogHadoopLogCount;
+  }
+  
+  public void setHadoopJavaProcs(JavaProc[] procs) {
+    hadoopJavaProcs = procs;
+  }
+  
+  public JavaProc[] getHadoopJavaProcs() {
+    return hadoopJavaProcs;
+  }
+  
+  public static class JavaProc {
+    private String user;
+    private int pid;
+    
+    public void setUser(String user) {
+      this.user = user;
+    }
+    
+    public String getUser() {
+      return user;
+    }
+    
+    public void setPid(int pid) {
+      this.pid = pid;
+    }
+    
+    public int getPid() {
+      return pid;
+    }
+  }
+  
+}

+ 9 - 0
ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeat.java

@@ -37,6 +37,7 @@ public class HeartBeat {
   List<CommandReport> reports = new ArrayList<CommandReport>();
   List<ComponentStatus> componentStatus = new ArrayList<ComponentStatus>();
   HostStatus nodeStatus;
+  private AgentEnv agentEnv = null;
 
   public long getResponseId() {
     return responseId;
@@ -79,6 +80,14 @@ public class HeartBeat {
   public void setNodeStatus(HostStatus nodeStatus) {
     this.nodeStatus = nodeStatus;
   }
+  
+  public AgentEnv getAgentEnv() {
+    return agentEnv;
+  }
+  
+  public void setAgentEnv(AgentEnv env) {
+    agentEnv = env;
+  }
 
   @JsonProperty("componentStatus")
   public List<ComponentStatus> getComponentStatus() {

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java

@@ -151,9 +151,9 @@ public class HeartBeatHandler {
     }
 
     try {
-      if (heartbeat.getNodeStatus().getStatus()
-          .equals(HostStatus.Status.HEALTHY)) {
-        hostObject.handleEvent(new HostHealthyHeartbeatEvent(hostname, now));
+      if (heartbeat.getNodeStatus().getStatus().equals(HostStatus.Status.HEALTHY)) {
+        hostObject.handleEvent(new HostHealthyHeartbeatEvent(hostname, now,
+            heartbeat.getAgentEnv()));
       } else {
         hostObject.handleEvent(new HostUnhealthyHeartbeatEvent(hostname, now,
             null));
@@ -349,7 +349,7 @@ public class HeartBeatHandler {
 
     hostObject.handleEvent(new HostRegistrationRequestEvent(hostname,
         null != register.getPublicHostname() ? register.getPublicHostname() : hostname,
-        new AgentVersion("v1"), now, register.getHardwareProfile()));
+        new AgentVersion("v1"), now, register.getHardwareProfile(), register.getAgentEnv()));
     RegistrationResponse response = new RegistrationResponse();
     if (cmds.isEmpty()) {
       //No status commands needed let the fsm know that status step is done

+ 18 - 9
ambari-server/src/main/java/org/apache/ambari/server/agent/Register.java

@@ -31,6 +31,7 @@ public class Register {
   private String hostname;
   private HostInfo hardwareProfile;
   private String publicHostname;
+  private AgentEnv agentEnv;
 
   @JsonProperty("responseId")
   public int getResponseId() {
@@ -50,22 +51,22 @@ public class Register {
     return hostname;
   }
   
-  public HostInfo getHardwareProfile() {
-    return hardwareProfile;
-  }
-
-  public void setTimestamp(long timestamp) {
-    this.timestamp = timestamp;
-  }
-
   public void setHostname(String hostname) {
     this.hostname = hostname;
   }
-
+  
+  public HostInfo getHardwareProfile() {
+    return hardwareProfile;
+  }
+  
   public void setHardwareProfile(HostInfo hardwareProfile) {
     this.hardwareProfile = hardwareProfile;
   }
   
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
   public String getPublicHostname() {
     return publicHostname;
   }
@@ -73,6 +74,14 @@ public class Register {
   public void setPublicHostname(String name) {
     publicHostname = name;
   }
+  
+  public AgentEnv getAgentEnv() {
+    return agentEnv;
+  }
+  
+  public void setAgentEnv(AgentEnv env) {
+    agentEnv = env;
+  }
 
   @Override
   public String toString() {

+ 23 - 6
ambari-server/src/main/java/org/apache/ambari/server/controller/HostResponse.java

@@ -18,16 +18,15 @@
 
 package org.apache.ambari.server.controller;
 
-import org.apache.ambari.server.agent.DiskInfo;
-import org.apache.ambari.server.state.AgentVersion;
-import org.apache.ambari.server.state.HostHealthStatus;
-
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
+
+import org.apache.ambari.server.agent.AgentEnv;
+import org.apache.ambari.server.agent.DiskInfo;
+import org.apache.ambari.server.state.AgentVersion;
+import org.apache.ambari.server.state.HostHealthStatus;
 
 public class HostResponse {
 
@@ -84,6 +83,11 @@ public class HostResponse {
    * Last heartbeat timestamp from the Host
    */
   private long lastHeartbeatTime;
+  
+  /**
+   * Last environment information
+   */
+  private AgentEnv lastAgentEnv;
 
   /**
    * Last registration timestamp for the Host
@@ -439,4 +443,17 @@ public class HostResponse {
     this.hostState = hostState;
   }
 
+  
+  public AgentEnv getLastAgentEnv() {
+    return lastAgentEnv;
+  }
+  
+  /**
+   * @param lastAgentEnv
+   */
+  public void setLastAgentEnv(AgentEnv agentEnv) {
+    lastAgentEnv = agentEnv;
+  }
+  
+
 }

+ 4 - 0
ambari-server/src/main/java/org/apache/ambari/server/controller/internal/HostResourceProvider.java

@@ -68,6 +68,8 @@ class HostResourceProvider extends ResourceProviderImpl{
       PropertyHelper.getPropertyId("Hosts", "host_health_report");
   protected static final String HOST_STATE_PROPERTY_ID =
       PropertyHelper.getPropertyId("Hosts", "host_state");
+  protected static final String HOST_LAST_AGENT_ENV_PROPERTY_ID =
+      PropertyHelper.getPropertyId("Hosts", "last_agent_env");
 
   private static Set<String> pkPropertyIds =
       new HashSet<String>(Arrays.asList(new String[]{
@@ -167,6 +169,8 @@ class HostResourceProvider extends ResourceProviderImpl{
           response.getRackInfo(), requestedIds);
       setResourceProperty(resource, HOST_LAST_HEARTBEAT_TIME_PROPERTY_ID,
           response.getLastHeartbeatTime(), requestedIds);
+      setResourceProperty(resource, HOST_LAST_AGENT_ENV_PROPERTY_ID,
+          response.getLastAgentEnv(), requestedIds);
       setResourceProperty(resource, HOST_LAST_REGISTRATION_TIME_PROPERTY_ID,
           response.getLastRegistrationTime(), requestedIds);
       setResourceProperty(resource, HOST_HOST_STATUS_PROPERTY_ID,

+ 11 - 0
ambari-server/src/main/java/org/apache/ambari/server/state/Host.java

@@ -21,6 +21,7 @@ package org.apache.ambari.server.state;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.agent.DiskInfo;
 import org.apache.ambari.server.agent.HostInfo;
 import org.apache.ambari.server.controller.HostResponse;
@@ -209,6 +210,16 @@ public interface Host {
    */
   public void setLastHeartbeatTime(long lastHeartbeatTime);
 
+  /**
+   * Sets the latest agent environment that arrived in a heartbeat.
+   */
+  public void setLastAgentEnv(AgentEnv env);
+  
+  /**
+   * Gets the latest agent environment that arrived in a heartbeat.
+   */
+  public AgentEnv getLastAgentEnv();
+  
   /**
    * Version of the Ambari Agent running on the host
    * @return the agentVersion

+ 12 - 1
ambari-server/src/main/java/org/apache/ambari/server/state/host/HostHealthyHeartbeatEvent.java

@@ -19,16 +19,19 @@
 
 package org.apache.ambari.server.state.host;
 
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.state.HostEvent;
 import org.apache.ambari.server.state.HostEventType;
 
 public class HostHealthyHeartbeatEvent extends HostEvent {
 
   private final long heartbeatTime;
+  private AgentEnv agentEnv = null;
 
-  public HostHealthyHeartbeatEvent(String hostName, long heartbeatTime) {
+  public HostHealthyHeartbeatEvent(String hostName, long heartbeatTime, AgentEnv env) {
     super(hostName, HostEventType.HOST_HEARTBEAT_HEALTHY);
     this.heartbeatTime = heartbeatTime;
+    agentEnv = env;
   }
 
   /**
@@ -37,5 +40,13 @@ public class HostHealthyHeartbeatEvent extends HostEvent {
   public long getHeartbeatTime() {
     return heartbeatTime;
   }
+  
+  /**
+   * @return the heartbeatinfo, if present.  Can return <code>null</code> if
+   * there was no new status.
+   */
+  public AgentEnv getAgentEnv() {
+    return agentEnv;
+  }
 
 }

+ 22 - 2
ambari-server/src/main/java/org/apache/ambari/server/state/host/HostImpl.java

@@ -33,6 +33,7 @@ import com.google.inject.assistedinject.Assisted;
 import com.google.inject.persist.Transactional;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.agent.DiskInfo;
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.agent.HostInfo;
 import org.apache.ambari.server.controller.HostResponse;
 import org.apache.ambari.server.orm.dao.ClusterDAO;
@@ -72,6 +73,7 @@ public class HostImpl implements Host {
   private Clusters clusters;
 
   private long lastHeartbeatTime = 0L;
+  private AgentEnv lastAgentEnv = null;
   private boolean persisted = false;
 
   private static final String HARDWAREISA = "hardware_isa";
@@ -231,6 +233,7 @@ public class HostImpl implements Host {
       host.setLastRegistrationTime(e.registrationTime);
       //Initialize heartbeat time and timeInState with registration time.
       host.setLastHeartbeatTime(e.registrationTime);
+      host.setLastAgentEnv(e.agentEnv);
       host.setTimeInState(e.registrationTime);
       host.setAgentVersion(e.agentVersion);
       host.setPublicHostName(e.publicHostName);
@@ -270,8 +273,10 @@ public class HostImpl implements Host {
       long heartbeatTime = 0;
       switch (event.getType()) {
         case HOST_HEARTBEAT_HEALTHY:
-          heartbeatTime =
-            ((HostHealthyHeartbeatEvent)event).getHeartbeatTime();
+          HostHealthyHeartbeatEvent hhevent = (HostHealthyHeartbeatEvent) event;
+          heartbeatTime = hhevent.getHeartbeatTime();
+          if (null != hhevent.getAgentEnv())
+            host.setLastAgentEnv(hhevent.getAgentEnv());
           break;
         case HOST_HEARTBEAT_UNHEALTHY:
           heartbeatTime =
@@ -284,6 +289,7 @@ public class HostImpl implements Host {
         LOG.error("heartbeatTime = 0 !!!");
         // TODO handle error
       }
+      // host.setLastHeartbeatState(new Object());
       host.setLastHeartbeatTime(heartbeatTime);
     }
   }
@@ -444,6 +450,19 @@ public class HostImpl implements Host {
     }
   }
 
+  /**
+   * @param hostInfo
+   */
+  @Override
+  public void setLastAgentEnv(AgentEnv env) {
+    lastAgentEnv = env;
+  }
+  
+  @Override
+  public AgentEnv getLastAgentEnv() {
+    return lastAgentEnv;
+  }
+
   @Override
   public HostState getState() {
     try {
@@ -915,6 +934,7 @@ public class HostImpl implements Host {
       r.setIpv4(getIPv4());
       r.setIpv6(getIPv6());
       r.setLastHeartbeatTime(getLastHeartbeatTime());
+      r.setLastAgentEnv(lastAgentEnv);
       r.setLastRegistrationTime(getLastRegistrationTime());
       r.setOsArch(getOsArch());
       r.setOsInfo(getOsInfo());

+ 6 - 3
ambari-server/src/main/java/org/apache/ambari/server/state/host/HostRegistrationRequestEvent.java

@@ -18,6 +18,7 @@
 
 package org.apache.ambari.server.state.host;
 
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.agent.HostInfo;
 import org.apache.ambari.server.state.AgentVersion;
 import org.apache.ambari.server.state.HostEvent;
@@ -29,19 +30,21 @@ public class HostRegistrationRequestEvent extends HostEvent {
   final HostInfo hostInfo;
   final AgentVersion agentVersion;
   final String publicHostName;
+  final AgentEnv agentEnv;
 
   public HostRegistrationRequestEvent(String hostName,
-      AgentVersion agentVersion, long registrationTime, HostInfo hostInfo) {
-    this(hostName, hostName, agentVersion, registrationTime, hostInfo);
+      AgentVersion agentVersion, long registrationTime, HostInfo hostInfo, AgentEnv env) {
+    this(hostName, hostName, agentVersion, registrationTime, hostInfo, env);
   }
   
   public HostRegistrationRequestEvent(String hostName, String publicName,
-      AgentVersion agentVersion, long registrationTime, HostInfo hostInfo) {
+      AgentVersion agentVersion, long registrationTime, HostInfo hostInfo, AgentEnv env) {
     super(hostName, HostEventType.HOST_REGISTRATION_REQUEST);
     this.registrationTime = registrationTime;
     this.hostInfo = hostInfo;
     this.agentVersion = agentVersion;
     this.publicHostName = (null == publicName) ? hostName : publicName;
+    this.agentEnv = env;
   }
 
 

+ 1 - 0
ambari-server/src/main/resources/properties.json

@@ -27,6 +27,7 @@
         "Hosts/os_type",
         "Hosts/rack_info",
         "Hosts/last_heartbeat_time",
+        "Hosts/last_agent_env",
         "Hosts/last_registration_time",
         "Hosts/disk_info",
         "Hosts/host_status",

+ 21 - 5
ambari-server/src/test/java/org/apache/ambari/server/state/cluster/ClusterTest.java

@@ -21,15 +21,14 @@ package org.apache.ambari.server.state.cluster;
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.persist.PersistService;
 import junit.framework.Assert;
 
 import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.agent.DiskInfo;
 import org.apache.ambari.server.agent.HostInfo;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
@@ -55,6 +54,10 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.persist.PersistService;
+
 public class ClusterTest {
 
   private Clusters clusters;
@@ -139,11 +142,24 @@ public class ClusterTest {
         "5000000", "4000000", "10%", "size", "fstype"));
     hostInfo.setMounts(mounts);
 
+    AgentEnv agentEnv = new AgentEnv();
+    Map<String, String> etcDirs = new HashMap<String, String>();
+    etcDirs.put("hadoop", "not_exist");
+    agentEnv.setEtcDirs(etcDirs);
+    
+    Map<String, String> varRunDirs = new HashMap<String, String>();
+    varRunDirs.put("hadoop", "not_exist");
+    agentEnv.setVarRunDirs(varRunDirs);
+    
+    Map<String, String> varLogDirs = new HashMap<String, String>();
+    varLogDirs.put("hadoop", "not_exist");
+    agentEnv.setVarLogDirs(varLogDirs);
+    
     AgentVersion agentVersion = new AgentVersion("0.0.x");
     long currentTime = 1001;
 
     clusters.getHost("h1").handleEvent(new HostRegistrationRequestEvent(
-        "h1", agentVersion, currentTime, hostInfo));
+        "h1", agentVersion, currentTime, hostInfo, agentEnv));
 
     Assert.assertEquals(HostState.WAITING_FOR_HOST_STATUS_UPDATES,
         clusters.getHost("h1").getState());
@@ -152,7 +168,7 @@ public class ClusterTest {
 
     try {
       clusters.getHost("h1").handleEvent(
-          new HostHealthyHeartbeatEvent("h1", currentTime));
+          new HostHealthyHeartbeatEvent("h1", currentTime, null));
       fail("Exception should be thrown on invalid event");
     }
     catch (InvalidStateTransitionException e) {

+ 7 - 3
ambari-server/src/test/java/org/apache/ambari/server/state/host/HostTest.java

@@ -29,10 +29,10 @@ import java.util.List;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.agent.ActionQueue;
+import org.apache.ambari.server.agent.AgentEnv;
 import org.apache.ambari.server.agent.DiskInfo;
 import org.apache.ambari.server.agent.HeartBeatHandler;
 import org.apache.ambari.server.agent.HostInfo;
-import org.apache.ambari.server.agent.TestHeartbeatHandler;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.dao.HostDAO;
@@ -142,15 +142,19 @@ public class HostTest {
 
     AgentVersion agentVersion = null;
     long currentTime = System.currentTimeMillis();
+    
+    AgentEnv agentEnv = new AgentEnv();
 
     HostRegistrationRequestEvent e =
         new HostRegistrationRequestEvent("foo", agentVersion, currentTime,
-            info);
+            info, agentEnv);
     if (!firstReg) {
       Assert.assertTrue(host.isPersisted());
     }
     host.handleEvent(e);
     Assert.assertEquals(currentTime, host.getLastRegistrationTime());
+    
+    Assert.assertNotNull(host.getLastAgentEnv());
 
     HostEntity entity = hostDAO.findByName(host.getHostName());
     Assert.assertEquals(currentTime,
@@ -173,7 +177,7 @@ public class HostTest {
   private void sendHealthyHeartbeat(Host host, long counter)
       throws Exception {
     HostHealthyHeartbeatEvent e = new HostHealthyHeartbeatEvent(
-        host.getHostName(), counter);
+        host.getHostName(), counter, null);
     host.handleEvent(e);
   }