You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sw...@apache.org on 2014/11/26 22:32:49 UTC

ambari git commit: AMBARI-7679. Add psutil based resource monitoring to collect host metrics. Unit tests and fixes.

Repository: ambari
Updated Branches:
  refs/heads/branch-metrics-dev 43460193b -> d6cea463e


AMBARI-7679. Add psutil based resource monitoring to collect host metrics. Unit tests and fixes.


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

Branch: refs/heads/branch-metrics-dev
Commit: d6cea463efd3df60819a2994331d31a74ddb99a9
Parents: 4346019
Author: Siddharth Wagle <sw...@hortonworks.com>
Authored: Wed Nov 26 13:32:37 2014 -0800
Committer: Siddharth Wagle <sw...@hortonworks.com>
Committed: Wed Nov 26 13:32:37 2014 -0800

----------------------------------------------------------------------
 .../conf/unix/ambari-metrics-monitor            |   3 +-
 .../ambari-metrics-host-monitoring/pom.xml      |  67 +++++++++
 .../main/python/core/application_metric_map.py  | 103 +++++++-------
 .../src/main/python/core/controller.py          |  23 ----
 .../src/main/python/core/emitter.py             |  61 ++++-----
 .../src/main/python/core/metric_collector.py    |   6 +-
 .../src/main/python/main.py                     |  33 +++--
 .../src/main/python/psutil/build.out            | 137 -------------------
 .../python/core/TestApplicationMetricMap.py     |  67 +++++++++
 .../src/test/python/core/TestEmitter.py         |  78 +++++++++++
 .../src/test/python/core/TestHostInfo.py        |  97 +++++++++++++
 .../src/test/python/core/TestMetricCollector.py |  49 +++++++
 .../src/test/python/unitTests.py                | 133 ++++++++++++++++++
 13 files changed, 599 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/conf/unix/ambari-metrics-monitor
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/conf/unix/ambari-metrics-monitor b/ambari-metrics/ambari-metrics-host-monitoring/conf/unix/ambari-metrics-monitor
index 45e7a44..fd086d9 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/conf/unix/ambari-metrics-monitor
+++ b/ambari-metrics/ambari-metrics-host-monitoring/conf/unix/ambari-metrics-monitor
@@ -7,7 +7,7 @@
 # (the "License"); you may not use this file except in compliance with
 # the License.  You may obtain a copy of the License at
 #
-#     ht${PYTHON}w.apache.org/licenses/LICENSE-2.0
+#     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,
@@ -148,6 +148,7 @@ case "$1" in
     fi
 
     echo "Metric Monitor successfully started"
+    echo "Server log at: ${OUTFILE}"
   ;;
   status)
     if [ -f ${PIDFILE} ]; then

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/pom.xml
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/pom.xml b/ambari-metrics/ambari-metrics-host-monitoring/pom.xml
index 0dac918..8c4d112 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/pom.xml
+++ b/ambari-metrics/ambari-metrics-host-monitoring/pom.xml
@@ -197,6 +197,73 @@
           </mappings>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <version>1.7</version>
+        <executions>
+          <execution>
+            <id>psutils-compile</id>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target name="psutils-compile">
+                <exec dir="${basedir}/src/main/python/psutil" executable="python" failonerror="true">
+                  <arg value="setup.py" />
+                  <arg value="build" />
+                  <arg value="--build-base" />
+                  <arg value="${basedir}/target/psutil_build" />
+                </exec>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+       
+       
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <configuration>
+              <executable>python</executable>
+              <workingDirectory>src/test/python</workingDirectory>
+              <arguments>
+                <argument>unitTests.py</argument>
+              </arguments>
+              <environmentVariables>
+                <PYTHONPATH>../../main/python:$PYTHONPATH</PYTHONPATH>
+              </environmentVariables>
+            </configuration>
+            <id>python-test</id>
+            <phase>test</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/main/python/psutil/**</exclude>
+            <exclude>.pydevproject</exclude>
+          </excludes>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>test</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/application_metric_map.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/application_metric_map.py b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/application_metric_map.py
index 0b3f1dd..1be6fa2 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/application_metric_map.py
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/application_metric_map.py
@@ -35,34 +35,31 @@ class ApplicationMetricMap:
   metric_value   => numeric value
   """
 
-  app_metric_map = {}
 
   def __init__(self, hostname, ip_address):
     self.hostname = hostname
     self.ip_address = ip_address
     self.lock = RLock()
+    self.app_metric_map = {}
   pass
 
-  def acquire_lock(self):
-    self.lock.acquire()
-
-  def release_lock(self):
-    self.lock.release()
-
-  def put_metric(self, application_id, metric_id, timestamp, value):
-    metric_map = self.app_metric_map.get(application_id)
-    if not metric_map:
-      metric_map = { metric_id : { timestamp : value } }
-      self.app_metric_map[ application_id ] = metric_map
-    else:
-      metric_id_map = metric_map.get(metric_id)
-      if not metric_id_map:
-        metric_id_map = { timestamp : value }
-        metric_map[ metric_id ] = metric_id_map
-      else:
-        metric_map[ metric_id ].update( { timestamp : value } )
-      pass
-    pass
+  def put_metric(self, application_id, metric_id_to_value_map, timestamp):
+    with self.lock:
+      for metric_name, value in metric_id_to_value_map.iteritems():
+      
+        metric_map = self.app_metric_map.get(application_id)
+        if not metric_map:
+          metric_map = { metric_name : { timestamp : value } }
+          self.app_metric_map[ application_id ] = metric_map
+        else:
+          metric_id_map = metric_map.get(metric_name)
+          if not metric_id_map:
+            metric_id_map = { timestamp : value }
+            metric_map[ metric_name ] = metric_id_map
+          else:
+            metric_map[ metric_name ].update( { timestamp : value } )
+          pass
+        pass
   pass
 
   def delete_application_metrics(self, app_id):
@@ -76,41 +73,42 @@ class ApplicationMetricMap:
     {"metrics":[{"hostname":"a","metricname":"b","appid":"c",
     "instanceid":"d","starttime":"e","metrics":{"t":"v"}}]}
     """
-
-    timeline_metrics = { "metrics" : [] }
-    local_metric_map = {}
-
-    if application_id:
-      if self.app_metric_map.has_key(application_id):
-        local_metric_map = { application_id : self.app_metric_map[application_id] }
+    with self.lock:
+      timeline_metrics = { "metrics" : [] }
+      local_metric_map = {}
+  
+      if application_id:
+        if self.app_metric_map.has_key(application_id):
+          local_metric_map = { application_id : self.app_metric_map[application_id] }
+        else:
+          logger.info("application_id: {0}, not present in the map.".format(application_id))
       else:
-        logger.info("application_id: {0}, not present in the map.".format(application_id))
-    else:
-      local_metric_map = self.app_metric_map.copy()
-    pass
-
-    for appId, metrics in local_metric_map.iteritems():
-      for metricId, metricData in dict(metrics).iteritems():
-        # Create a timeline metric object
-        timeline_metric = {
-          "hostname" : self.hostname,
-          "metricname" : metricId,
-          "appid" : "HOST",
-          "instanceid" : "",
-          "starttime" : self.get_start_time(appId, metricId),
-          "metrics" : metricData
-        }
-        timeline_metrics[ "metrics" ].append( timeline_metric )
+        local_metric_map = self.app_metric_map.copy()
       pass
-    pass
-
-    return json.dumps(timeline_metrics)
+  
+      for appId, metrics in local_metric_map.iteritems():
+        for metricId, metricData in dict(metrics).iteritems():
+          # Create a timeline metric object
+          timeline_metric = {
+            "hostname" : self.hostname,
+            "metricname" : metricId,
+            "appid" : "HOST",
+            "instanceid" : "",
+            "starttime" : self.get_start_time(appId, metricId),
+            "metrics" : metricData
+          }
+          timeline_metrics[ "metrics" ].append( timeline_metric )
+        pass
+      pass
+      return json.dumps(timeline_metrics) if len(timeline_metrics[ "metrics" ]) > 0 else None
   pass
 
   def get_start_time(self, app_id, metric_id):
-    if self.app_metric_map.has_key(app_id):
-      if self.app_metric_map.get(app_id).has_key(metric_id):
-        return self.app_metric_map.get(app_id).get(metric_id).iteritems().next()[0]
+    with self.lock:
+      if self.app_metric_map.has_key(app_id):
+        if self.app_metric_map.get(app_id).has_key(metric_id):
+          metrics = self.app_metric_map.get(app_id).get(metric_id)
+          return min(metrics.iterkeys())
   pass
 
   def format_app_id(self, app_id, instance_id = None):
@@ -127,5 +125,6 @@ class ApplicationMetricMap:
   pass
 
   def clear(self):
-    self.app_metric_map.clear()
+    with self.lock:
+      self.app_metric_map.clear()
   pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/controller.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/controller.py b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/controller.py
index d5299d1..51f0980 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/controller.py
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/controller.py
@@ -19,14 +19,11 @@ limitations under the License.
 '''
 
 import logging
-import signal
 import threading
 import time
-import sys
 from Queue import Queue
 from threading import Timer
 from application_metric_map import ApplicationMetricMap
-from config_reader import Configuration
 from event_definition import HostMetricCollectEvent, ProcessMetricCollectEvent
 from metric_collector import MetricsCollector
 from emitter import Emitter
@@ -104,23 +101,3 @@ class Controller(threading.Thread):
 
   def start_emitter(self):
     self.emitter.start()
-
-def main(argv=None):
-  # Allow Ctrl-C
-  signal.signal(signal.SIGINT, signal.SIG_DFL)
-
-  config = Configuration()
-  collector = Controller(config)
-
-  logger.setLevel(config.get_log_level())
-  #formatter = logging.Formatter("%(asctime)s %(filename)s:%(lineno)d - %(message)s")
-  stream_handler = logging.StreamHandler(sys.stdout)
-  stream_handler.setFormatter(formatter)
-  logger.addHandler(stream_handler)
-  logger.info('Starting Server RPC Thread: {0}'.format(sys.argv))
-
-  collector.start()
-  collector.start_emitter()
-
-if __name__ == '__main__':
-  main()

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/emitter.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/emitter.py b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/emitter.py
index 7045c2e..be83250 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/emitter.py
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/emitter.py
@@ -24,15 +24,14 @@ import time
 import urllib2
 
 logger = logging.getLogger()
-COLLECTOR_URL = "http://{0}/ws/v1/timeline/metrics"
-RETRY_SLEEP_INTERVAL = 5
-MAX_RETRY_COUNT = 3
 
 class Emitter(threading.Thread):
+  COLLECTOR_URL = "http://{0}/ws/v1/timeline/metrics"
+  RETRY_SLEEP_INTERVAL = 5
+  MAX_RETRY_COUNT = 3
   """
   Wake up every send interval seconds and empty the application metric map.
   """
-
   def __init__(self, config, application_metric_map):
     threading.Thread.__init__(self)
     logger.debug('Initializing Emitter thread.')
@@ -45,40 +44,40 @@ class Emitter(threading.Thread):
     logger.info('Running Emitter thread: %s' % threading.currentThread().getName())
     while True:
       try:
-        retry_count = 0
-        while retry_count < MAX_RETRY_COUNT:
-          self.application_metric_map.acquire_lock()
-          json_data = self.application_metric_map.flatten()
-          if json_data is None:
-            logger.info("Nothing to emit, resume waiting.")
-            break
-          pass
-          response = self.push_metrics(json_data)
-
-          if response and response.getcode() == 200:
-            retry_count = MAX_RETRY_COUNT
-            self.application_metric_map.clear()
-            self.application_metric_map.release_lock()
-          else:
-            logger.warn("Error sending metrics to server. Retrying after {0} "
-                        "...".format(RETRY_SLEEP_INTERVAL))
-            self.application_metric_map.release_lock()
-            retry_count += 1
-            time.sleep(RETRY_SLEEP_INTERVAL)
-          pass
-        pass
-
+        self.submit_metrics()
         time.sleep(self.send_interval)
       except Exception, e:
         logger.warn('Unable to emit events. %s' % str(e))
-        time.sleep(RETRY_SLEEP_INTERVAL)
-        logger.info('Retrying emit after %s seconds.' % RETRY_SLEEP_INTERVAL)
+        time.sleep(self.RETRY_SLEEP_INTERVAL)
+        logger.info('Retrying emit after %s seconds.' % self.RETRY_SLEEP_INTERVAL)
     pass
-
+  
+  def submit_metrics(self):
+    retry_count = 0
+    while retry_count < self.MAX_RETRY_COUNT:
+      json_data = self.application_metric_map.flatten()
+      if json_data is None:
+        logger.info("Nothing to emit, resume waiting.")
+        break
+      pass
+      response = self.push_metrics(json_data)
+  
+      if response and response.getcode() == 200:
+        retry_count = self.MAX_RETRY_COUNT
+        self.application_metric_map.clear()
+      else:
+        logger.warn("Error sending metrics to server. Retrying after {0} "
+                    "...".format(self.RETRY_SLEEP_INTERVAL))
+        retry_count += 1
+        time.sleep(self.RETRY_SLEEP_INTERVAL)
+      pass
+    pass
+  
   def push_metrics(self, data):
     headers = {"Content-Type" : "application/json", "Accept" : "*/*"}
-    server = COLLECTOR_URL.format(self.collector_address.strip())
+    server = self.COLLECTOR_URL.format(self.collector_address.strip())
     logger.info("server: %s" % server)
+    logger.debug("message to sent: %s" % data)
     req = urllib2.Request(server, data, headers)
     response = urllib2.urlopen(req, timeout=int(self.send_interval - 10))
     if response:

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/metric_collector.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/metric_collector.py b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/metric_collector.py
index d4b96dc..8b82e20 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/metric_collector.py
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/core/metric_collector.py
@@ -77,11 +77,7 @@ class MetricsCollector():
     pass
 
     if metrics:
-      self.application_metric_map.acquire_lock()
-      for metric_name, value in metrics.iteritems():
-        self.application_metric_map.put_metric(DEFAULT_HOST_APP_ID, metric_name, startTime, value)
-      pass
-      self.application_metric_map.release_lock()
+      self.application_metric_map.put_metric(DEFAULT_HOST_APP_ID, metrics, startTime)
     pass
 
   def process_process_collection_event(self, event):

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/main.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/main.py b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/main.py
index 365e1df..09ae7e4 100644
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/main.py
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/main.py
@@ -19,7 +19,8 @@ limitations under the License.
 '''
 
 import core
-from core.controller import Controller, Configuration
+from core.controller import Controller
+from core.config_reader import Configuration
 import logging
 import signal
 import sys
@@ -31,18 +32,32 @@ def main(argv=None):
   signal.signal(signal.SIGINT, signal.SIG_DFL)
 
   config = Configuration()
-  collector = Controller(config)
+  controller = Controller(config)
+  
+  _init_logging(config)
+  
+  logger.info('Starting Server RPC Thread: %s' % ' '.join(sys.argv))
+  controller.start()
+  controller.start_emitter()
 
-  logger.setLevel(logging.DEBUG)
-  formatter = logging.Formatter("%(asctime)s %(filename)s:%(lineno)d - %(message)s")
+def _init_logging(config):
+  _levels = {
+          'DEBUG': logging.DEBUG,
+          'INFO': logging.INFO,
+          'WARNING': logging.WARNING,
+          'ERROR': logging.ERROR,
+          'CRITICAL': logging.CRITICAL,
+          'NOTSET' : logging.NOTSET
+          }
+  level = logging.INFO
+  if config.get_log_level() in _levels:
+    level = _levels.get(config.get_log_level())
+  logger.setLevel(level)
+  formatter = logging.Formatter("%(asctime)s [%(levelname)s] %(filename)s:%(lineno)d - %(message)s")
   stream_handler = logging.StreamHandler()
   stream_handler.setFormatter(formatter)
   logger.addHandler(stream_handler)
-  logger.info('Starting Server RPC Thread: %s' % ' '.join(sys.argv))
-
-  collector.start()
-  collector.start_emitter()
-
+  
 
 if __name__ == '__main__':
   main()

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/psutil/build.out
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/psutil/build.out b/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/psutil/build.out
deleted file mode 100644
index cdedb21..0000000
--- a/ambari-metrics/ambari-metrics-host-monitoring/src/main/python/psutil/build.out
+++ /dev/null
@@ -1,137 +0,0 @@
-rm -f `find . -type f -name \*.py[co]`
-rm -f `find . -type f -name \*.so`
-rm -f `find . -type f -name .\*~`
-rm -f `find . -type f -name \*.orig`
-rm -f `find . -type f -name \*.bak`
-rm -f `find . -type f -name \*.rej`
-rm -rf `find . -type d -name __pycache__`
-rm -rf *.egg-info
-rm -rf *\estfile*
-rm -rf build
-rm -rf dist
-rm -rf docs/_build
-python setup.py build
-running build
-running build_py
-creating build
-creating build/lib.macosx-10.8-intel-2.7
-creating build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/__init__.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_common.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_compat.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_psbsd.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_pslinux.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_psosx.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_psposix.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_pssunos.py -> build/lib.macosx-10.8-intel-2.7/psutil
-copying psutil/_pswindows.py -> build/lib.macosx-10.8-intel-2.7/psutil
-running build_ext
-building '_psutil_osx' extension
-creating build/temp.macosx-10.8-intel-2.7
-creating build/temp.macosx-10.8-intel-2.7/psutil
-creating build/temp.macosx-10.8-intel-2.7/psutil/arch
-creating build/temp.macosx-10.8-intel-2.7/psutil/arch/osx
-clang -fno-strict-aliasing -fno-common -dynamic -g -Os -pipe -fno-common -fno-strict-aliasing -fwrapv -mno-fused-madd -DENABLE_DTRACE -DMACOSX -DNDEBUG -Wall -Wstrict-prototypes -Wshorten-64-to-32 -DNDEBUG -g -Os -Wall -Wstrict-prototypes -DENABLE_DTRACE -arch i386 -arch x86_64 -pipe -I/System/Library/Frameworks/Python.framework/Versions/2.7/include/python2.7 -c psutil/_psutil_osx.c -o build/temp.macosx-10.8-intel-2.7/psutil/_psutil_osx.o
-clang: warning: argument unused during compilation: '-mno-fused-madd'
-psutil/_psutil_osx.c:341:48: warning: format specifies type 'unsigned long' but the argument has type 'vm_address_t' (aka 'unsigned int') [-Wformat]
-            sprintf(addr_str, "%016lx-%016lx", address, address + size);
-                               ~~~~~~          ^~~~~~~
-                               %016x
-/usr/include/secure/_stdio.h:49:56: note: expanded from macro 'sprintf'
-  __builtin___sprintf_chk (str, 0, __darwin_obsz(str), __VA_ARGS__)
-                                                       ^
-psutil/_psutil_osx.c:341:57: warning: format specifies type 'unsigned long' but the argument has type 'unsigned int' [-Wformat]
-            sprintf(addr_str, "%016lx-%016lx", address, address + size);
-                                      ~~~~~~            ^~~~~~~~~~~~~~
-                                      %016x
-/usr/include/secure/_stdio.h:49:56: note: expanded from macro 'sprintf'
-  __builtin___sprintf_chk (str, 0, __darwin_obsz(str), __VA_ARGS__)
-                                                       ^
-2 warnings generated.
-psutil/_psutil_osx.c:126:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:168:24: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    ret = proc_pidpath(pid, &buf, sizeof(buf));
-          ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:211:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:229:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:250:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:271:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:306:42: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    err = task_for_pid(mach_task_self(), pid, &task);
-          ~~~~~~~~~~~~                   ^~~
-psutil/_psutil_osx.c:350:39: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-            err = proc_regionfilename(pid, address, buf, sizeof(buf));
-                  ~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:502:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:808:25: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    num = getfsstat(fs, len, MNT_NOWAIT);
-          ~~~~~~~~~     ^~~
-psutil/_psutil_osx.c:906:31: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    if (psutil_get_kinfo_proc(pid, &kp) == -1) {
-        ~~~~~~~~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:942:42: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    err = task_for_pid(mach_task_self(), pid, &task);
-          ~~~~~~~~~~~~                   ^~~
-psutil/_psutil_osx.c:1055:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, NULL, 0);
-                     ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1068:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, fds_pointer,
-                     ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1085:33: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-            nb = proc_pidfdinfo(pid,
-                 ~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1191:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, NULL, 0);
-                     ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1201:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, fds_pointer,
-                     ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1219:33: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-            nb = proc_pidfdinfo(pid, fdp_pointer->proc_fd,
-                 ~~~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1393:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, NULL, 0);
-                     ~~~~~~~~~~~~ ^~~
-psutil/_psutil_osx.c:1402:35: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    pidinfo_result = proc_pidinfo(pid, PROC_PIDLISTFDS, 0, fds_pointer,
-                     ~~~~~~~~~~~~ ^~~
-20 warnings generated.
-clang -fno-strict-aliasing -fno-common -dynamic -g -Os -pipe -fno-common -fno-strict-aliasing -fwrapv -mno-fused-madd -DENABLE_DTRACE -DMACOSX -DNDEBUG -Wall -Wstrict-prototypes -Wshorten-64-to-32 -DNDEBUG -g -Os -Wall -Wstrict-prototypes -DENABLE_DTRACE -arch i386 -arch x86_64 -pipe -I/System/Library/Frameworks/Python.framework/Versions/2.7/include/python2.7 -c psutil/_psutil_common.c -o build/temp.macosx-10.8-intel-2.7/psutil/_psutil_common.o
-clang: warning: argument unused during compilation: '-mno-fused-madd'
-clang -fno-strict-aliasing -fno-common -dynamic -g -Os -pipe -fno-common -fno-strict-aliasing -fwrapv -mno-fused-madd -DENABLE_DTRACE -DMACOSX -DNDEBUG -Wall -Wstrict-prototypes -Wshorten-64-to-32 -DNDEBUG -g -Os -Wall -Wstrict-prototypes -DENABLE_DTRACE -arch i386 -arch x86_64 -pipe -I/System/Library/Frameworks/Python.framework/Versions/2.7/include/python2.7 -c psutil/arch/osx/process_info.c -o build/temp.macosx-10.8-intel-2.7/psutil/arch/osx/process_info.o
-clang: warning: argument unused during compilation: '-mno-fused-madd'
-psutil/arch/osx/process_info.c:40:21: warning: implicit conversion loses integer precision: 'long' to 'pid_t' (aka 'int') [-Wshorten-64-to-32]
-    kill_ret = kill(pid , 0);
-               ~~~~ ^~~
-psutil/arch/osx/process_info.c:176:14: warning: implicit conversion loses integer precision: 'long' to 'int' [-Wshorten-64-to-32]
-    mib[2] = pid;
-           ~ ^~~
-psutil/arch/osx/process_info.c:194:11: warning: implicit conversion loses integer precision: 'size_t' (aka 'unsigned long') to 'int' [-Wshorten-64-to-32]
-    len = strlen(arg_ptr);
-        ~ ^~~~~~~~~~~~~~~
-3 warnings generated.
-clang -bundle -undefined dynamic_lookup -Wl,-F. -arch i386 -arch x86_64 build/temp.macosx-10.8-intel-2.7/psutil/_psutil_osx.o build/temp.macosx-10.8-intel-2.7/psutil/_psutil_common.o build/temp.macosx-10.8-intel-2.7/psutil/arch/osx/process_info.o -o build/lib.macosx-10.8-intel-2.7/_psutil_osx.so -framework CoreFoundation -framework IOKit
-building '_psutil_posix' extension
-clang -fno-strict-aliasing -fno-common -dynamic -g -Os -pipe -fno-common -fno-strict-aliasing -fwrapv -mno-fused-madd -DENABLE_DTRACE -DMACOSX -DNDEBUG -Wall -Wstrict-prototypes -Wshorten-64-to-32 -DNDEBUG -g -Os -Wall -Wstrict-prototypes -DENABLE_DTRACE -arch i386 -arch x86_64 -pipe -I/System/Library/Frameworks/Python.framework/Versions/2.7/include/python2.7 -c psutil/_psutil_posix.c -o build/temp.macosx-10.8-intel-2.7/psutil/_psutil_posix.o
-clang: warning: argument unused during compilation: '-mno-fused-madd'
-psutil/_psutil_posix.c:29:42: warning: implicit conversion loses integer precision: 'long' to 'id_t' (aka 'unsigned int') [-Wshorten-64-to-32]
-    priority = getpriority(PRIO_PROCESS, pid);
-               ~~~~~~~~~~~               ^~~
-psutil/_psutil_posix.c:49:40: warning: implicit conversion loses integer precision: 'long' to 'id_t' (aka 'unsigned int') [-Wshorten-64-to-32]
-    retval = setpriority(PRIO_PROCESS, pid, priority);
-             ~~~~~~~~~~~               ^~~
-2 warnings generated.
-clang -bundle -undefined dynamic_lookup -Wl,-F. -arch i386 -arch x86_64 build/temp.macosx-10.8-intel-2.7/psutil/_psutil_posix.o -o build/lib.macosx-10.8-intel-2.7/_psutil_posix.so

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestApplicationMetricMap.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestApplicationMetricMap.py b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestApplicationMetricMap.py
new file mode 100644
index 0000000..e653c48
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestApplicationMetricMap.py
@@ -0,0 +1,67 @@
+#!/usr/bin/env python
+
+'''
+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 json
+import logging
+from unittest import TestCase
+from application_metric_map import ApplicationMetricMap
+
+logger = logging.getLogger()
+
+class TestApplicationMetricMap(TestCase):
+  
+  def testApplicationMetricMap(self):
+    application_metric_map = ApplicationMetricMap("host1", "10.10.10.10")
+    
+    application_id = application_metric_map.format_app_id("A","1")
+    timestamp = int(round(1415390657.3806491 * 1000))
+    
+    metrics = {}
+    metrics.update({"b" : 'bv'})
+    
+    application_metric_map.put_metric(application_id, metrics, timestamp)
+    application_metric_map.put_metric(application_id, metrics, timestamp + 1)
+    application_metric_map.put_metric(application_id, metrics, timestamp + 2)
+    application_metric_map.put_metric(application_id, metrics, timestamp + 3)
+    
+    p = json.loads(application_metric_map.flatten(application_id))
+    self.assertEqual(len(p['metrics']), 1)
+    self.assertEqual(p['metrics'][0]['metricname'], "b")
+#     self.assertEqual(p['metrics'][0]['appid'], application_id)
+    self.assertEqual(p['metrics'][0]['hostname'], "host1")
+    self.assertEqual(len(p['metrics'][0]['metrics']), 4)
+    self.assertEqual(p['metrics'][0]['metrics'][str(timestamp)], 'bv')
+    
+    self.assertEqual(application_metric_map.get_start_time(application_id, "b"), timestamp)
+    
+    metrics = {}
+    metrics.update({"b" : 'bv'})
+    metrics.update({"a" : 'av'})
+    application_metric_map.put_metric(application_id, metrics, timestamp)
+    p = json.loads(application_metric_map.flatten(application_id))
+    self.assertEqual(len(p['metrics']), 2)
+    self.assertTrue((p['metrics'][0]['metricname'] == 'a' and p['metrics'][1]['metricname'] == 'b') or 
+                    (p['metrics'][1]['metricname'] == 'a' and p['metrics'][0]['metricname'] == 'b'))
+    
+    
+  def testEmptyMapReturnNone(self):
+    application_metric_map = ApplicationMetricMap("host","10.10.10.10")
+    self.assertTrue(application_metric_map.flatten() == None)
+    

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestEmitter.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestEmitter.py b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestEmitter.py
new file mode 100644
index 0000000..05362bf
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestEmitter.py
@@ -0,0 +1,78 @@
+#!/usr/bin/env python
+
+'''
+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 logging
+from unittest import TestCase
+
+from application_metric_map import ApplicationMetricMap
+from config_reader import Configuration
+from emitter import Emitter
+
+from mock.mock import patch, MagicMock
+
+import json
+import urllib2
+
+logger = logging.getLogger()
+
+class TestEmitter(TestCase):
+  
+  @patch("urllib2.urlopen")
+  def testJavaHomeAvailableCheck(self, url_open_mock):
+    url_open_mock.return_value = MagicMock()
+    url_open_mock.return_value.getcode.return_value = 200
+    self.assertEqual(urllib2.urlopen(None, None).getcode(), 200)
+    url_open_mock.reset_mock()
+    
+    config = Configuration()
+    application_metric_map = ApplicationMetricMap("host","10.10.10.10")
+    application_metric_map.clear()
+    application_metric_map.put_metric("APP1", {"metric1":1}, 1)
+    emitter = Emitter(config, application_metric_map)
+    emitter.submit_metrics()
+    
+    self.assertEqual(url_open_mock.call_count, 1)
+    self.assertUrlData(url_open_mock)
+    
+    
+  @patch("urllib2.urlopen")
+  def testRetryFetch(self, url_open_mock):
+    
+    config = Configuration()
+    application_metric_map = ApplicationMetricMap("host","10.10.10.10")
+    application_metric_map.clear()
+    application_metric_map.put_metric("APP1", {"metric1":1}, 1)
+    emitter = Emitter(config, application_metric_map)
+    emitter.RETRY_SLEEP_INTERVAL = .001
+    emitter.submit_metrics()
+    
+    self.assertEqual(url_open_mock.call_count, 3)
+    self.assertUrlData(url_open_mock)
+    
+
+  def assertUrlData(self, url_open_mock):
+    self.assertEqual(len(url_open_mock.call_args), 2)
+    data = url_open_mock.call_args[0][0].data
+    self.assertTrue(data is not None)
+    
+    metrics = json.loads(data)
+    self.assertEqual(len(metrics['metrics']), 1)
+    self.assertEqual(metrics['metrics'][0]['metricname'],'metric1')
+    self.assertEqual(metrics['metrics'][0]['starttime'],1)
+    pass
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestHostInfo.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestHostInfo.py b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestHostInfo.py
new file mode 100644
index 0000000..e1baabf
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestHostInfo.py
@@ -0,0 +1,97 @@
+#!/usr/bin/env python
+
+'''
+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 logging
+from host_info import HostInfo
+
+from unittest import TestCase
+from mock.mock import patch
+
+logger = logging.getLogger()
+
+class TestHostInfo(TestCase):
+  
+  @patch("os.getloadavg")
+  @patch("psutil.cpu_times")
+  def testCpuTimes(self, cp_mock, avg_mock):
+    
+    cp = cp_mock.return_value
+    cp.user = "user"
+    cp.system = "system"
+    cp.idle = "idle"
+    cp.nice = "nice"
+    cp.iowait = "iowait"
+    cp.irq = "irq"
+    cp.softirq = "softirq"
+    avg_mock.return_value  = [13, 13, 13]
+    
+    hostinfo = HostInfo()
+    
+    cpu = hostinfo.get_cpu_times()
+    
+    self.assertEqual(cpu['cpu_user'], 'user')
+    self.assertEqual(cpu['cpu_system'], 'system')
+    self.assertEqual(cpu['cpu_idle'], 'idle')
+    self.assertEqual(cpu['cpu_nice'], 'nice')
+    self.assertEqual(cpu['cpu_wio'], 'iowait')
+    self.assertEqual(cpu['cpu_intr'], 'irq')
+    self.assertEqual(cpu['cpu_sintr'], 'softirq')
+    self.assertEqual(cpu['load_one'], 13)
+    self.assertEqual(cpu['load_five'], 13)
+    self.assertEqual(cpu['load_fifteen'], 13)
+    
+  @patch("psutil.disk_usage")
+  @patch("psutil.disk_partitions")
+  @patch("psutil.swap_memory")
+  @patch("psutil.virtual_memory")
+  def testMemInfo(self, vm_mock, sw_mock, dm_mock, du_mock):
+    
+    vm = vm_mock.return_value
+    vm.free = "free"
+    vm.shared = "shared"
+    vm.buffers = "buffers"
+    vm.cached = "cached"
+    
+    sw = sw_mock.return_value
+    sw.free = "free"
+    
+    hostinfo = HostInfo()
+    
+    cpu = hostinfo.get_mem_info()
+    
+    self.assertEqual(cpu['mem_free'], 'free')
+    self.assertEqual(cpu['mem_shared'], 'shared')
+    self.assertEqual(cpu['mem_buffered'], 'buffers')
+    self.assertEqual(cpu['mem_cached'], 'cached')
+    self.assertEqual(cpu['swap_free'], 'free')
+
+  @patch("psutil.disk_usage")
+  @patch("psutil.disk_partitions")
+  def testCombinedDiskUsage(self, dp_mock, du_mock):
+    
+    dp_mock.__iter__.return_value = ['a', 'b', 'c']
+    
+    hostinfo = HostInfo()
+    
+    cdu = hostinfo.get_combined_disk_usage()
+    self.assertEqual(cdu['disk_total'], "0.00")
+    self.assertEqual(cdu['disk_used'], "0.00")
+    self.assertEqual(cdu['disk_free'], "0.00")
+    self.assertEqual(cdu['disk_percent'], "0.00")

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestMetricCollector.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestMetricCollector.py b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestMetricCollector.py
new file mode 100644
index 0000000..bc4fba7
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/core/TestMetricCollector.py
@@ -0,0 +1,49 @@
+#!/usr/bin/env python
+
+'''
+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 logging
+from unittest import TestCase
+
+from application_metric_map import ApplicationMetricMap
+from metric_collector import MetricsCollector
+from event_definition import HostMetricCollectEvent
+from mock.mock import patch
+from host_info import HostInfo
+
+logger = logging.getLogger()
+
+class TestMetricCollector(TestCase):
+  
+  @patch("os.getloadavg")
+  @patch.object(HostInfo, "get_cpu_times")
+  @patch.object(ApplicationMetricMap, "__init__")
+  def testCollectEvent(self, amm_mock, host_info_mock, avg_mock):
+    amm_mock.return_value = None
+    host_info_mock.return_value = {'metric_name' : 'metric_value'}
+    avg_mock.return_value.__getitem__.return_value = 13
+
+    metric_collector = MetricsCollector(None, amm_mock)
+    
+    group_config = {'collect_every' : 1, 'metrics' : 'cpu'}
+    
+    e = HostMetricCollectEvent(group_config, 'cpu')
+    
+    metric_collector.process_event(e)
+    
+    self.assertEqual(amm_mock.put_metric.call_count, 1)

http://git-wip-us.apache.org/repos/asf/ambari/blob/d6cea463/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/unitTests.py
----------------------------------------------------------------------
diff --git a/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/unitTests.py b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/unitTests.py
new file mode 100644
index 0000000..931b02a
--- /dev/null
+++ b/ambari-metrics/ambari-metrics-host-monitoring/src/test/python/unitTests.py
@@ -0,0 +1,133 @@
+'''
+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 unittest
+import os
+import sys
+from random import shuffle
+import fnmatch
+
+#excluded directories with non-test staff from stack and service scanning,
+#also we can add service or stack to skip here
+STACK_EXCLUDE = ["utils"]
+SERVICE_EXCLUDE = ["configs"]
+
+TEST_MASK = '[Tt]est*.py'
+CUSTOM_TEST_MASK = '_[Tt]est*.py'
+def get_parent_path(base, directory_name):
+  """
+  Returns absolute path for directory_name, if directory_name present in base.
+  For example, base=/home/user/test2, directory_name=user - will return /home/user
+  """
+  done = False
+  while not done:
+    base = os.path.dirname(base)
+    if base == "/":
+      return None
+    done = True if os.path.split(base)[-1] == directory_name else False
+  return base
+
+def get_test_files(path, mask = None, recursive=True):
+  """
+  Returns test files for path recursively
+  """
+  current = []
+  directory_items = os.listdir(path)
+
+  for item in directory_items:
+    add_to_pythonpath = False
+    if os.path.isfile(path + "/" + item):
+      if fnmatch.fnmatch(item, mask):
+        add_to_pythonpath = True
+        current.append(item)
+    elif os.path.isdir(path + "/" + item):
+      if recursive:
+        current.extend(get_test_files(path + "/" + item, mask = mask))
+    if add_to_pythonpath:
+      sys.path.append(path)
+  return current
+
+
+def main():
+  custom_tests = False
+  if len(sys.argv) > 1:
+    if sys.argv[1] == "true":
+      custom_tests = True
+  pwd = os.path.abspath(os.path.dirname(__file__))
+
+  project_folder = get_parent_path(pwd,'ambari-metrics-host-monitoring')
+  ambari_common_folder = os.path.join(project_folder,"../../ambari-common")
+  sys.path.append(ambari_common_folder + "/src/main/python")
+  sys.path.append(ambari_common_folder + "/src/main/python/ambari_jinja2")
+  sys.path.append(ambari_common_folder + "/src/main/python")
+  sys.path.append(ambari_common_folder + "/src/test/python")
+  sys.path.append(project_folder + "/src/test/python")
+  sys.path.append(project_folder + "/src/main/python")
+  sys.path.append(project_folder + "/src/main/python/core")
+  sys.path.append(project_folder + "/src/main/resources/scripts")
+  sys.path.append(project_folder + "/src/main/resources/custom_actions")
+  sys.path.append(project_folder + "/target/psutil_build/lib.linux-x86_64-2.6")
+
+  has_failures = False
+  test_runs = 0
+  test_failures = []
+  test_errors = []
+  #run base ambari-server tests
+  sys.stderr.write("Running tests\n")
+  if custom_tests:
+    test_mask = CUSTOM_TEST_MASK
+  else:
+    test_mask = TEST_MASK
+
+  tests = get_test_files(pwd, mask=test_mask, recursive=True)
+  shuffle(tests)
+  modules = [os.path.basename(s)[:-3] for s in tests]
+  suites = [unittest.defaultTestLoader.loadTestsFromName(name) for name in
+    modules]
+  testSuite = unittest.TestSuite(suites)
+  textRunner = unittest.TextTestRunner(verbosity=2).run(testSuite)
+  test_runs += textRunner.testsRun
+  test_errors.extend([(str(item[0]),str(item[1]),"ERROR") for item in textRunner.errors])
+  test_failures.extend([(str(item[0]),str(item[1]),"FAIL") for item in textRunner.failures])
+  tests_status = textRunner.wasSuccessful() and not has_failures
+
+  if not tests_status:
+    sys.stderr.write("----------------------------------------------------------------------\n")
+    sys.stderr.write("Failed tests:\n")
+  for failed_tests in [test_errors,test_failures]:
+    for err in failed_tests:
+      sys.stderr.write("{0}: {1}\n".format(err[2],err[0]))
+      sys.stderr.write("----------------------------------------------------------------------\n")
+      sys.stderr.write("{0}\n".format(err[1]))
+  sys.stderr.write("----------------------------------------------------------------------\n")
+  sys.stderr.write("Total run:{0}\n".format(test_runs))
+  sys.stderr.write("Total errors:{0}\n".format(len(test_errors)))
+  sys.stderr.write("Total failures:{0}\n".format(len(test_failures)))
+
+  if tests_status:
+    sys.stderr.write("OK\n")
+    exit_code = 0
+  else:
+    sys.stderr.write("ERROR\n")
+    exit_code = 1
+  return exit_code
+
+
+if __name__ == "__main__":
+  sys.exit(main())
+