Refactors codebase 92/6692/3
authorBenjamin Diaz <bdiaz@whitestack.com>
Fri, 12 Oct 2018 20:37:30 +0000 (17:37 -0300)
committerBenjamin Diaz <bdiaz@whitestack.com>
Tue, 16 Oct 2018 19:52:26 +0000 (16:52 -0300)
Refactors duplicated code. Renames and reorganizes classes and methods.
Refactors tests.
Kafka message handling is removed from plugins. They now return response msg
dict and top level code handles publishing.

Signed-off-by: Benjamin Diaz <bdiaz@whitestack.com>
Change-Id: I9826db9c4586d961f40af6cb194fc388fe855ae1

38 files changed:
osm_mon/core/message_bus/common_consumer.py
osm_mon/core/message_bus/consumer.py [changed mode: 0755->0644]
osm_mon/core/message_bus/producer.py [changed mode: 0755->0644]
osm_mon/core/models/acknowledge_alarm.json [deleted file]
osm_mon/core/models/acknowledge_alarm_req.json [new file with mode: 0644]
osm_mon/plugins/CloudWatch/access_credentials.py
osm_mon/plugins/CloudWatch/connection.py
osm_mon/plugins/CloudWatch/metric_alarms.py
osm_mon/plugins/CloudWatch/metrics.py
osm_mon/plugins/CloudWatch/plugin_alarm.py
osm_mon/plugins/CloudWatch/plugin_metric.py
osm_mon/plugins/OpenStack/Aodh/alarm_handler.py [new file with mode: 0644]
osm_mon/plugins/OpenStack/Aodh/alarming.py [deleted file]
osm_mon/plugins/OpenStack/Aodh/notifier.py
osm_mon/plugins/OpenStack/Gnocchi/metric_handler.py [new file with mode: 0644]
osm_mon/plugins/OpenStack/Gnocchi/metrics.py [deleted file]
osm_mon/plugins/OpenStack/response.py
osm_mon/plugins/vRealiseOps/kafka_consumer_vrops.py [deleted file]
osm_mon/plugins/vRealiseOps/mon_plugin_vrops.py
osm_mon/plugins/vRealiseOps/plugin_receiver.py
osm_mon/plugins/vRealiseOps/vROPs_Webservice/vrops_webservice
osm_mon/test/OpenStack/integration/__init__.py
osm_mon/test/OpenStack/integration/test_alarm_integration.py
osm_mon/test/OpenStack/integration/test_metric_integration.py
osm_mon/test/OpenStack/integration/test_notify_alarm.py
osm_mon/test/OpenStack/unit/__init__.py
osm_mon/test/OpenStack/unit/test_alarm_req.py
osm_mon/test/OpenStack/unit/test_alarming.py
osm_mon/test/OpenStack/unit/test_metric_calls.py
osm_mon/test/OpenStack/unit/test_metric_req.py
osm_mon/test/OpenStack/unit/test_notifier.py
osm_mon/test/OpenStack/unit/test_responses.py
osm_mon/test/VMware/__init__.py
osm_mon/test/VMware/test_mon_plugin_vrops.py
osm_mon/test/VMware/test_plugin_receiver.py
osm_mon/test/core/kafka_test.py [deleted file]
osm_mon/test/core/test_common_consumer.py
osm_mon/test/core/test_producer.py [deleted file]

index fd97f83..b8e33d2 100755 (executable)
@@ -23,21 +23,23 @@ import json
 import logging
 import sys
 import threading
+from json import JSONDecodeError
 
 import six
 import yaml
-from kafka import KafkaConsumer
 
 from osm_mon.common.common_db_client import CommonDbClient
 from osm_mon.core.auth import AuthManager
 from osm_mon.core.database import DatabaseManager
+from osm_mon.core.message_bus.consumer import Consumer
+from osm_mon.core.message_bus.producer import Producer
 from osm_mon.core.settings import Config
 from osm_mon.plugins.CloudWatch.access_credentials import AccessCredentials
 from osm_mon.plugins.CloudWatch.connection import Connection
 from osm_mon.plugins.CloudWatch.plugin_alarm import plugin_alarms
 from osm_mon.plugins.CloudWatch.plugin_metric import plugin_metrics
-from osm_mon.plugins.OpenStack.Aodh import alarming
-from osm_mon.plugins.OpenStack.Gnocchi import metrics
+from osm_mon.plugins.OpenStack.Aodh import alarm_handler
+from osm_mon.plugins.OpenStack.Gnocchi import metric_handler
 from osm_mon.plugins.vRealiseOps import plugin_receiver
 
 cfg = Config.instance()
@@ -59,15 +61,13 @@ kafka_logger.addHandler(kafka_handler)
 class CommonConsumer:
 
     def __init__(self):
-        cfg = Config.instance()
-
         self.auth_manager = AuthManager()
         self.database_manager = DatabaseManager()
         self.database_manager.create_tables()
 
         # Create OpenStack alarming and metric instances
-        self.openstack_metrics = metrics.Metrics()
-        self.openstack_alarms = alarming.Alarming()
+        self.openstack_metrics = metric_handler.OpenstackMetricHandler()
+        self.openstack_alarms = alarm_handler.OpenstackAlarmHandler()
 
         # Create CloudWatch alarm and metric instances
         self.cloudwatch_alarms = plugin_alarms()
@@ -88,12 +88,7 @@ class CommonConsumer:
         return credentials.type
 
     def run(self):
-        common_consumer = KafkaConsumer(bootstrap_servers=cfg.BROKER_URI,
-                                        key_deserializer=bytes.decode,
-                                        value_deserializer=bytes.decode,
-                                        group_id="mon-consumer",
-                                        session_timeout_ms=60000,
-                                        heartbeat_interval_ms=20000)
+        common_consumer = Consumer("mon-consumer")
 
         topics = ['metric_request', 'alarm_request', 'vim_account']
         common_consumer.subscribe(topics)
@@ -108,9 +103,11 @@ class CommonConsumer:
         try:
             try:
                 values = json.loads(message.value)
-            except ValueError:
+            except JSONDecodeError:
                 values = yaml.safe_load(message.value)
 
+            response = None
+
             if message.topic == "vim_account":
                 if message.key == "create" or message.key == "edit":
                     self.auth_manager.store_auth_credentials(values)
@@ -153,31 +150,42 @@ class CommonConsumer:
                 if vim_type == "openstack":
                     log.info("This message is for the OpenStack plugin.")
                     if message.topic == "metric_request":
-                        self.openstack_metrics.metric_calls(message, vim_uuid)
+                        response = self.openstack_metrics.handle_request(message.key, values, vim_uuid)
                     if message.topic == "alarm_request":
-                        self.openstack_alarms.alarming(message, vim_uuid)
+                        response = self.openstack_alarms.handle_message(message.key, values, vim_uuid)
 
                 elif vim_type == "aws":
                     log.info("This message is for the CloudWatch plugin.")
                     aws_conn = self.aws_connection.setEnvironment()
                     if message.topic == "metric_request":
-                        self.cloudwatch_metrics.metric_calls(message, aws_conn)
+                        response = self.cloudwatch_metrics.metric_calls(message.key, values, aws_conn)
                     if message.topic == "alarm_request":
-                        self.cloudwatch_alarms.alarm_calls(message, aws_conn)
-                    if message.topic == "access_credentials":
-                        self.aws_access_credentials.access_credential_calls(message)
+                        response = self.cloudwatch_alarms.alarm_calls(message.key, values, aws_conn)
 
                 elif vim_type == "vmware":
                     log.info("This metric_request message is for the vROPs plugin.")
-                    self.vrops_rcvr.consume(message, vim_uuid)
+                    if message.topic == "metric_request":
+                        response = self.vrops_rcvr.handle_metric_requests(message.key, values, vim_uuid)
+                    if message.topic == "alarm_request":
+                        response = self.vrops_rcvr.handle_alarm_requests(message.key, values, vim_uuid)
 
                 else:
                     log.debug("vim_type is misconfigured or unsupported; %s",
                               vim_type)
+            if response:
+                self._publish_response(message.topic, message.key, response)
 
         except Exception:
             log.exception("Exception processing message: ")
 
+    def _publish_response(self, topic: str, key: str, msg: dict):
+        topic = topic.replace('request', 'response')
+        key = key.replace('request', 'response')
+        producer = Producer()
+        producer.send(topic=topic, key=key, value=json.dumps(msg))
+        producer.flush()
+        producer.close()
+
 
 if __name__ == '__main__':
     CommonConsumer().run()
old mode 100755 (executable)
new mode 100644 (file)
index c9021d2..b12cd88
@@ -1,95 +1,15 @@
-# Copyright 2017 Intel Research and Development Ireland Limited
-# *************************************************************
-
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
-##
-
-'''
-This is a kafka consumer app that reads the messages from the message bus for
-alarms and metrics responses.
-
-'''
-
-__author__ = "Prithiv Mohan"
-__date__ = "06/Sep/2017"
-
-
 from kafka import KafkaConsumer
-from kafka.errors import KafkaError
-import json
-import logging
-import logging.config
-import os
-
-
-def logging_handler(filename, mode='a+', encoding=None):
-    if not os.path.exists(filename):
-        open(filename, 'a').close()
-    return logging.FileHandler(filename, mode)
-
-log_config = {
-    'version': 1,
-    'formatters': {
-        'default': {
-            'format': '%(asctime)s %(levelname)s %(name)s %(message)s'
-        },
-    },
-    'handlers': {
-        'file': {
-            '()': logging_handler,
-            'level': 'DEBUG',
-            'formatter': 'default',
-            'filename': '/var/log/osm_mon.log',
-            'mode': 'a+',
-            'encoding': 'utf-8',
-        },
-    },
-    'kafka': {
-        'handlers': ['file'],
-        'level': 'DEBUG',
-    },
-    'root': {
-        'handlers': ['file'],
-        'level': 'DEBUG',
-    },
-}
-
-
-logging.config.dictConfig(log_config)
-logger = logging.getLogger('kafka')
 
-if "BROKER_URI" in os.environ:
-    broker = os.getenv("BROKER_URI")
-else:
-    broker = "localhost:9092"
+from osm_mon.core.settings import Config
 
-alarm_consumer = KafkaConsumer(
-    'alarm_response', 'osm_mon', bootstrap_servers=broker)
-metric_consumer = KafkaConsumer(
-    'metric_response', 'osm_mon', bootstrap_servers=broker)
-try:
-    for message in alarm_consumer:
-        logger.debug(message)
-    for message in metric_consumer:
-        logger.debug(message)
-except KafkaError:
-    log.exception()
 
-alarm_consumer.subscribe('alarm_response')
-metric_consumer.subscribe('metric_response')
+# noinspection PyAbstractClass
+class Consumer(KafkaConsumer):
+    def __init__(self, group_id):
+        cfg = Config.instance()
+        super().__init__(bootstrap_servers=cfg.BROKER_URI,
+                         key_deserializer=bytes.decode,
+                         value_deserializer=bytes.decode,
+                         group_id=group_id,
+                         session_timeout_ms=60000,
+                         heartbeat_interval_ms=20000)
old mode 100755 (executable)
new mode 100644 (file)
index f6feba1..573e332
@@ -1,8 +1,9 @@
-# Copyright 2017 Intel Research and Development Ireland Limited
+# Copyright 2018 Whitestack, LLC
 # *************************************************************
 
 # This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
+# All Rights Reserved to Whitestack, LLC
+
 # Licensed under the Apache License, Version 2.0 (the "License"); you may
 # not use this file except in compliance with the License. You may obtain
 # a copy of the License at
 # under the License.
 
 # For those usages not covered by the Apache License, Version 2.0 please
-# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
+# contact: bdiaz@whitestack.com or glavado@whitestack.com
 ##
-"""This is a common kafka producer app.
-
-It interacts with the SO and the plugins of the datacenters: OpenStack, VMWare
-and AWS.
-"""
-
-import logging
-import os
-
-from kafka import KafkaProducer as kaf
-from kafka.errors import KafkaError
-
-__author__ = "Prithiv Mohan"
-__date__ = "06/Sep/2017"
-
-current_path = os.path.realpath(__file__)
-json_path = os.path.abspath(os.path.join(current_path, '..', '..', 'models'))
-
-# TODO(): validate all of the request and response messages against the
-# json_schemas
-
-
-class KafkaProducer(object):
-    """A common KafkaProducer for requests and responses."""
-
-    def __init__(self, topic):
-        """Initialize the common kafka producer."""
-        self._topic = topic
-
-        if "BROKER_URI" in os.environ:
-            broker = os.getenv("BROKER_URI")
-        else:
-            broker = "localhost:9092"
-
-        '''
-        If the broker URI is not set in the env by default,
-        localhost container is taken as the host because an instance of
-        is already running.
-        '''
-
-        self.producer = kaf(
-            key_serializer=str.encode,
-            value_serializer=str.encode,
-            bootstrap_servers=broker, api_version=(0, 10, 1))
-
-    def publish(self, key, value, topic=None):
-        """Send the required message on the Kafka message bus."""
-        try:
-            future = self.producer.send(topic=topic, key=key, value=value)
-            future.get(timeout=10)
-        except Exception:
-            logging.exception("Error publishing to {} topic." .format(topic))
-            raise
-
-    def publish_alarm_request(self, key, message):
-        """Publish an alarm request."""
-        # External to MON
-
-        self.publish(key,
-                     value=message,
-                     topic='alarm_request')
-
-    def publish_alarm_response(self, key, message):
-        """Publish an alarm response."""
-        # Internal to MON
-
-        self.publish(key,
-                     value=message,
-                     topic='alarm_response')
-
-    def publish_metrics_request(self, key, message):
-        """Create metrics request from SO to MON."""
-        # External to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_request')
-
-    def publish_metrics_response(self, key, message):
-        """Response for a create metric request from MON to SO."""
-        # Internal to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_response')
-
-    def read_metric_data_request(self, key, message):
-        """Read metric data request from SO to MON."""
-        # External to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_request')
-
-    def read_metric_data_response(self, key, message):
-        """Response from MON to SO for read metric data request."""
-        # Internal to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_response')
-
-    def list_metric_request(self, key, message):
-        """List metric request from SO to MON."""
-        # External to MON
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_request')
-
-    def list_metric_response(self, key, message):
-        """Response from SO to MON for list metrics request."""
-        # Internal to MON
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_response')
-
-    def delete_metric_request(self, key, message):
-        """Delete metric request from SO to MON."""
-        # External to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_request')
-
-    def delete_metric_response(self, key, message):
-        """Response from MON to SO for delete metric request."""
-        # Internal to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_response')
-
-    def update_metric_request(self, key, message):
-        """Metric update request from SO to MON."""
-        # External to Mon
-
-        self.publish(key,
-                     value=message,
-                     topic='metric_request')
+from kafka import KafkaProducer
 
-    def update_metric_response(self, key, message):
-        """Reponse from MON to SO for metric update."""
-        # Internal to Mon
+from osm_mon.core.settings import Config
 
-        self.publish(key,
-                     value=message,
-                     topic='metric_response')
 
-    def access_credentials(self, key, message):
-        """Send access credentials to MON from SO."""
+class Producer(KafkaProducer):
+    def __init__(self):
+        cfg = Config.instance()
+        super().__init__(bootstrap_servers=cfg.BROKER_URI,
+                         key_serializer=str.encode,
+                         value_serializer=str.encode)
 
-        self.publish(key,
-                     value=message,
-                     topic='access_credentials')
+    def send(self, topic, value=None, key=None, partition=None, timestamp_ms=None):
+        return super().send(topic, value, key, partition, timestamp_ms)
diff --git a/osm_mon/core/models/acknowledge_alarm.json b/osm_mon/core/models/acknowledge_alarm.json
deleted file mode 100644 (file)
index 8a9e50a..0000000
+++ /dev/null
@@ -1,41 +0,0 @@
-
-/* Copyright© 2017 Intel Research and Development Ireland Limited
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
-
-# This is the message bus schema for acknowledge_alarm */
-
-{  
-  "schema_version": { "type": "string" },
-  "schema_type": { "type": "string" },
-  "vim_type": { "type": "string" },
-  "vim_uuid": { "type": "string" },
-  "ack_details":
-  {
-    "alarm_uuid": { "type": "string" },
-    "ns_id": { "type": "string"},
-    "vnf_member_index": { "type": "integer"},
-    "vdu_name": { "type": "string"}
-  },
-  "required": [ "schema_version",
-                "schema_type",
-                "alarm_uuid",
-                "ns_id",
-                "vnf_member_index",
-                "vdu_name"]
-}
diff --git a/osm_mon/core/models/acknowledge_alarm_req.json b/osm_mon/core/models/acknowledge_alarm_req.json
new file mode 100644 (file)
index 0000000..8a9e50a
--- /dev/null
@@ -0,0 +1,41 @@
+
+/* Copyright© 2017 Intel Research and Development Ireland Limited
+# This file is part of OSM Monitoring module
+# All Rights Reserved to Intel Corporation
+
+# Licensed under the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License. You may obtain
+# a copy of the License at
+
+#         http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+# License for the specific language governing permissions and limitations
+# under the License.
+
+# For those usages not covered by the Apache License, Version 2.0 please
+# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
+
+# This is the message bus schema for acknowledge_alarm */
+
+{  
+  "schema_version": { "type": "string" },
+  "schema_type": { "type": "string" },
+  "vim_type": { "type": "string" },
+  "vim_uuid": { "type": "string" },
+  "ack_details":
+  {
+    "alarm_uuid": { "type": "string" },
+    "ns_id": { "type": "string"},
+    "vnf_member_index": { "type": "integer"},
+    "vdu_name": { "type": "string"}
+  },
+  "required": [ "schema_version",
+                "schema_type",
+                "alarm_uuid",
+                "ns_id",
+                "vnf_member_index",
+                "vdu_name"]
+}
index 3774858..8c110ab 100644 (file)
 # contact with: usman.javaid@xflowresearch.com
 ##
 
-'''
+"""
 Access credentials class implements all the methods to store the access credentials for AWS
-'''
+"""
 
 __author__ = "Usman Javaid"
-__date__   = "20-December-2017"
+__date__ = "20-December-2017"
 
-import os
-import sys
 import json
 import logging
+import os
 
 log = logging.getLogger(__name__)
 
+
 class AccessCredentials():
 
     def logtest(self):
         log.info("Access credentials sourced for CloudWatch MON plugin")
 
-
-    def access_credential_calls(self,message):  
-        try:   
+    def access_credential_calls(self, message):
+        try:
             message = json.loads(message.value)['access_config']
-            
+
             AWS_KEY = message['user']
             AWS_SECRET = message['password']
             AWS_REGION = message['vim_tenant_name']
@@ -51,17 +50,15 @@ class AccessCredentials():
             os.environ['AWS_SECRET_ACCESS_KEY'] = AWS_SECRET
             os.environ['AWS_EC2_REGION'] = AWS_REGION
 
+            # aws_credentials.txt file to save the access credentials
+            cloudwatch_credentials = open("../../plugins/CloudWatch/cloudwatch_credentials.txt", "w+")
+            cloudwatch_credentials.write("AWS_ACCESS_KEY_ID=" + AWS_KEY +
+                                         "\nAWS_SECRET_ACCESS_KEY=" + AWS_SECRET +
+                                         "\nAWS_EC2_REGION=" + AWS_REGION)
 
-            #aws_credentials.txt file to save the access credentials 
-            cloudwatch_credentials = open("../../plugins/CloudWatch/cloudwatch_credentials.txt","w+")
-            cloudwatch_credentials.write("AWS_ACCESS_KEY_ID="+AWS_KEY+
-                                         "\nAWS_SECRET_ACCESS_KEY="+AWS_SECRET+
-                                         "\nAWS_EC2_REGION="+AWS_REGION)
-            
-            #Closing the file
+            # Closing the file
             cloudwatch_credentials.close()
             log.info("Access credentials sourced for CloudWatch MON plugin")
 
         except Exception as e:
-                log.error("Access credentials not provided correctly: %s", str(e))
-
+            log.error("Access credentials not provided correctly: %s", str(e))
index 5853ae9..98b05aa 100644 (file)
 # contact with: wajeeha.hamid@xflowresearch.com
 ##
 
-'''
+"""
 Connecting with AWS services --CloudWatch/EC2 using Required keys  
-'''
+"""
 
 __author__ = "Wajeeha Hamid"
-__date__   = "18-September-2017"
+__date__ = "18-September-2017"
 
 import os
 
@@ -34,62 +34,61 @@ try:
     import boto.vpc
     import boto.ec2.cloudwatch
     import boto.ec2.connection
-    import logging 
+    import logging
     from boto.ec2.cloudwatch.alarm import MetricAlarm
     from boto.ec2.cloudwatch.dimension import Dimension
     from boto.sns import connect_to_region
     from boto.utils import get_instance_metadata
 
 except:
-    exit("Boto not avialable. Try activating your virtualenv OR `pip install boto`")
+    exit("Boto not available. Try activating your virtualenv OR `pip install boto`")
 
 log = logging.getLogger(__name__)
 
-class Connection():
-    """Connection Establishement with AWS -- VPC/EC2/CloudWatch"""
-#-----------------------------------------------------------------------------------------------------------------------------
-    def setEnvironment(self):  
+
+class Connection:
+    """Connection Establishment with AWS -- VPC/EC2/CloudWatch"""
+
+    def setEnvironment(self):
         try:
-            """Credentials for connecting to AWS-CloudWatch""" 
-            #Reads from the environment variables
+            """Credentials for connecting to AWS-CloudWatch"""
+            # Reads from the environment variables
             self.AWS_KEY = os.environ.get("AWS_ACCESS_KEY_ID")
             self.AWS_SECRET = os.environ.get("AWS_SECRET_ACCESS_KEY")
-            self.AWS_REGION = os.environ.get("AWS_EC2_REGION","us-west-2")
+            self.AWS_REGION = os.environ.get("AWS_EC2_REGION", "us-west-2")
 
-            #TODO Read from the cloudwatch_credentials.txt file
+            # TODO Read from the cloudwatch_credentials.txt file
 
             return self.connection_instance()
         except Exception as e:
-            log.error("AWS Credentials not configured, Try setting the access credentials first %s: ",str(e)) 
-#-----------------------------------------------------------------------------------------------------------------------------
+            log.error("AWS Credentials not configured, Try setting the access credentials first %s: ", str(e))
+
     def connection_instance(self):
-            try:
-                #VPC Connection
-                self.vpc_conn = boto.vpc.connect_to_region(self.AWS_REGION,
-                    aws_access_key_id=self.AWS_KEY,
-                    aws_secret_access_key=self.AWS_SECRET)
-               
-                
-                #EC2 Connection
-                self.ec2_conn = boto.ec2.connect_to_region(self.AWS_REGION,
-                    aws_access_key_id=self.AWS_KEY,
-                    aws_secret_access_key=self.AWS_SECRET)               
-                
-                """ TODO : Required to add actions against alarms when needed """
-                #self.sns = connect_to_region(self.AWS_REGION)
-                #self.topics = self.sns.get_all_topics()
-                #self.topic = self.topics[u'ListTopicsResponse']['ListTopicsResult']['Topics'][0]['TopicArn']
-
-                #Cloudwatch Connection
-                self.cloudwatch_conn = boto.ec2.cloudwatch.connect_to_region(
-                    self.AWS_REGION,
-                    aws_access_key_id=self.AWS_KEY,
-                    aws_secret_access_key=self.AWS_SECRET) 
-                connection_dict = dict()
-                connection_dict['ec2_connection'] = self.ec2_conn
-                connection_dict['cloudwatch_connection'] = self.cloudwatch_conn
-                return connection_dict
-                
-            except Exception as e:
-                log.error("Failed to Connect with AWS %s: ",str(e)) 
+        try:
+            # VPC Connection
+            self.vpc_conn = boto.vpc.connect_to_region(self.AWS_REGION,
+                                                       aws_access_key_id=self.AWS_KEY,
+                                                       aws_secret_access_key=self.AWS_SECRET)
 
+            # EC2 Connection
+            self.ec2_conn = boto.ec2.connect_to_region(self.AWS_REGION,
+                                                       aws_access_key_id=self.AWS_KEY,
+                                                       aws_secret_access_key=self.AWS_SECRET)
+
+            """ TODO : Required to add actions against alarms when needed """
+            # self.sns = connect_to_region(self.AWS_REGION)
+            # self.topics = self.sns.get_all_topics()
+            # self.topic = self.topics[u'ListTopicsResponse']['ListTopicsResult']['Topics'][0]['TopicArn']
+
+            # Cloudwatch Connection
+            self.cloudwatch_conn = boto.ec2.cloudwatch.connect_to_region(
+                self.AWS_REGION,
+                aws_access_key_id=self.AWS_KEY,
+                aws_secret_access_key=self.AWS_SECRET)
+            connection_dict = dict()
+            connection_dict['ec2_connection'] = self.ec2_conn
+            connection_dict['cloudwatch_connection'] = self.cloudwatch_conn
+            return connection_dict
+
+        except Exception as e:
+            log.error("Failed to Connect with AWS %s: ", str(e))
index 8e5b6fe..5d330b0 100644 (file)
@@ -22,7 +22,7 @@
 ''' Handling of alarms requests via BOTO 2.48 '''
 
 __author__ = "Wajeeha Hamid"
-__date__   = "18-September-2017"
+__date__ = "18-September-2017"
 
 import logging
 
@@ -41,64 +41,66 @@ STATISTICS = {
     "AVERAGE": "Average",
     "MINIMUM": "Minimum",
     "MAXIMUM": "Maximum",
-    "COUNT"  : "SampleCount",
-    "SUM"    : "Sum"}
+    "COUNT": "SampleCount",
+    "SUM": "Sum"}
 
 OPERATIONS = {
-    "GE"     : ">=",
-    "LE"     : "<=",
-    "GT"     : ">",
-    "LT"     : "<",
-    "EQ"     : "="}   
+    "GE": ">=",
+    "LE": "<=",
+    "GT": ">",
+    "LT": "<",
+    "EQ": "="}
+
 
 class MetricAlarm():
     """Alarms Functionality Handler -- Carries out alarming requests and responses via BOTO.Cloudwatch """
+
     def __init__(self):
         self.alarm_resp = dict()
         self.del_resp = dict()
 
-    def config_alarm(self,cloudwatch_conn,create_info):
+    def config_alarm(self, cloudwatch_conn, create_info):
         """Configure or Create a new alarm"""
         inner_dict = dict()
         """ Alarm Name to ID Mapping """
         alarm_info = create_info['alarm_create_request']
         alarm_id = alarm_info['alarm_name'] + "_" + alarm_info['resource_uuid']
-        if self.is_present(cloudwatch_conn,alarm_id)['status'] == True: 
+        if self.is_present(cloudwatch_conn, alarm_id)['status'] == True:
             alarm_id = None
-            log.debug ("Alarm already exists, Try updating the alarm using 'update_alarm_configuration()'")
-            return alarm_id   
-        else:              
+            log.debug("Alarm already exists, Try updating the alarm using 'update_alarm_configuration()'")
+            return alarm_id
+        else:
             try:
                 if alarm_info['statistic'] in STATISTICS:
                     if alarm_info['operation'] in OPERATIONS:
                         alarm = boto.ec2.cloudwatch.alarm.MetricAlarm(
-                            connection = cloudwatch_conn,
-                            name = alarm_info['alarm_name'] + "_" + alarm_info['resource_uuid'],
-                            metric = alarm_info['metric_name'],
-                            namespace = "AWS/EC2",
-                            statistic = STATISTICS[alarm_info['statistic']],
-                            comparison = OPERATIONS[alarm_info['operation']],
-                            threshold = alarm_info['threshold_value'],
-                            period = 60,
-                            evaluation_periods = 1,
+                            connection=cloudwatch_conn,
+                            name=alarm_info['alarm_name'] + "_" + alarm_info['resource_uuid'],
+                            metric=alarm_info['metric_name'],
+                            namespace="AWS/EC2",
+                            statistic=STATISTICS[alarm_info['statistic']],
+                            comparison=OPERATIONS[alarm_info['operation']],
+                            threshold=alarm_info['threshold_value'],
+                            period=60,
+                            evaluation_periods=1,
                             unit=alarm_info['unit'],
-                            description = alarm_info['severity'] + ";" + alarm_id + ";" + alarm_info['description'],
-                            dimensions = {'InstanceId':alarm_info['resource_uuid']},
-                            alarm_actions = None,
-                            ok_actions = None,
-                            insufficient_data_actions = None)
+                            description=alarm_info['severity'] + ";" + alarm_id + ";" + alarm_info['description'],
+                            dimensions={'InstanceId': alarm_info['resource_uuid']},
+                            alarm_actions=None,
+                            ok_actions=None,
+                            insufficient_data_actions=None)
 
                         """Setting Alarm Actions : 
                         alarm_actions = ['arn:aws:swf:us-west-2:465479087178:action/actions/AWS_EC2.InstanceId.Stop/1.0']"""
 
-                        status=cloudwatch_conn.put_metric_alarm(alarm)
+                        status = cloudwatch_conn.put_metric_alarm(alarm)
 
-                        log.debug ("Alarm Configured Succesfully")
+                        log.debug("Alarm Configured Succesfully")
                         self.alarm_resp['schema_version'] = str(create_info['schema_version'])
                         self.alarm_resp['schema_type'] = 'create_alarm_response'
 
                         inner_dict['correlation_id'] = str(alarm_info['correlation_id'])
-                        inner_dict['alarm_uuid'] = str(alarm_id) 
+                        inner_dict['alarm_uuid'] = str(alarm_id)
                         inner_dict['status'] = status
 
                         self.alarm_resp['alarm_create_response'] = inner_dict
@@ -106,18 +108,17 @@ class MetricAlarm():
                         if status == True:
                             return self.alarm_resp
                         else:
-                            return None        
-                    else: 
+                            return None
+                    else:
                         log.error("Operation not supported")
-                        return None        
+                        return None
                 else:
                     log.error("Statistic not supported")
                     return None
             except Exception as e:
                 log.error("Alarm Configuration Failed: " + str(e))
-            
-#-----------------------------------------------------------------------------------------------------------------------------
-    def update_alarm(self,cloudwatch_conn,update_info):
+
+    def update_alarm(self, cloudwatch_conn, update_info):
 
         """Update or reconfigure an alarm"""
         inner_dict = dict()
@@ -125,67 +126,66 @@ class MetricAlarm():
 
         """Alarm Name to ID Mapping"""
         alarm_id = alarm_info['alarm_uuid']
-        status = self.is_present(cloudwatch_conn,alarm_id)
+        status = self.is_present(cloudwatch_conn, alarm_id)
 
         """Verifying : Alarm exists already"""
-        if status['status'] == False: 
+        if status['status'] == False:
             alarm_id = None
             log.debug("Alarm not found, Try creating the alarm using 'configure_alarm()'")
-            return alarm_id   
-        else:            
+            return alarm_id
+        else:
             try:
                 if alarm_info['statistic'] in STATISTICS:
                     if alarm_info['operation'] in OPERATIONS:
                         alarm = boto.ec2.cloudwatch.alarm.MetricAlarm(
-                            connection = cloudwatch_conn,
-                            name = status['info'].name ,
-                            metric = alarm_info['metric_name'],
-                            namespace = "AWS/EC2",
-                            statistic = STATISTICS[alarm_info['statistic']],
-                            comparison = OPERATIONS[alarm_info['operation']],
-                            threshold = alarm_info['threshold_value'],
-                            period = 60,
-                            evaluation_periods = 1,
+                            connection=cloudwatch_conn,
+                            name=status['info'].name,
+                            metric=alarm_info['metric_name'],
+                            namespace="AWS/EC2",
+                            statistic=STATISTICS[alarm_info['statistic']],
+                            comparison=OPERATIONS[alarm_info['operation']],
+                            threshold=alarm_info['threshold_value'],
+                            period=60,
+                            evaluation_periods=1,
                             unit=alarm_info['unit'],
-                            description = alarm_info['severity'] + ";" + alarm_id + ";" + alarm_info['description'],
-                            dimensions = {'InstanceId':str(status['info'].dimensions['InstanceId']).split("'")[1]},
-                            alarm_actions = None,
-                            ok_actions = None,
-                            insufficient_data_actions = None)
+                            description=alarm_info['severity'] + ";" + alarm_id + ";" + alarm_info['description'],
+                            dimensions={'InstanceId': str(status['info'].dimensions['InstanceId']).split("'")[1]},
+                            alarm_actions=None,
+                            ok_actions=None,
+                            insufficient_data_actions=None)
 
                         """Setting Alarm Actions : 
                         alarm_actions = ['arn:aws:swf:us-west-2:465479087178:action/actions/AWS_EC2.InstanceId.Stop/1.0']"""
 
-                        status=cloudwatch_conn.put_metric_alarm(alarm)
-                        log.debug("Alarm %s Updated ",alarm.name)
+                        status = cloudwatch_conn.put_metric_alarm(alarm)
+                        log.debug("Alarm %s Updated ", alarm.name)
                         self.alarm_resp['schema_version'] = str(update_info['schema_version'])
                         self.alarm_resp['schema_type'] = 'update_alarm_response'
 
                         inner_dict['correlation_id'] = str(alarm_info['correlation_id'])
-                        inner_dict['alarm_uuid'] = str(alarm_id) 
+                        inner_dict['alarm_uuid'] = str(alarm_id)
                         inner_dict['status'] = status
 
                         self.alarm_resp['alarm_update_response'] = inner_dict
                         return self.alarm_resp
-                    else: 
+                    else:
                         log.error("Operation not supported")
-                        return None        
+                        return None
                 else:
                     log.error("Statistic not supported")
-                    return None        
+                    return None
             except Exception as e:
-                log.error ("Error in Updating Alarm " + str(e))
-        
-#-----------------------------------------------------------------------------------------------------------------------------
-    def delete_Alarm(self,cloudwatch_conn,del_info_all):
+                log.error("Error in Updating Alarm " + str(e))
+
+    def delete_Alarm(self, cloudwatch_conn, del_info_all):
 
         """Deletes an Alarm with specified alarm_id"""
         inner_dict = dict()
         del_info = del_info_all['alarm_delete_request']
-        status = self.is_present(cloudwatch_conn,del_info['alarm_uuid'])
+        status = self.is_present(cloudwatch_conn, del_info['alarm_uuid'])
         try:
-            if status['status'] == True:                
-                del_status=cloudwatch_conn.delete_alarms(status['info'].name)
+            if status['status'] == True:
+                del_status = cloudwatch_conn.delete_alarms(status['info'].name)
                 self.del_resp['schema_version'] = str(del_info_all['schema_version'])
                 self.del_resp['schema_type'] = 'delete_alarm_response'
                 inner_dict['correlation_id'] = str(del_info['correlation_id'])
@@ -193,90 +193,98 @@ class MetricAlarm():
                 inner_dict['status'] = del_status
                 self.del_resp['alarm_deletion_response'] = inner_dict
                 return self.del_resp
-            return None 
+            return None
         except Exception as e:
-                log.error("Alarm Not Deleted: " + str(e))      
-#-----------------------------------------------------------------------------------------------------------------------------
-    def alarms_list(self,cloudwatch_conn,list_info):
+            log.error("Alarm Not Deleted: " + str(e))
+
+    def alarms_list(self, cloudwatch_conn, list_info):
 
         """Get a list of alarms that are present on a particular VIM type"""
         alarm_list = []
         alarm_info = dict()
         inner_dict = list_info['alarm_list_request']
-        try: #id vim 
+        try:  # id vim 
             alarms = cloudwatch_conn.describe_alarms()
             itr = 0
             for alarm in alarms:
                 list_info['alarm_list_request']['alarm_uuid'] = str(alarm.description).split(';')[1]
 
-                #Severity = alarm_name = resource_uuid = ""
-                if inner_dict['severity'] == "" and inner_dict['alarm_name'] == "" and inner_dict['resource_uuid'] == "":
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # Severity = alarm_name = resource_uuid = ""
+                if inner_dict['severity'] == "" and inner_dict['alarm_name'] == "" and inner_dict[
+                    'resource_uuid'] == "":
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
+                    itr += 1
+                # alarm_name = resource_uuid = ""
+                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] == "" and \
+                        inner_dict['resource_uuid'] == "":
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #alarm_name = resource_uuid = ""
-                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] == "" and inner_dict['resource_uuid'] == "":
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # severity = resource_uuid = ""
+                if inner_dict['severity'] == "" and inner_dict['alarm_name'] in alarm.name and inner_dict[
+                    'resource_uuid'] == "":
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #severity = resource_uuid = ""
-                if inner_dict['severity'] == "" and inner_dict['alarm_name'] in alarm.name and inner_dict['resource_uuid'] == "":
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # severity = alarm_name = ""    
+                if inner_dict['severity'] == "" and inner_dict['alarm_name'] == "" and inner_dict['resource_uuid'] == \
+                        str(alarm.dimensions['InstanceId']).split("'")[1]:
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #severity = alarm_name = ""    
-                if inner_dict['severity'] == "" and inner_dict['alarm_name'] == "" and inner_dict['resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # resource_uuid = ""    
+                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict[
+                    'alarm_name'] in alarm.name and inner_dict['resource_uuid'] == "":
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #resource_uuid = ""    
-                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] in alarm.name and inner_dict['resource_uuid'] == "":
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # alarm_name = ""    
+                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] == "" and \
+                        inner_dict['resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #alarm_name = ""    
-                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] == "" and inner_dict['resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                # severity = ""    
+                if inner_dict['severity'] == "" and inner_dict['alarm_name'] in alarm.name and inner_dict[
+                    'resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
-                #severity = ""    
-                if inner_dict['severity'] == "" and inner_dict['alarm_name'] in alarm.name and inner_dict['resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
-                    itr += 1                    
-                #Everything provided    
-                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict['alarm_name'] in alarm.name and inner_dict['resource_uuid'] == str(alarm.dimensions['InstanceId']).split("'")[1]:
-                    alarm_list.insert(itr,self.alarm_details(cloudwatch_conn,list_info))
+                    # Everything provided    
+                if inner_dict['severity'] == str(alarm.description).split(';')[0] and inner_dict[
+                    'alarm_name'] in alarm.name and inner_dict['resource_uuid'] == \
+                        str(alarm.dimensions['InstanceId']).split("'")[1]:
+                    alarm_list.insert(itr, self.alarm_details(cloudwatch_conn, list_info))
                     itr += 1
 
             alarm_info['schema_version'] = str(list_info['schema_version'])
-            alarm_info['schema_type'] = 'list_alarm_response'    
+            alarm_info['schema_type'] = 'list_alarm_response'
             alarm_info['list_alarm_response'] = alarm_list
 
-            return alarm_info                  
+            return alarm_info
         except Exception as e:
-                log.error("Error in Getting List : %s",str(e))    
-#-----------------------------------------------------------------------------------------------------------------------------
-    def alarm_details(self,cloudwatch_conn,ack_info):
+            log.error("Error in Getting List : %s", str(e))
+
+    def alarm_details(self, cloudwatch_conn, ack_info):
 
         """Get an individual alarm details specified by alarm_name"""
         try:
-            alarms_details=cloudwatch_conn.describe_alarm_history()  
-            alarm_details_all = dict()     
+            alarms_details = cloudwatch_conn.describe_alarm_history()
+            alarm_details_all = dict()
             alarm_details_dict = dict()
             ack_info_all = ack_info
 
-
             if 'ack_details' in ack_info:
                 ack_info = ack_info['ack_details']
             elif 'alarm_list_request' in ack_info:
-                ack_info = ack_info['alarm_list_request']    
-            
-            is_present = self.is_present(cloudwatch_conn,ack_info['alarm_uuid'])
+                ack_info = ack_info['alarm_list_request']
 
-            for itr in range (len(alarms_details)):
-                if alarms_details[itr].name == is_present['info'].name :#name, timestamp, summary
+            is_present = self.is_present(cloudwatch_conn, ack_info['alarm_uuid'])
+
+            for itr in range(len(alarms_details)):
+                if alarms_details[itr].name == is_present['info'].name:  # name, timestamp, summary
                     if 'created' in alarms_details[itr].summary:
                         alarm_details_dict['status'] = "New"
                     elif 'updated' in alarms_details[itr].summary:
                         alarm_details_dict['status'] = "Update"
-                    elif 'deleted' in alarms_details[itr].summary:   
+                    elif 'deleted' in alarms_details[itr].summary:
                         alarm_details_dict['status'] = "Canceled"
 
-                    status = alarms_details[itr].summary.split()                  
+                    status = alarms_details[itr].summary.split()
                     alarms = cloudwatch_conn.describe_alarms()
                     for alarm in alarms:
                         if str(alarm.description).split(';')[1] == ack_info['alarm_uuid']:
@@ -284,9 +292,9 @@ class MetricAlarm():
                             alarm_details_dict['resource_uuid'] = str(alarm.dimensions['InstanceId']).split("'")[1]
                             alarm_details_dict['description'] = str(alarm.description).split(';')[1]
                             alarm_details_dict['severity'] = str(alarm.description).split(';')[0]
-                            alarm_details_dict['start_date_time'] = str(alarms_details[itr].timestamp) 
+                            alarm_details_dict['start_date_time'] = str(alarms_details[itr].timestamp)
                             alarm_details_dict['vim_type'] = str(ack_info_all['vim_type'])
-                            #TODO : tenant id
+                            # TODO : tenant id
                             if 'ack_details' in ack_info_all:
                                 alarm_details_all['schema_version'] = str(ack_info_all['schema_version'])
                                 alarm_details_all['schema_type'] = 'notify_alarm'
@@ -294,12 +302,12 @@ class MetricAlarm():
                                 return alarm_details_all
 
                             elif 'alarm_list_request' in ack_info_all:
-                                return alarm_details_dict                     
-                  
+                                return alarm_details_dict
+
         except Exception as e:
-            log.error("Error getting alarm details: %s",str(e))
-#-----------------------------------------------------------------------------------------------------------------------------
-    def is_present(self,cloudwatch_conn,alarm_id):
+            log.error("Error getting alarm details: %s", str(e))
+
+    def is_present(self, cloudwatch_conn, alarm_id):
         """Finding alarm from already configured alarms"""
         alarm_info = dict()
         try:
@@ -309,9 +317,7 @@ class MetricAlarm():
                     alarm_info['status'] = True
                     alarm_info['info'] = alarm
                     return alarm_info
-            alarm_info['status'] = False        
+            alarm_info['status'] = False
             return alarm_info
         except Exception as e:
-                log.error("Error Finding Alarm",str(e))             
-#-----------------------------------------------------------------------------------------------------------------------------
-    
\ No newline at end of file
+            log.error("Error Finding Alarm", str(e))
index 1586359..1812002 100644 (file)
 # contact with: wajeeha.hamid@xflowresearch.com
 ##
 
-'''
+"""
 AWS-Plugin implements all the methods of MON to interact with AWS using the BOTO client
-'''
+"""
 
 __author__ = "Wajeeha Hamid"
-__date__   = "18-Sept-2017"
+__date__ = "18-Sept-2017"
 
 import datetime
 import logging
@@ -40,34 +40,34 @@ except:
 
 log = logging.getLogger(__name__)
 
-class Metrics():
 
-    def createMetrics(self,cloudwatch_conn,metric_info):
+class Metrics:
+
+    def createMetrics(self, cloudwatch_conn, metric_info):
         try:
-            
-            '''createMetrics will be returning the metric_uuid=0 and
-             status=True when the metric is supported by AWS'''
 
-            supported=self.check_metric(metric_info['metric_name'])
+            """createMetrics will be returning the metric_uuid=0 and
+             status=True when the metric is supported by AWS"""
+
+            supported = self.check_metric(metric_info['metric_name'])
             metric_resp = dict()
             metric_resp['resource_uuid'] = metric_info['resource_uuid']
-            
-            if supported['status'] == True:
+
+            if supported['status']:
                 metric_resp['status'] = True
                 metric_resp['metric_uuid'] = 0
-                log.debug("Metrics Configured Succesfully : %s" , metric_resp)
+                log.debug("Metrics Configured Successfully : %s", metric_resp)
             else:
                 metric_resp['status'] = False
                 metric_resp['metric_uuid'] = None
-                log.error("Metric name is not supported") 
-            
-            return metric_resp         
+                log.error("Metric name is not supported")
+
+            return metric_resp
 
         except Exception as e:
             log.error("Metric Configuration Failed: " + str(e))
-#-----------------------------------------------------------------------------------------------------------------------------
-    
-    def metricsData(self,cloudwatch_conn,data_info):
+
+    def metricsData(self, cloudwatch_conn, data_info):
 
         """Getting Metrics Stats for an Hour.The datapoints are
         received after every one minute.
@@ -79,43 +79,48 @@ class Metrics():
             timestamp_arr = {}
             value_arr = {}
 
-            supported=self.check_metric(data_info['metric_name'])
+            supported = self.check_metric(data_info['metric_name'])
 
-            if supported['status'] == True:
+            if supported['status']:
                 if int(data_info['collection_period']) % 60 == 0:
-                    metric_stats=cloudwatch_conn.get_metric_statistics(60, datetime.datetime.utcnow() - datetime.timedelta(seconds=int(data_info['collection_period'])),
-                                        datetime.datetime.utcnow(),supported['metric_name'],'AWS/EC2', 'Maximum',
-                                        dimensions={'InstanceId':data_info['resource_uuid']}, unit='Percent')  
+                    metric_stats = cloudwatch_conn.get_metric_statistics(60,
+                                                                         datetime.datetime.utcnow() - datetime.timedelta(
+                                                                             seconds=int(
+                                                                                 data_info['collection_period'])),
+                                                                         datetime.datetime.utcnow(),
+                                                                         supported['metric_name'], 'AWS/EC2', 'Maximum',
+                                                                         dimensions={
+                                                                             'InstanceId': data_info['resource_uuid']},
+                                                                         unit='Percent')
                     index = 0
-                    for itr in range (len(metric_stats)):
+                    for itr in range(len(metric_stats)):
                         timestamp_arr[index] = str(metric_stats[itr]['Timestamp'])
                         value_arr[index] = metric_stats[itr]['Maximum']
-                        index +=1
+                        index += 1
                     metric_info_dict['time_series'] = timestamp_arr
                     metric_info_dict['metrics_series'] = value_arr
                     log.debug("Metrics Data : %s", metric_info_dict)
                     return metric_info_dict
-                else: 
+                else:
                     log.error("Collection Period should be a multiple of 60")
                     return False
 
             else:
                 log.error("Metric name is not supported")
                 return False
-        
+
         except Exception as e:
             log.error("Error returning Metrics Data" + str(e))
 
-#-----------------------------------------------------------------------------------------------------------------------------
-    def updateMetrics(self,cloudwatch_conn,metric_info):
-        
-        '''updateMetrics will be returning the metric_uuid=0 and
-         status=True when the metric is supported by AWS'''
+    def updateMetrics(self, cloudwatch_conn, metric_info):
+
+        """updateMetrics will be returning the metric_uuid=0 and
+         status=True when the metric is supported by AWS"""
         try:
-            supported=self.check_metric(metric_info['metric_name'])
+            supported = self.check_metric(metric_info['metric_name'])
             update_resp = dict()
             update_resp['resource_uuid'] = metric_info['resource_uuid']
-            if supported['status'] == True:
+            if supported['status']:
                 update_resp['status'] = True
                 update_resp['metric_uuid'] = 0
                 log.debug("Metric Updated : %s", update_resp)
@@ -123,21 +128,21 @@ class Metrics():
                 update_resp['status'] = False
                 update_resp['metric_uuid'] = None
                 log.error("Metric name is not supported")
-             
-            return update_resp  
-        
+
+            return update_resp
+
         except Exception as e:
             log.error("Error in Update Metrics" + str(e))
-#-----------------------------------------------------------------------------------------------------------------------------
-    def deleteMetrics(self,cloudwatch_conn,del_info):
-        
-        ''' " Not supported in AWS"
-        Returning the required parameters with status = False'''
+
+    def deleteMetrics(self, cloudwatch_conn, del_info):
+
+        """ " Not supported in AWS"
+        Returning the required parameters with status = False"""
         try:
-            supported=self.check_metric(del_info['metric_name'])
+            supported = self.check_metric(del_info['metric_name'])
             metric_resp = dict()
             del_resp = dict()
-            if supported['status'] == True:      
+            if supported['status']:
                 del_resp['schema_version'] = del_info['schema_version']
                 del_resp['schema_type'] = "delete_metric_response"
                 del_resp['metric_name'] = del_info['metric_name']
@@ -147,54 +152,52 @@ class Metrics():
                 del_resp['tenant_uuid'] = del_info['tenant_uuid']
                 del_resp['correlation_id'] = del_info['correlation_uuid']
                 del_resp['status'] = False
-                log.info("Metric Deletion Not supported in AWS : %s",del_resp)
+                log.info("Metric Deletion Not supported in AWS : %s", del_resp)
                 return del_resp
             else:
                 log.error("Metric name is not supported")
                 return False
 
         except Exception as e:
-                log.error(" Metric Deletion Not supported in AWS : " + str(e))
-#------------------------------------------------------------------------------------------------------------------------------------
-    
-    def listMetrics(self,cloudwatch_conn ,list_info):
+            log.error(" Metric Deletion Not supported in AWS : " + str(e))
+
+    def listMetrics(self, cloudwatch_conn, list_info):
 
-        '''Returns the list of available AWS/EC2 metrics on which
-        alarms have been configured and the metrics are being monitored'''
+        """Returns the list of available AWS/EC2 metrics on which
+        alarms have been configured and the metrics are being monitored"""
         try:
             supported = self.check_metric(list_info['metric_name'])
-            if supported['status'] == True: 
+            if supported['status']:
                 metrics_list = []
-                metrics_data = dict()    
+                metrics_data = dict()
 
-                #To get the list of associated metrics with the alarms
+                # To get the list of associated metrics with the alarms
                 alarms = cloudwatch_conn.describe_alarms()
                 itr = 0
                 if list_info['metric_name'] == "":
                     for alarm in alarms:
                         metrics_info = dict()
-                        instance_id = str(alarm.dimensions['InstanceId']).split("'")[1] 
-                        metrics_info['metric_name'] = str(alarm.metric) 
-                        metrics_info['metric_uuid'] = 0     
-                        metrics_info['metric_unit'] = str(alarm.unit)    
-                        metrics_info['resource_uuid'] = instance_id 
-                        metrics_list.insert(itr,metrics_info)
+                        instance_id = str(alarm.dimensions['InstanceId']).split("'")[1]
+                        metrics_info['metric_name'] = str(alarm.metric)
+                        metrics_info['metric_uuid'] = 0
+                        metrics_info['metric_unit'] = str(alarm.unit)
+                        metrics_info['resource_uuid'] = instance_id
+                        metrics_list.insert(itr, metrics_info)
                         itr += 1
                     log.info(metrics_list)
                     return metrics_list
-                else: 
+                else:
                     for alarm in alarms:
                         metrics_info = dict()
                         if alarm.metric == supported['metric_name']:
-                            instance_id = str(alarm.dimensions['InstanceId']).split("'")[1] 
+                            instance_id = str(alarm.dimensions['InstanceId']).split("'")[1]
                             metrics_info['metric_name'] = str(alarm.metric)
-                            metrics_info['metric_uuid'] = 0     
-                            metrics_info['metric_unit'] = str(alarm.unit)    
+                            metrics_info['metric_uuid'] = 0
+                            metrics_info['metric_unit'] = str(alarm.unit)
                             metrics_info['resource_uuid'] = instance_id
-                            metrics_list.insert(itr,metrics_info)
+                            metrics_list.insert(itr, metrics_info)
                             itr += 1
-                    return metrics_list               
-                log.debug("Metrics List : %s",metrics_list)
+                    return metrics_list
             else:
                 log.error("Metric name is not supported")
                 return False
@@ -202,11 +205,9 @@ class Metrics():
         except Exception as e:
             log.error("Error in Getting Metric List " + str(e))
 
-#------------------------------------------------------------------------------------------------------------------------------------
+    def check_metric(self, metric_name):
 
-    def check_metric(self,metric_name):
-    
-        ''' Checking whether the metric is supported by AWS '''
+        """ Checking whether the metric is supported by AWS """
         try:
             check_resp = dict()
             # metric_name
@@ -240,22 +241,13 @@ class Metrics():
                 metric_status = False
                 log.info("Metric Not Supported by AWS plugin ")
             check_resp['metric_name'] = metric_name
-            #status
-            if metric_status == True:
+            # status
+            if metric_status:
                 check_resp['status'] = True
             else:
                 check_resp['status'] = False
 
             return check_resp
 
-        except Exception as e: 
-            log.error("Error in Plugin Inputs %s",str(e))     
-#--------------------------------------------------------------------------------------------------------------------------------------
-
-      
-
-
-
-
-
-
+        except Exception as e:
+            log.error("Error in Plugin Inputs %s", str(e))
index 40e7fe5..c125bab 100644 (file)
 # contact with: wajeeha.hamid@xflowresearch.com
 ##
 
-'''
+"""
 AWS-Plugin implements all the methods of MON to interact with AWS using the BOTO client
-'''
-from io import open
-from osm_mon.core.message_bus.producer import KafkaProducer
+"""
+from io import UnsupportedOperation
+
+from osm_mon.core.settings import Config
 from osm_mon.plugins.CloudWatch.metric_alarms import MetricAlarm
 from osm_mon.plugins.CloudWatch.metrics import Metrics
 
 __author__ = "Wajeeha Hamid"
-__date__   = "18-September-2017"
+__date__ = "18-September-2017"
 
-import json
 import logging
 
 log = logging.getLogger(__name__)
 
-class plugin_alarms():
+
+class plugin_alarms:
     """Receives Alarm info from MetricAlarm and connects with the consumer/producer"""
-    def __init__ (self): 
+
+    def __init__(self):
+        self._cfg = Config.instance()
         self.metricAlarm = MetricAlarm()
         self.metric = Metrics()
-        self.producer = KafkaProducer('')     
-#---------------------------------------------------------------------------------------------------------------------------   
-    def configure_alarm(self,alarm_info):
-        alarm_id = self.metricAlarm.config_alarm(self.cloudwatch_conn,alarm_info)
+
+    def configure_alarm(self, alarm_info):
+        alarm_id = self.metricAlarm.config_alarm(self.cloudwatch_conn, alarm_info)
         return alarm_id
-#---------------------------------------------------------------------------------------------------------------------------          
-    def update_alarm_configuration(self,test):
-        alarm_id = self.metricAlarm.update_alarm(self.cloudwatch_conn,test)
+
+    def update_alarm_configuration(self, test):
+        alarm_id = self.metricAlarm.update_alarm(self.cloudwatch_conn, test)
         return alarm_id
-#---------------------------------------------------------------------------------------------------------------------------            
-    def delete_alarm(self,alarm_id):
-        return self.metricAlarm.delete_Alarm(self.cloudwatch_conn,alarm_id)
-#---------------------------------------------------------------------------------------------------------------------------  
-    def get_alarms_list(self,instance_id):
-        return self.metricAlarm.alarms_list(self.cloudwatch_conn,instance_id) 
-#---------------------------------------------------------------------------------------------------------------------------            
-    def get_ack_details(self,ack_info):
-        return self.metricAlarm.alarm_details(self.cloudwatch_conn,ack_info)
-#---------------------------------------------------------------------------------------------------------------------------            
-    def get_metrics_data(self,metric_name,period,instance_id):
-        return self.metric.metricsData(self.cloudwatch_conn,metric_name,period,instance_id)
-#--------------------------------------------------------------------------------------------------------------------------- 
-
-    def alarm_calls(self,message,aws_conn):
+
+    def delete_alarm(self, alarm_id):
+        return self.metricAlarm.delete_Alarm(self.cloudwatch_conn, alarm_id)
+
+    def get_alarms_list(self, instance_id):
+        return self.metricAlarm.alarms_list(self.cloudwatch_conn, instance_id)
+
+    def get_ack_details(self, ack_info):
+        return self.metricAlarm.alarm_details(self.cloudwatch_conn, ack_info)
+
+    def get_metrics_data(self, metric_name, period, instance_id):
+        # TODO: Investigate and fix this call
+        return self.metric.metricsData(self.cloudwatch_conn, metric_name, period, instance_id)
+
+    def alarm_calls(self, key: str, alarm_info: dict, aws_conn: dict):
         """Gets the message from the common consumer"""
         try:
             self.cloudwatch_conn = aws_conn['cloudwatch_connection']
-            self.ec2_conn = aws_conn['ec2_connection'] 
-
-            log.info("Action required against: %s" % (message.topic))
-            alarm_info = json.loads(message.value)
+            self.ec2_conn = aws_conn['ec2_connection']
 
-            if message.key == "create_alarm_request":  
+            if key == "create_alarm_request":
                 alarm_inner_dict = alarm_info['alarm_create_request']
-                metric_status = self.check_metric(alarm_inner_dict['metric_name'])                            
-             
-                if self.check_resource(alarm_inner_dict['resource_uuid']) == True and metric_status['status'] == True:
-                    log.debug ("Resource and Metrics exists")
-                
+                metric_status = self.check_metric(alarm_inner_dict['metric_name'])
+
+                if self.check_resource(alarm_inner_dict['resource_uuid']) and metric_status['status']:
+                    log.debug("Resource and Metrics exists")
+
                     alarm_info['alarm_create_request']['metric_name'] = metric_status['metric_name']
-                    #Generate a valid response message, send via producer
-                    config_resp = self.configure_alarm(alarm_info) #alarm_info = message.value
-             
-                    if config_resp == None:
+                    # Generate a valid response message, send via producer
+                    config_resp = self.configure_alarm(alarm_info)  # alarm_info = message.value
+
+                    if config_resp is None:
                         log.debug("Alarm Already exists")
-                        payload = json.dumps(config_resp)                                   
-                        file = open('../../core/models/create_alarm_resp.json','wb').write((payload))
-                        self.producer.publish_alarm_response(key='create_alarm_response',message=payload)
-             
-                    else: 
-                        payload = json.dumps(config_resp)                                
-                        file = open('../../core/models/create_alarm_resp.json','wb').write((payload))                           
-                        self.producer.publish_alarm_response(key='create_alarm_response',message=payload)
-                        log.info("New alarm created with alarm info: %s", config_resp)                           
-             
+                        # TODO: This should return a response with status False
+                        return config_resp
+
+                    else:
+                        log.info("New alarm created with alarm info: %s", config_resp)
+                        return config_resp
+
                 else:
-                    log.error("Resource ID doesn't exists")                
-                
-            elif message.key == "acknowledge_alarm":
+                    log.error("Resource ID doesn't exists")
+
+            elif key == "acknowledge_alarm":
                 alarm_inner_dict = alarm_info['ack_details']
-                
-                if self.check_resource(alarm_inner_dict['resource_uuid']) == True: 
-                    alarm_info = json.loads(message.value)
-                    #Generate a valid response message, send via producer
+
+                if self.check_resource(alarm_inner_dict['resource_uuid']):
                     ack_details = self.get_ack_details(alarm_info)
-                    payload = json.dumps(ack_details)                                  
-                    file = open('../../core/models/notify_alarm.json','wb').write((payload))
-                    self.producer.notify_alarm(key='notify_alarm',message=payload)
                     log.info("Acknowledge sent: %s", ack_details)
+                    return ack_details
 
                 else:
-                    log.error("Resource ID is Incorrect")                        
-
+                    log.error("Resource ID is Incorrect")
 
-            elif message.key == "update_alarm_request":                         
+            elif key == "update_alarm_request":
                 alarm_inner_dict = alarm_info['alarm_update_request']
                 metric_status = self.check_metric(alarm_inner_dict['metric_name'])
-                
-                if metric_status['status'] == True:
-                    log.debug ("Resource and Metrics exists")
+
+                if metric_status['status']:
+                    log.debug("Resource and Metrics exists")
                     alarm_info['alarm_update_request']['metric_name'] = metric_status['metric_name']
-                    #Generate a valid response message, send via producer
+                    # Generate a valid response message, send via producer
                     update_resp = self.update_alarm_configuration(alarm_info)
 
-                    if update_resp == None:                                    
-                        payload = json.dumps(update_resp)                                   
-                        file = open('../../core/models/update_alarm_resp.json','wb').write((payload))
-                        self.producer.update_alarm_response(key='update_alarm_response',message=payload)
+                    if update_resp is None:
+                        # TODO: This should return a response with status False
                         log.debug("Alarm Already exists")
+                        return update_resp
 
-                    else: 
-                        payload = json.dumps(update_resp)                                   
-                        file = open('../../core/models/update_alarm_resp.json','wb').write((payload))
-                        self.producer.update_alarm_response(key='update_alarm_response',message=payload)
-                        log.info("Alarm Updated with alarm info: %s", update_resp)                           
+                    else:
+                        log.info("Alarm Updated with alarm info: %s", update_resp)
+                        return update_resp
 
                 else:
-                    log.info ("Metric Not Supported")
-         
-            
-            elif message.key == "delete_alarm_request":  
-                del_info = json.loads(message.value)
-                #Generate a valid response message, send via producer
-                del_resp = self.delete_alarm(del_info)
-                payload = json.dumps(del_resp)                                   
-                file = open('../../core/models/delete_alarm_resp.json','wb').write((payload))
-                self.producer.delete_alarm_response(key='delete_alarm_response',message=payload)
+                    log.info("Metric Not Supported")
+
+            elif key == "delete_alarm_request":
+                # Generate a valid response message, send via producer
+                del_resp = self.delete_alarm(alarm_info)
                 log.info("Alarm Deleted with alarm info: %s", del_resp)
+                return del_resp
 
-       
-            elif message.key == "alarm_list_request":
+            elif key == "alarm_list_request":
                 alarm_inner_dict = alarm_info['alarm_list_request']
-                
-                if self.check_resource(alarm_inner_dict['resource_uuid']) == True or alarm_inner_dict['resource_uuid'] == "": 
-                    #Generate a valid response message, send via producer
-                    list_resp = self.get_alarms_list(alarm_info)#['alarm_names']
-                    payload = json.dumps(list_resp)                                                                 
-                    file = open('../../core/models/list_alarm_resp.json','wb').write((payload))
-                    self.producer.list_alarm_response(key='list_alarm_response',message=payload)
 
+                if self.check_resource(alarm_inner_dict['resource_uuid']) or alarm_inner_dict['resource_uuid'] == "":
+                    # Generate a valid response message, send via producer
+                    list_resp = self.get_alarms_list(alarm_info)  # ['alarm_names']
+                    return list_resp
                 else:
-                    log.error("Resource ID is Incorrect")             
+                    log.error("Resource ID is Incorrect")
 
             else:
-                log.debug("Unknown key, no action will be performed")    
+                raise UnsupportedOperation("Unknown key, no action will be performed")
 
         except Exception as e:
-                log.error("Message retrieval exception: %s", str(e))             
-#--------------------------------------------------------------------------------------------------------------------------- 
-    def check_resource(self,resource_uuid):
-        '''Finding Resource with the resource_uuid'''
+            log.error("Message retrieval exception: %s", str(e))
+
+    def check_resource(self, resource_uuid):
+        """Finding Resource with the resource_uuid"""
         try:
             check_resp = dict()
             instances = self.ec2_conn.get_all_instance_status()
 
-            #resource_id
+            # resource_id
             for instance_id in instances:
                 instance_id = str(instance_id).split(':')[1]
 
                 if instance_id == resource_uuid:
                     check_resp['resource_uuid'] = resource_uuid
-                    return True 
+                    return True
             return False
 
-        except Exception as e: 
-            log.error("Error in Plugin Inputs %s",str(e)) 
-#--------------------------------------------------------------------------------------------------------------------------- 
-    def check_metric(self,metric_name):
-        ''' Checking whether the metric is supported by AWS '''
+        except Exception as e:
+            log.error("Error in Plugin Inputs %s", str(e))
+
+    def check_metric(self, metric_name):
+        """ Checking whether the metric is supported by AWS """
         try:
             check_resp = dict()
-            
-            #metric_name
+
+            # metric_name
             if metric_name == 'CPU_UTILIZATION':
                 metric_name = 'CPUUtilization'
                 metric_status = True
@@ -225,12 +205,11 @@ class plugin_alarms():
                 metric_name = None
                 metric_status = False
             check_resp['metric_name'] = metric_name
-            #status
+            # status
 
-            if metric_status == True:
+            if metric_status:
                 check_resp['status'] = True
-                return check_resp   
+                return check_resp
 
-        except Exception as e: 
-            log.error("Error in Plugin Inputs %s",str(e)) 
-#--------------------------------------------------------------------------------------------------------------------------- 
+        except Exception as e:
+            log.error("Error in Plugin Inputs %s", str(e))
index 36b89e3..d31b608 100644 (file)
 # contact with: wajeeha.hamid@xflowresearch.com
 ##
 
-'''
+"""
 AWS-Plugin implements all the methods of MON to interact with AWS using the BOTO client
-'''
-from osm_mon.core.message_bus.producer import KafkaProducer
+"""
+from io import UnsupportedOperation
+
+from osm_mon.core.settings import Config
 from osm_mon.plugins.CloudWatch.metrics import Metrics
 
 __author__ = "Wajeeha Hamid"
-__date__   = "18-September-2017"
+__date__ = "18-September-2017"
 
-import json
 import logging
 
 log = logging.getLogger(__name__)
 
-class plugin_metrics():
+
+class plugin_metrics:
     """Receives Alarm info from MetricAlarm and connects with the consumer/producer """
-    def __init__ (self): 
+
+    def __init__(self):
+        self._cfg = Config.instance()
         self.metric = Metrics()
-        self.producer = KafkaProducer('')
-#---------------------------------------------------------------------------------------------------------------------------   
-    def create_metric_request(self,metric_info):
-        '''Comaptible API using normalized parameters'''
-        metric_resp = self.metric.createMetrics(self.cloudwatch_conn,metric_info)
+
+    def create_metric_request(self, metric_info):
+        """Compatible API using normalized parameters"""
+        metric_resp = self.metric.createMetrics(self.cloudwatch_conn, metric_info)
         return metric_resp
-#---------------------------------------------------------------------------------------------------------------------------          
-    def update_metric_request(self,updated_info):
-        '''Comaptible API using normalized parameters'''
-        update_resp = self.metric.updateMetrics(self.cloudwatch_conn,updated_info)
+
+    def update_metric_request(self, updated_info):
+        """Compatible API using normalized parameters"""
+        update_resp = self.metric.updateMetrics(self.cloudwatch_conn, updated_info)
         return update_resp
-#---------------------------------------------------------------------------------------------------------------------------            
-    def delete_metric_request(self,delete_info):
-        '''Comaptible API using normalized parameters'''
-        del_resp = self.metric.deleteMetrics(self.cloudwatch_conn,delete_info)
+
+    def delete_metric_request(self, delete_info):
+        """Compatible API using normalized parameters"""
+        del_resp = self.metric.deleteMetrics(self.cloudwatch_conn, delete_info)
         return del_resp
-#---------------------------------------------------------------------------------------------------------------------------  
-    def list_metrics_request(self,list_info):
-        '''Comaptible API using normalized parameters'''
-        list_resp = self.metric.listMetrics(self.cloudwatch_conn,list_info)
+
+    def list_metrics_request(self, list_info):
+        """Compatible API using normalized parameters"""
+        list_resp = self.metric.listMetrics(self.cloudwatch_conn, list_info)
         return list_resp
-#---------------------------------------------------------------------------------------------------------------------------                        
-    def read_metrics_data(self,list_info):
-        '''Comaptible API using normalized parameters
-        Read all metric data related to a specified metric'''
-        data_resp=self.metric.metricsData(self.cloudwatch_conn,list_info)
+
+    def read_metrics_data(self, list_info):
+        """Compatible API using normalized parameters
+        Read all metric data related to a specified metric"""
+        data_resp = self.metric.metricsData(self.cloudwatch_conn, list_info)
         return data_resp
-#--------------------------------------------------------------------------------------------------------------------------- 
 
-    def metric_calls(self,message,aws_conn):
+    def metric_calls(self, key: str, metric_info: dict, aws_conn: dict):
         """Gets the message from the common consumer"""
-        
+
         try:
             self.cloudwatch_conn = aws_conn['cloudwatch_connection']
             self.ec2_conn = aws_conn['ec2_connection']
 
-            metric_info = json.loads(message.value)
             metric_response = dict()
 
-            if metric_info['vim_type'] == 'AWS':
-                log.debug ("VIM support : AWS")
-
-            # Check the Functionlity that needs to be performed: topic = 'alarms'/'metrics'/'Access_Credentials'
-                if message.topic == "metric_request":
-                    log.info("Action required against: %s" % (message.topic))
-
-                    if message.key == "create_metric_request":                            
-                        if self.check_resource(metric_info['metric_create_request']['resource_uuid']) == True:
-                            metric_resp = self.create_metric_request(metric_info['metric_create_request']) #alarm_info = message.value
-                            metric_response['schema_version'] = metric_info['schema_version']
-                            metric_response['schema_type']    = "create_metric_response"
-                            metric_response['metric_create_response'] = metric_resp
-                            payload = json.dumps(metric_response)                                                                  
-                            file = open('../../core/models/create_metric_resp.json','wb').write((payload))
-                            self.producer.publish_metrics_response(key='create_metric_response', message=payload, topic ='metric_response')
-                            
-                            log.info("Metric configured: %s", metric_resp)
-                            return metric_response
-                            
-                    elif message.key == "update_metric_request":
-                        if self.check_resource(metric_info['metric_create_request']['resource_uuid']) == True:
-                            update_resp = self.update_metric_request(metric_info['metric_create_request'])
-                            metric_response['schema_version'] = metric_info['schema_version']
-                            metric_response['schema_type'] = "update_metric_response"
-                            metric_response['metric_update_response'] = update_resp
-                            payload = json.dumps(metric_response)                                                                                               
-                            file = open('../../core/models/update_metric_resp.json','wb').write((payload))
-                            self.producer.update_metric_response(key='update_metric_response',message=payload,topic = 'metric_response')
-
-                            log.info("Metric Updates: %s",metric_response)
-                            return metric_response
-                            
-                    elif message.key == "delete_metric_request":
-                        if self.check_resource(metric_info['resource_uuid']) == True:
-                            del_resp=self.delete_metric_request(metric_info)
-                            payload = json.dumps(del_resp)                                                                                               
-                            file = open('../../core/models/delete_metric_resp.json','wb').write((payload))
-                            self.producer.delete_metric_response(key='delete_metric_response',message=payload,topic = 'metric_response')
-
-                            log.info("Metric Deletion Not supported in AWS : %s",del_resp)
-                            return del_resp
-
-                    elif message.key == "list_metric_request": 
-                        if self.check_resource(metric_info['metrics_list_request']['resource_uuid']) == True:
-                            list_resp = self.list_metrics_request(metric_info['metrics_list_request'])
-                            metric_response['schema_version'] = metric_info['schema_version']
-                            metric_response['schema_type'] = "list_metric_response"
-                            metric_response['correlation_id'] = metric_info['metrics_list_request']['correlation_id']
-                            metric_response['vim_type'] = metric_info['vim_type']
-                            metric_response['metrics_list'] = list_resp
-                            payload = json.dumps(metric_response)                                                                                                
-                            file = open('../../core/models/list_metric_resp.json','wb').write((payload))
-                            self.producer.list_metric_response(key='list_metrics_response',message=payload,topic = 'metric_response')
-
-                            log.info("Metric List: %s",metric_response)
-                            return metric_response
-
-                    elif message.key == "read_metric_data_request":
-                        if self.check_resource(metric_info['resource_uuid']) == True:
-                            data_resp = self.read_metrics_data(metric_info)
-                            metric_response['schema_version'] = metric_info['schema_version']
-                            metric_response['schema_type'] = "read_metric_data_response"
-                            metric_response['metric_name'] = metric_info['metric_name']
-                            metric_response['metric_uuid'] = metric_info['metric_uuid']
-                            metric_response['correlation_id'] = metric_info['correlation_uuid']
-                            metric_response['resource_uuid'] = metric_info['resource_uuid']
-                            metric_response['tenant_uuid'] = metric_info['tenant_uuid']
-                            metric_response['metrics_data'] = data_resp
-                            payload = json.dumps(metric_response)                                                                
-                            file = open('../../core/models/read_metric_data_resp.json','wb').write((payload))
-                            self.producer.read_metric_data_response(key='read_metric_data_response',message=payload,topic = 'metric_response')
-                            
-                            log.info("Metric Data Response: %s",metric_response)
-                            return metric_response 
-
-                    else:
-                        log.debug("Unknown key, no action will be performed")
-                else:
-                    log.info("Message topic not relevant to this plugin: %s",
-                         message.topic)
-            
+            log.debug("VIM support : AWS")
+
+            if key == "create_metric_request":
+                if self.check_resource(metric_info['metric_create_request']['resource_uuid']):
+                    metric_resp = self.create_metric_request(
+                        metric_info['metric_create_request'])  # alarm_info = message.value
+                    metric_response['schema_version'] = metric_info['schema_version']
+                    metric_response['schema_type'] = "create_metric_response"
+                    metric_response['metric_create_response'] = metric_resp
+                    log.info("Metric configured: %s", metric_resp)
+                    return metric_response
+
+            elif key == "update_metric_request":
+                if self.check_resource(metric_info['metric_create_request']['resource_uuid']):
+                    update_resp = self.update_metric_request(metric_info['metric_create_request'])
+                    metric_response['schema_version'] = metric_info['schema_version']
+                    metric_response['schema_type'] = "update_metric_response"
+                    metric_response['metric_update_response'] = update_resp
+                    log.info("Metric Updates: %s", metric_response)
+                    return metric_response
+
+            elif key == "delete_metric_request":
+                if self.check_resource(metric_info['resource_uuid']):
+                    del_resp = self.delete_metric_request(metric_info)
+                    log.info("Metric Deletion Not supported in AWS : %s", del_resp)
+                    return del_resp
+
+            elif key == "list_metric_request":
+                if self.check_resource(metric_info['metrics_list_request']['resource_uuid']):
+                    list_resp = self.list_metrics_request(metric_info['metrics_list_request'])
+                    metric_response['schema_version'] = metric_info['schema_version']
+                    metric_response['schema_type'] = "list_metric_response"
+                    metric_response['correlation_id'] = metric_info['metrics_list_request']['correlation_id']
+                    metric_response['vim_type'] = metric_info['vim_type']
+                    metric_response['metrics_list'] = list_resp
+                    log.info("Metric List: %s", metric_response)
+                    return metric_response
+
+            elif key == "read_metric_data_request":
+                if self.check_resource(metric_info['resource_uuid']):
+                    data_resp = self.read_metrics_data(metric_info)
+                    metric_response['schema_version'] = metric_info['schema_version']
+                    metric_response['schema_type'] = "read_metric_data_response"
+                    metric_response['metric_name'] = metric_info['metric_name']
+                    metric_response['metric_uuid'] = metric_info['metric_uuid']
+                    metric_response['correlation_id'] = metric_info['correlation_uuid']
+                    metric_response['resource_uuid'] = metric_info['resource_uuid']
+                    metric_response['tenant_uuid'] = metric_info['tenant_uuid']
+                    metric_response['metrics_data'] = data_resp
+                    log.info("Metric Data Response: %s", metric_response)
+                    return metric_response
+
+            else:
+                raise UnsupportedOperation("Unknown key, no action will be performed")
+
         except Exception as e:
             log.error("Consumer exception: %s", str(e))
 
-#---------------------------------------------------------------------------------------------------------------------------
-    def check_resource(self,resource_uuid):
+    def check_resource(self, resource_uuid):
 
-        '''Checking the resource_uuid is present in EC2 instances'''
+        """Checking the resource_uuid is present in EC2 instances"""
         try:
             check_resp = dict()
             instances = self.ec2_conn.get_all_instance_status()
             status_resource = False
 
-            #resource_id
+            # resource_id
             for instance_id in instances:
                 instance_id = str(instance_id).split(':')[1]
                 if instance_id == resource_uuid:
@@ -179,10 +152,8 @@ class plugin_metrics():
                 else:
                     status_resource = False
 
-            #status
+            # status
             return status_resource
 
-        except Exception as e: 
-            log.error("Error in Plugin Inputs %s",str(e))          
-#---------------------------------------------------------------------------------------------------------------------------   
-
+        except Exception as e:
+            log.error("Error in Plugin Inputs %s", str(e))
diff --git a/osm_mon/plugins/OpenStack/Aodh/alarm_handler.py b/osm_mon/plugins/OpenStack/Aodh/alarm_handler.py
new file mode 100644 (file)
index 0000000..7e6347f
--- /dev/null
@@ -0,0 +1,390 @@
+# Copyright 2017 Intel Research and Development Ireland Limited
+# *************************************************************
+
+# This file is part of OSM Monitoring module
+# All Rights Reserved to Intel Corporation
+
+# Licensed under the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License. You may obtain
+# a copy of the License at
+
+#         http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+# License for the specific language governing permissions and limitations
+# under the License.
+
+# For those usages not covered by the Apache License, Version 2.0 please
+# contact: helena.mcgough@intel.com or adrian.hoban@intel.com
+##
+"""Carry out alarming requests via Aodh API."""
+
+import json
+import logging
+from io import UnsupportedOperation
+
+import six
+
+from osm_mon.core.auth import AuthManager
+from osm_mon.core.database import DatabaseManager
+from osm_mon.core.settings import Config
+from osm_mon.plugins.OpenStack.Gnocchi.metric_handler import METRIC_MAPPINGS
+from osm_mon.plugins.OpenStack.common import Common
+from osm_mon.plugins.OpenStack.response import OpenStackResponseBuilder
+
+log = logging.getLogger(__name__)
+
+SEVERITIES = {
+    "warning": "low",
+    "minor": "low",
+    "major": "moderate",
+    "critical": "critical",
+    "indeterminate": "critical"}
+
+STATISTICS = {
+    "average": "mean",
+    "minimum": "min",
+    "maximum": "max",
+    "count": "count",
+    "sum": "sum"}
+
+
+class OpenstackAlarmHandler(object):
+    """Carries out alarming requests and responses via Aodh API."""
+
+    def __init__(self):
+        """Create the OpenStack alarming instance."""
+        self._database_manager = DatabaseManager()
+        self._auth_manager = AuthManager()
+        self._cfg = Config.instance()
+
+        # Use the Response class to generate valid json response messages
+        self._response = OpenStackResponseBuilder()
+
+    def handle_message(self, key: str, values: dict, vim_uuid: str):
+        """
+        Processes alarm request message depending on it's key
+        :param key: Kafka message key
+        :param values: Dict containing alarm request data. Follows models defined in core.models.
+        :param vim_uuid: UUID of the VIM to handle the alarm request.
+        :return: Dict containing alarm response data. Follows models defined in core.models.
+        """
+
+        log.info("OpenStack alarm action required.")
+
+        verify_ssl = self._auth_manager.is_verify_ssl(vim_uuid)
+
+        auth_token = Common.get_auth_token(vim_uuid, verify_ssl=verify_ssl)
+
+        alarm_endpoint = Common.get_endpoint("alarming", vim_uuid, verify_ssl=verify_ssl)
+        metric_endpoint = Common.get_endpoint("metric", vim_uuid, verify_ssl=verify_ssl)
+
+        vim_account = self._auth_manager.get_credentials(vim_uuid)
+        vim_config = json.loads(vim_account.config)
+
+        if key == "create_alarm_request":
+            alarm_details = values['alarm_create_request']
+            alarm_id = None
+            status = False
+            try:
+                metric_name = alarm_details['metric_name'].lower()
+                resource_id = alarm_details['resource_uuid']
+
+                self.check_for_metric(auth_token, metric_endpoint, metric_name, resource_id, verify_ssl)
+
+                alarm_id = self.configure_alarm(
+                    alarm_endpoint, auth_token, alarm_details, vim_config, verify_ssl)
+
+                log.info("Alarm successfully created")
+                self._database_manager.save_alarm(alarm_id,
+                                                  vim_uuid,
+                                                  alarm_details['threshold_value'],
+                                                  alarm_details['operation'].lower(),
+                                                  alarm_details['metric_name'].lower(),
+                                                  alarm_details['vdu_name'].lower(),
+                                                  alarm_details['vnf_member_index'],
+                                                  alarm_details['ns_id'].lower()
+                                                  )
+                status = True
+            except Exception as e:
+                log.exception("Error creating alarm")
+                raise e
+            finally:
+                return self._response.generate_response('create_alarm_response',
+                                                        cor_id=alarm_details['correlation_id'],
+                                                        status=status,
+                                                        alarm_id=alarm_id)
+
+        elif key == "list_alarm_request":
+            list_details = values['alarm_list_request']
+            alarm_list = None
+            try:
+                alarm_list = self.list_alarms(
+                    alarm_endpoint, auth_token, list_details, verify_ssl)
+            except Exception as e:
+                log.exception("Error listing alarms")
+                raise e
+            finally:
+                return self._response.generate_response('list_alarm_response',
+                                                        cor_id=list_details['correlation_id'],
+                                                        alarm_list=alarm_list)
+
+        elif key == "delete_alarm_request":
+            request_details = values['alarm_delete_request']
+            alarm_id = request_details['alarm_uuid']
+            status = False
+            try:
+                self.delete_alarm(
+                    alarm_endpoint, auth_token, alarm_id, verify_ssl)
+                status = True
+            except Exception as e:
+                log.exception("Error deleting alarm")
+                raise e
+            finally:
+                return self._response.generate_response('delete_alarm_response',
+                                                        cor_id=request_details['correlation_id'],
+                                                        status=status,
+                                                        alarm_id=alarm_id)
+
+        elif key == "acknowledge_alarm_request":
+            try:
+                alarm_id = values['ack_details']['alarm_uuid']
+
+                self.update_alarm_state(
+                    alarm_endpoint, auth_token, alarm_id, verify_ssl)
+
+                log.info("Acknowledged the alarm and cleared it.")
+            except Exception as e:
+                log.exception("Error acknowledging alarm")
+                raise
+            finally:
+                return None
+
+        elif key == "update_alarm_request":
+            # Update alarm configurations
+            alarm_details = values['alarm_update_request']
+            alarm_id = None
+            status = False
+            try:
+                alarm_id = self.update_alarm(
+                    alarm_endpoint, auth_token, alarm_details, vim_config, verify_ssl)
+                status = True
+            except Exception as e:
+                log.exception("Error updating alarm")
+                raise e
+            finally:
+                return self._response.generate_response('update_alarm_response',
+                                                        cor_id=alarm_details['correlation_id'],
+                                                        status=status,
+                                                        alarm_id=alarm_id)
+
+        else:
+            raise UnsupportedOperation("Unknown key {}, no action will be performed.".format(key))
+
+    def configure_alarm(self, alarm_endpoint, auth_token, values, vim_config, verify_ssl):
+        """Create requested alarm in Aodh."""
+        url = "{}/v2/alarms/".format(alarm_endpoint)
+
+        # Check if the desired alarm is supported
+        alarm_name = values['alarm_name'].lower()
+        metric_name = values['metric_name'].lower()
+        resource_id = values['resource_uuid']
+
+        if metric_name not in METRIC_MAPPINGS.keys():
+            raise KeyError("Metric {} is not supported.".format(metric_name))
+
+        if 'granularity' in vim_config and 'granularity' not in values:
+            values['granularity'] = vim_config['granularity']
+        payload = self.check_payload(values, metric_name, resource_id,
+                                     alarm_name)
+        new_alarm = Common.perform_request(
+            url, auth_token, req_type="post", payload=payload, verify_ssl=verify_ssl)
+        return json.loads(new_alarm.text)['alarm_id']
+
+    def delete_alarm(self, endpoint, auth_token, alarm_id, verify_ssl):
+        """Delete alarm function."""
+        url = "{}/v2/alarms/%s".format(endpoint) % alarm_id
+
+        result = Common.perform_request(
+            url, auth_token, req_type="delete", verify_ssl=verify_ssl)
+        if str(result.status_code) == "404":
+            raise ValueError("Alarm {} doesn't exist".format(alarm_id))
+
+    def list_alarms(self, endpoint, auth_token, list_details, verify_ssl):
+        """Generate the requested list of alarms."""
+        url = "{}/v2/alarms/".format(endpoint)
+        a_list, name_list, sev_list, res_list = [], [], [], []
+
+        # TODO(mcgoughh): for now resource_id is a mandatory field
+        # Check for a resource id
+        try:
+            resource = list_details['resource_uuid']
+            name = list_details['alarm_name'].lower()
+            severity = list_details['severity'].lower()
+            sev = SEVERITIES[severity]
+        except KeyError as e:
+            log.warning("Missing parameter for alarm list request: %s", e)
+            raise e
+
+        # Perform the request to get the desired list
+        try:
+            result = Common.perform_request(
+                url, auth_token, req_type="get", verify_ssl=verify_ssl)
+
+            if result is not None:
+                # Get list based on resource id
+                for alarm in json.loads(result.text):
+                    rule = alarm['gnocchi_resources_threshold_rule']
+                    if resource == rule['resource_id']:
+                        res_list.append(alarm['alarm_id'])
+
+                # Generate specified listed if requested
+                if name is not None and sev is not None:
+                    log.info("Return a list of %s alarms with %s severity.",
+                             name, sev)
+                    for alarm in json.loads(result.text):
+                        if name == alarm['name']:
+                            name_list.append(alarm['alarm_id'])
+                    for alarm in json.loads(result.text):
+                        if sev == alarm['severity']:
+                            sev_list.append(alarm['alarm_id'])
+                    name_sev_list = list(set(name_list).intersection(sev_list))
+                    a_list = list(set(name_sev_list).intersection(res_list))
+                elif name is not None:
+                    log.info("Returning a %s list of alarms.", name)
+                    for alarm in json.loads(result.text):
+                        if name == alarm['name']:
+                            name_list.append(alarm['alarm_id'])
+                    a_list = list(set(name_list).intersection(res_list))
+                elif sev is not None:
+                    log.info("Returning %s severity alarm list.", sev)
+                    for alarm in json.loads(result.text):
+                        if sev == alarm['severity']:
+                            sev_list.append(alarm['alarm_id'])
+                    a_list = list(set(sev_list).intersection(res_list))
+                else:
+                    log.info("Returning an entire list of alarms.")
+                    a_list = res_list
+            else:
+                log.info("There are no alarms!")
+            response_list = []
+            for alarm in json.loads(result.text):
+                if alarm['alarm_id'] in a_list:
+                    response_list.append(alarm)
+            return response_list
+
+        except Exception as e:
+            log.exception("Failed to generate alarm list: ")
+            raise e
+
+    def update_alarm_state(self, endpoint, auth_token, alarm_id, verify_ssl):
+        """Set the state of an alarm to ok when ack message is received."""
+        url = "{}/v2/alarms/%s/state".format(endpoint) % alarm_id
+        payload = json.dumps("ok")
+
+        result = Common.perform_request(
+            url, auth_token, req_type="put", payload=payload, verify_ssl=verify_ssl)
+
+        return json.loads(result.text)
+
+    def update_alarm(self, endpoint, auth_token, values, vim_config, verify_ssl):
+        """Get alarm name for an alarm configuration update."""
+        # Get already existing alarm details
+        url = "{}/v2/alarms/%s".format(endpoint) % values['alarm_uuid']
+
+        # Gets current configurations about the alarm
+        result = Common.perform_request(
+            url, auth_token, req_type="get")
+        alarm_name = json.loads(result.text)['name']
+        rule = json.loads(result.text)['gnocchi_resources_threshold_rule']
+        alarm_state = json.loads(result.text)['state']
+        resource_id = rule['resource_id']
+        metric_name = [key for key, value in six.iteritems(METRIC_MAPPINGS) if value == rule['metric']][0]
+
+        # Generates and check payload configuration for alarm update
+        if 'granularity' in vim_config and 'granularity' not in values:
+            values['granularity'] = vim_config['granularity']
+        payload = self.check_payload(values, metric_name, resource_id,
+                                     alarm_name, alarm_state=alarm_state)
+
+        # Updates the alarm configurations with the valid payload
+        update_alarm = Common.perform_request(
+            url, auth_token, req_type="put", payload=payload, verify_ssl=verify_ssl)
+
+        return json.loads(update_alarm.text)['alarm_id']
+
+    def check_payload(self, values, metric_name, resource_id,
+                      alarm_name, alarm_state=None):
+        """Check that the payload is configuration for update/create alarm."""
+        cfg = Config.instance()
+        # Check state and severity
+
+        severity = 'critical'
+        if 'severity' in values:
+            severity = values['severity'].lower()
+
+        if severity == "indeterminate":
+            alarm_state = "insufficient data"
+        if alarm_state is None:
+            alarm_state = "ok"
+
+        statistic = values['statistic'].lower()
+
+        granularity = cfg.OS_DEFAULT_GRANULARITY
+        if 'granularity' in values:
+            granularity = values['granularity']
+
+        resource_type = 'generic'
+        if 'resource_type' in values:
+            resource_type = values['resource_type'].lower()
+
+        # Try to configure the payload for the update/create request
+        # Can only update: threshold, operation, statistic and
+        # the severity of the alarm
+        rule = {'threshold': values['threshold_value'],
+                'comparison_operator': values['operation'].lower(),
+                'metric': METRIC_MAPPINGS[metric_name],
+                'resource_id': resource_id,
+                'resource_type': resource_type,
+                'aggregation_method': STATISTICS[statistic],
+                'granularity': granularity, }
+        payload = json.dumps({'state': alarm_state,
+                              'name': alarm_name,
+                              'severity': SEVERITIES[severity],
+                              'type': 'gnocchi_resources_threshold',
+                              'gnocchi_resources_threshold_rule': rule,
+                              'alarm_actions': [cfg.OS_NOTIFIER_URI],
+                              'repeat_actions': True}, sort_keys=True)
+        return payload
+
+    def get_alarm_state(self, endpoint, auth_token, alarm_id):
+        """Get the state of the alarm."""
+        url = "{}/v2/alarms/%s/state".format(endpoint) % alarm_id
+
+        alarm_state = Common.perform_request(
+            url, auth_token, req_type="get")
+        return json.loads(alarm_state.text)
+
+    def check_for_metric(self, auth_token, metric_endpoint, metric_name, resource_id, verify_ssl):
+        """
+        Checks if resource has a specific metric. If not, throws exception.
+        :param verify_ssl: Boolean flag to set SSL cert validation
+        :param auth_token: OpenStack auth token
+        :param metric_endpoint: OpenStack metric endpoint
+        :param metric_name: Metric name
+        :param resource_id: Resource UUID
+        :return: Metric details from resource
+        :raise Exception: Could not retrieve metric from resource
+        """
+        try:
+            url = "{}/v1/resource/generic/{}".format(metric_endpoint, resource_id)
+            result = Common.perform_request(
+                url, auth_token, req_type="get", verify_ssl=verify_ssl)
+            resource = json.loads(result.text)
+            metrics_dict = resource['metrics']
+            return metrics_dict[METRIC_MAPPINGS[metric_name]]
+        except Exception as e:
+            log.exception("Desired Gnocchi metric not found:", e)
+            raise e
diff --git a/osm_mon/plugins/OpenStack/Aodh/alarming.py b/osm_mon/plugins/OpenStack/Aodh/alarming.py
deleted file mode 100644 (file)
index 453e77c..0000000
+++ /dev/null
@@ -1,404 +0,0 @@
-# Copyright 2017 Intel Research and Development Ireland Limited
-# *************************************************************
-
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: helena.mcgough@intel.com or adrian.hoban@intel.com
-##
-"""Carry out alarming requests via Aodh API."""
-
-import json
-import logging
-from json import JSONDecodeError
-
-import six
-import yaml
-
-from osm_mon.core.auth import AuthManager
-from osm_mon.core.database import DatabaseManager
-from osm_mon.core.message_bus.producer import KafkaProducer
-from osm_mon.core.settings import Config
-from osm_mon.plugins.OpenStack.Gnocchi.metrics import METRIC_MAPPINGS
-from osm_mon.plugins.OpenStack.common import Common
-from osm_mon.plugins.OpenStack.response import OpenStack_Response
-
-log = logging.getLogger(__name__)
-
-SEVERITIES = {
-    "warning": "low",
-    "minor": "low",
-    "major": "moderate",
-    "critical": "critical",
-    "indeterminate": "critical"}
-
-STATISTICS = {
-    "average": "mean",
-    "minimum": "min",
-    "maximum": "max",
-    "count": "count",
-    "sum": "sum"}
-
-
-class Alarming(object):
-    """Carries out alarming requests and responses via Aodh API."""
-
-    def __init__(self):
-        """Create the OpenStack alarming instance."""
-        self._database_manager = DatabaseManager()
-        self._auth_manager = AuthManager()
-
-        # Use the Response class to generate valid json response messages
-        self._response = OpenStack_Response()
-
-        # Initializer a producer to send responses back to SO
-        self._producer = KafkaProducer("alarm_response")
-
-    def alarming(self, message, vim_uuid):
-        """
-        Processes alarm request message depending on it's key
-        :param message: Message containing key and value attributes. This last one can be in JSON or YAML format.
-        :param vim_uuid: UUID of the VIM to handle the alarm request.
-        :return:
-        """
-        try:
-            values = json.loads(message.value)
-        except JSONDecodeError:
-            values = yaml.safe_load(message.value)
-
-        log.info("OpenStack alarm action required.")
-
-        verify_ssl = self._auth_manager.is_verify_ssl(vim_uuid)
-
-        auth_token = Common.get_auth_token(vim_uuid, verify_ssl=verify_ssl)
-
-        alarm_endpoint = Common.get_endpoint("alarming", vim_uuid, verify_ssl=verify_ssl)
-        metric_endpoint = Common.get_endpoint("metric", vim_uuid, verify_ssl=verify_ssl)
-
-        vim_account = self._auth_manager.get_credentials(vim_uuid)
-        vim_config = json.loads(vim_account.config)
-
-        if message.key == "create_alarm_request":
-            alarm_details = values['alarm_create_request']
-            alarm_id = None
-            status = False
-            try:
-                metric_name = alarm_details['metric_name'].lower()
-                resource_id = alarm_details['resource_uuid']
-
-                self.check_for_metric(auth_token, metric_endpoint, metric_name, resource_id, verify_ssl)
-
-                alarm_id = self.configure_alarm(
-                    alarm_endpoint, auth_token, alarm_details, vim_config, verify_ssl)
-
-                log.info("Alarm successfully created")
-                self._database_manager.save_alarm(alarm_id,
-                                                  vim_uuid,
-                                                  alarm_details['threshold_value'],
-                                                  alarm_details['operation'].lower(),
-                                                  alarm_details['metric_name'].lower(),
-                                                  alarm_details['vdu_name'].lower(),
-                                                  alarm_details['vnf_member_index'],
-                                                  alarm_details['ns_id'].lower()
-                                                  )
-                status = True
-            except Exception as e:
-                log.exception("Error creating alarm")
-                raise e
-            finally:
-                self._generate_and_send_response('create_alarm_response',
-                                                 alarm_details['correlation_id'],
-                                                 status=status,
-                                                 alarm_id=alarm_id)
-
-        elif message.key == "list_alarm_request":
-            list_details = values['alarm_list_request']
-            alarm_list = None
-            try:
-                alarm_list = self.list_alarms(
-                    alarm_endpoint, auth_token, list_details, verify_ssl)
-            except Exception as e:
-                log.exception("Error listing alarms")
-                raise e
-            finally:
-                self._generate_and_send_response('list_alarm_response',
-                                                 list_details['correlation_id'],
-                                                 alarm_list=alarm_list)
-
-        elif message.key == "delete_alarm_request":
-            request_details = values['alarm_delete_request']
-            alarm_id = request_details['alarm_uuid']
-            status = False
-            try:
-                self.delete_alarm(
-                    alarm_endpoint, auth_token, alarm_id, verify_ssl)
-                status = True
-            except Exception as e:
-                log.exception("Error deleting alarm")
-                raise e
-            finally:
-                self._generate_and_send_response('delete_alarm_response',
-                                                 request_details['correlation_id'],
-                                                 status=status,
-                                                 alarm_id=alarm_id)
-
-        elif message.key == "acknowledge_alarm":
-            try:
-                alarm_id = values['ack_details']['alarm_uuid']
-
-                self.update_alarm_state(
-                    alarm_endpoint, auth_token, alarm_id, verify_ssl)
-
-                log.info("Acknowledged the alarm and cleared it.")
-            except Exception as e:
-                log.exception("Error acknowledging alarm")
-                raise e
-
-        elif message.key == "update_alarm_request":
-            # Update alarm configurations
-            alarm_details = values['alarm_update_request']
-            alarm_id = None
-            status = False
-            try:
-                alarm_id = self.update_alarm(
-                    alarm_endpoint, auth_token, alarm_details, vim_config, verify_ssl)
-                status = True
-            except Exception as e:
-                log.exception("Error updating alarm")
-                raise e
-            finally:
-                self._generate_and_send_response('update_alarm_response',
-                                                 alarm_details['correlation_id'],
-                                                 status=status,
-                                                 alarm_id=alarm_id)
-
-        else:
-            log.debug("Unknown key, no action will be performed")
-
-    def configure_alarm(self, alarm_endpoint, auth_token, values, vim_config, verify_ssl):
-        """Create requested alarm in Aodh."""
-        url = "{}/v2/alarms/".format(alarm_endpoint)
-
-        # Check if the desired alarm is supported
-        alarm_name = values['alarm_name'].lower()
-        metric_name = values['metric_name'].lower()
-        resource_id = values['resource_uuid']
-
-        if metric_name not in METRIC_MAPPINGS.keys():
-            raise KeyError("Metric {} is not supported.".format(metric_name))
-
-        #FIXME
-        if 'granularity' in vim_config and 'granularity' not in values:
-            values['granularity'] = vim_config['granularity']
-        payload = self.check_payload(values, metric_name, resource_id,
-                                     alarm_name)
-        new_alarm = Common.perform_request(
-            url, auth_token, req_type="post", payload=payload, verify_ssl=verify_ssl)
-        return json.loads(new_alarm.text)['alarm_id']
-
-    def delete_alarm(self, endpoint, auth_token, alarm_id, verify_ssl):
-        """Delete alarm function."""
-        url = "{}/v2/alarms/%s".format(endpoint) % alarm_id
-
-        result = Common.perform_request(
-            url, auth_token, req_type="delete", verify_ssl=verify_ssl)
-        if str(result.status_code) == "404":
-            raise ValueError("Alarm {} doesn't exist".format(alarm_id))
-
-    def list_alarms(self, endpoint, auth_token, list_details, verify_ssl):
-        """Generate the requested list of alarms."""
-        url = "{}/v2/alarms/".format(endpoint)
-        a_list, name_list, sev_list, res_list = [], [], [], []
-
-        # TODO(mcgoughh): for now resource_id is a mandatory field
-        # Check for a resource id
-        try:
-            resource = list_details['resource_uuid']
-            name = list_details['alarm_name'].lower()
-            severity = list_details['severity'].lower()
-            sev = SEVERITIES[severity]
-        except KeyError as e:
-            log.warning("Missing parameter for alarm list request: %s", e)
-            raise e
-
-        # Perform the request to get the desired list
-        try:
-            result = Common.perform_request(
-                url, auth_token, req_type="get", verify_ssl=verify_ssl)
-
-            if result is not None:
-                # Get list based on resource id
-                for alarm in json.loads(result.text):
-                    rule = alarm['gnocchi_resources_threshold_rule']
-                    if resource == rule['resource_id']:
-                        res_list.append(alarm['alarm_id'])
-
-                # Generate specified listed if requested
-                if name is not None and sev is not None:
-                    log.info("Return a list of %s alarms with %s severity.",
-                             name, sev)
-                    for alarm in json.loads(result.text):
-                        if name == alarm['name']:
-                            name_list.append(alarm['alarm_id'])
-                    for alarm in json.loads(result.text):
-                        if sev == alarm['severity']:
-                            sev_list.append(alarm['alarm_id'])
-                    name_sev_list = list(set(name_list).intersection(sev_list))
-                    a_list = list(set(name_sev_list).intersection(res_list))
-                elif name is not None:
-                    log.info("Returning a %s list of alarms.", name)
-                    for alarm in json.loads(result.text):
-                        if name == alarm['name']:
-                            name_list.append(alarm['alarm_id'])
-                    a_list = list(set(name_list).intersection(res_list))
-                elif sev is not None:
-                    log.info("Returning %s severity alarm list.", sev)
-                    for alarm in json.loads(result.text):
-                        if sev == alarm['severity']:
-                            sev_list.append(alarm['alarm_id'])
-                    a_list = list(set(sev_list).intersection(res_list))
-                else:
-                    log.info("Returning an entire list of alarms.")
-                    a_list = res_list
-            else:
-                log.info("There are no alarms!")
-            response_list = []
-            for alarm in json.loads(result.text):
-                if alarm['alarm_id'] in a_list:
-                    response_list.append(alarm)
-            return response_list
-
-        except Exception as e:
-            log.exception("Failed to generate alarm list: ")
-            raise e
-
-    def update_alarm_state(self, endpoint, auth_token, alarm_id, verify_ssl):
-        """Set the state of an alarm to ok when ack message is received."""
-        url = "{}/v2/alarms/%s/state".format(endpoint) % alarm_id
-        payload = json.dumps("ok")
-
-        Common.perform_request(
-            url, auth_token, req_type="put", payload=payload, verify_ssl=verify_ssl)
-
-    def update_alarm(self, endpoint, auth_token, values, vim_config, verify_ssl):
-        """Get alarm name for an alarm configuration update."""
-        # Get already existing alarm details
-        url = "{}/v2/alarms/%s".format(endpoint) % values['alarm_uuid']
-
-        # Gets current configurations about the alarm
-        result = Common.perform_request(
-            url, auth_token, req_type="get")
-        alarm_name = json.loads(result.text)['name']
-        rule = json.loads(result.text)['gnocchi_resources_threshold_rule']
-        alarm_state = json.loads(result.text)['state']
-        resource_id = rule['resource_id']
-        metric_name = [key for key, value in six.iteritems(METRIC_MAPPINGS) if value == rule['metric']][0]
-
-        # Generates and check payload configuration for alarm update
-        if 'granularity' in vim_config and 'granularity' not in values:
-            values['granularity'] = vim_config['granularity']
-        payload = self.check_payload(values, metric_name, resource_id,
-                                     alarm_name, alarm_state=alarm_state)
-
-        # Updates the alarm configurations with the valid payload
-        update_alarm = Common.perform_request(
-            url, auth_token, req_type="put", payload=payload, verify_ssl=verify_ssl)
-
-        return json.loads(update_alarm.text)['alarm_id']
-
-    def check_payload(self, values, metric_name, resource_id,
-                      alarm_name, alarm_state=None):
-        """Check that the payload is configuration for update/create alarm."""
-        cfg = Config.instance()
-        # Check state and severity
-
-        severity = 'critical'
-        if 'severity' in values:
-            severity = values['severity'].lower()
-
-        if severity == "indeterminate":
-            alarm_state = "insufficient data"
-        if alarm_state is None:
-            alarm_state = "ok"
-
-        statistic = values['statistic'].lower()
-
-        granularity = cfg.OS_DEFAULT_GRANULARITY
-        if 'granularity' in values:
-            granularity = values['granularity']
-
-        resource_type = 'generic'
-        if 'resource_type' in values:
-            resource_type = values['resource_type'].lower()
-
-        # Try to configure the payload for the update/create request
-        # Can only update: threshold, operation, statistic and
-        # the severity of the alarm
-        rule = {'threshold': values['threshold_value'],
-                'comparison_operator': values['operation'].lower(),
-                'metric': METRIC_MAPPINGS[metric_name],
-                'resource_id': resource_id,
-                'resource_type': resource_type,
-                'aggregation_method': STATISTICS[statistic],
-                'granularity': granularity, }
-        payload = json.dumps({'state': alarm_state,
-                              'name': alarm_name,
-                              'severity': SEVERITIES[severity],
-                              'type': 'gnocchi_resources_threshold',
-                              'gnocchi_resources_threshold_rule': rule,
-                              'alarm_actions': [cfg.OS_NOTIFIER_URI],
-                              'repeat_actions': True}, sort_keys=True)
-        return payload
-
-    def get_alarm_state(self, endpoint, auth_token, alarm_id):
-        """Get the state of the alarm."""
-        url = "{}/v2/alarms/%s/state".format(endpoint) % alarm_id
-
-        alarm_state = Common.perform_request(
-            url, auth_token, req_type="get")
-        return json.loads(alarm_state.text)
-
-    def check_for_metric(self, auth_token, metric_endpoint, metric_name, resource_id, verify_ssl):
-        """
-        Checks if resource has a specific metric. If not, throws exception.
-        :param auth_token: OpenStack auth token
-        :param metric_endpoint: OpenStack metric endpoint
-        :param metric_name: Metric name
-        :param resource_id: Resource UUID
-        :return: Metric details from resource
-        :raise Exception: Could not retrieve metric from resource
-        """
-        try:
-            url = "{}/v1/resource/generic/{}".format(metric_endpoint, resource_id)
-            result = Common.perform_request(
-                url, auth_token, req_type="get", verify_ssl=verify_ssl)
-            resource = json.loads(result.text)
-            metrics_dict = resource['metrics']
-            return metrics_dict[METRIC_MAPPINGS[metric_name]]
-        except Exception as e:
-            log.exception("Desired Gnocchi metric not found:", e)
-            raise e
-
-    def _generate_and_send_response(self, key, correlation_id, **kwargs):
-        try:
-            resp_message = self._response.generate_response(
-                key, cor_id=correlation_id, **kwargs)
-            log.info("Response Message: %s", resp_message)
-            self._producer.publish_alarm_response(
-                key, resp_message)
-        except Exception as e:
-            log.exception("Response creation failed:")
-            raise e
index c74a73f..a1c85e6 100644 (file)
@@ -32,6 +32,7 @@ from six.moves.BaseHTTPServer import BaseHTTPRequestHandler
 from six.moves.BaseHTTPServer import HTTPServer
 
 # Initialise a logger for alarm notifier
+from osm_mon.core.message_bus.producer import Producer
 from osm_mon.core.settings import Config
 
 cfg = Config.instance()
@@ -52,9 +53,8 @@ kafka_logger.addHandler(kafka_handler)
 sys.path.append(os.path.abspath(os.path.join(os.path.realpath(__file__), '..', '..', '..', '..', '..')))
 
 from osm_mon.core.database import DatabaseManager
-from osm_mon.core.message_bus.producer import KafkaProducer
 
-from osm_mon.plugins.OpenStack.response import OpenStack_Response
+from osm_mon.plugins.OpenStack.response import OpenStackResponseBuilder
 
 
 class NotifierHandler(BaseHTTPRequestHandler):
@@ -94,8 +94,7 @@ class NotifierHandler(BaseHTTPRequestHandler):
         """Sends alarm notification message to bus."""
 
         # Initialise configuration and authentication for response message
-        response = OpenStack_Response()
-        producer = KafkaProducer('alarm_response')
+        response = OpenStackResponseBuilder()
 
         database_manager = DatabaseManager()
 
@@ -117,10 +116,14 @@ class NotifierHandler(BaseHTTPRequestHandler):
             sev=values['severity'],
             date=a_date,
             state=values['current'])
-        producer.publish_alarm_response(
-            'notify_alarm', resp_message)
+        self._publish_response('notify_alarm', json.dumps(resp_message))
         log.info("Sent alarm notification: %s", resp_message)
 
+    def _publish_response(self, key: str, msg: str):
+        producer = Producer()
+        producer.send(topic='alarm_response', key=key, value=msg)
+        producer.flush()
+
 
 def run(server_class=HTTPServer, handler_class=NotifierHandler, port=8662):
     """Run the webserver application to retrieve alarm notifications."""
diff --git a/osm_mon/plugins/OpenStack/Gnocchi/metric_handler.py b/osm_mon/plugins/OpenStack/Gnocchi/metric_handler.py
new file mode 100644 (file)
index 0000000..91dc402
--- /dev/null
@@ -0,0 +1,456 @@
+# Copyright 2017 Intel Research and Development Ireland Limited
+# *************************************************************
+
+# This file is part of OSM Monitoring module
+# All Rights Reserved to Intel Corporation
+
+# Licensed under the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License. You may obtain
+# a copy of the License at
+
+#         http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+# License for the specific language governing permissions and limitations
+# under the License.
+
+# For those usages not covered by the Apache License, Version 2.0 please
+# contact: helena.mcgough@intel.com or adrian.hoban@intel.com
+##
+"""Carry out OpenStack metric requests via Gnocchi API."""
+
+import datetime
+import json
+import logging
+import time
+
+import six
+
+from osm_mon.core.auth import AuthManager
+from osm_mon.core.settings import Config
+from osm_mon.plugins.OpenStack.common import Common
+from osm_mon.plugins.OpenStack.response import OpenStackResponseBuilder
+
+log = logging.getLogger(__name__)
+
+METRIC_MAPPINGS = {
+    "average_memory_utilization": "memory.usage",
+    "disk_read_ops": "disk.read.requests",
+    "disk_write_ops": "disk.write.requests",
+    "disk_read_bytes": "disk.read.bytes",
+    "disk_write_bytes": "disk.write.bytes",
+    "packets_dropped": "interface.if_dropped",
+    "packets_received": "interface.if_packets",
+    "packets_sent": "interface.if_packets",
+    "cpu_utilization": "cpu_util",
+}
+
+PERIOD_MS = {
+    "HR": 3600000,
+    "DAY": 86400000,
+    "WEEK": 604800000,
+    "MONTH": 2629746000,
+    "YEAR": 31556952000
+}
+
+
+class OpenstackMetricHandler(object):
+    """OpenStack metric requests performed via the Gnocchi API."""
+
+    def __init__(self):
+        """Initialize the metric actions."""
+        self._cfg = Config.instance()
+
+        # Use the Response class to generate valid json response messages
+        self._response = OpenStackResponseBuilder()
+
+        self._auth_manager = AuthManager()
+
+    def handle_request(self, key: str, values: dict, vim_uuid: str) -> dict:
+        """
+        Processes metric request message depending on it's key
+        :param key: Kafka message key
+        :param values: Dict containing metric request data. Follows models defined in core.models.
+        :param vim_uuid: UUID of the VIM to handle the metric request.
+        :return: Dict containing metric response data. Follows models defined in core.models.
+        """
+
+        log.info("OpenStack metric action required.")
+
+        if 'metric_name' in values and values['metric_name'] not in METRIC_MAPPINGS.keys():
+            raise ValueError('Metric ' + values['metric_name'] + ' is not supported.')
+
+        verify_ssl = self._auth_manager.is_verify_ssl(vim_uuid)
+
+        endpoint = Common.get_endpoint("metric", vim_uuid, verify_ssl=verify_ssl)
+
+        auth_token = Common.get_auth_token(vim_uuid, verify_ssl=verify_ssl)
+
+        if key == "create_metric_request":
+            metric_details = values['metric_create_request']
+            status = False
+            metric_id = None
+            resource_id = None
+            try:
+                # Configure metric
+                metric_id, resource_id = self.configure_metric(endpoint, auth_token, metric_details, verify_ssl)
+                log.info("Metric successfully created")
+                status = True
+            except Exception as e:
+                log.exception("Error creating metric")
+                raise e
+            finally:
+                return self._response.generate_response('create_metric_response',
+                                                        cor_id=metric_details['correlation_id'],
+                                                        status=status,
+                                                        metric_id=metric_id,
+                                                        resource_id=resource_id)
+
+        elif key == "read_metric_data_request":
+            metric_id = None
+            timestamps = []
+            metric_data = []
+            status = False
+            try:
+                metric_id = self.get_metric_id(endpoint,
+                                               auth_token,
+                                               METRIC_MAPPINGS[values['metric_name']],
+                                               values['resource_uuid'],
+                                               verify_ssl)
+                # Read all metric data related to a specified metric
+                timestamps, metric_data = self.read_metric_data(endpoint, auth_token, values, verify_ssl)
+                log.info("Metric data collected successfully")
+                status = True
+            except Exception as e:
+                log.exception("Error reading metric data")
+                raise e
+            finally:
+                return self._response.generate_response('read_metric_data_response',
+                                                        cor_id=values['correlation_id'],
+                                                        status=status,
+                                                        metric_id=metric_id,
+                                                        metric_name=values['metric_name'],
+                                                        resource_id=values['resource_uuid'],
+                                                        times=timestamps,
+                                                        metrics=metric_data)
+
+        elif key == "delete_metric_request":
+            metric_id = None
+            status = False
+            try:
+                # delete the specified metric in the request
+                metric_id = self.get_metric_id(endpoint, auth_token, METRIC_MAPPINGS[values['metric_name']],
+                                               values['resource_uuid'], verify_ssl)
+                self.delete_metric(
+                    endpoint, auth_token, metric_id, verify_ssl)
+                log.info("Metric deleted successfully")
+                status = True
+
+            except Exception as e:
+                log.exception("Error deleting metric")
+                raise e
+            finally:
+                return self._response.generate_response('delete_metric_response',
+                                                        cor_id=values['correlation_id'],
+                                                        metric_id=metric_id,
+                                                        metric_name=values['metric_name'],
+                                                        status=status,
+                                                        resource_id=values['resource_uuid'])
+
+        elif key == "update_metric_request":
+            # Gnocchi doesn't support configuration updates
+            # Log and send a response back to this effect
+            log.warning("Gnocchi doesn't support metric configuration updates.")
+            req_details = values['metric_update_request']
+            metric_name = req_details['metric_name']
+            resource_id = req_details['resource_uuid']
+            metric_id = self.get_metric_id(endpoint, auth_token, metric_name, resource_id, verify_ssl)
+            return self._response.generate_response('update_metric_response',
+                                                    cor_id=req_details['correlation_id'],
+                                                    status=False,
+                                                    resource_id=resource_id,
+                                                    metric_id=metric_id)
+
+        elif key == "list_metric_request":
+            list_details = values['metrics_list_request']
+            metric_list = []
+            status = False
+            try:
+                metric_list = self.list_metrics(
+                    endpoint, auth_token, list_details, verify_ssl)
+                log.info("Metrics listed successfully")
+                status = True
+            except Exception as e:
+                log.exception("Error listing metrics")
+                raise e
+            finally:
+                return self._response.generate_response('list_metric_response',
+                                                        cor_id=list_details['correlation_id'],
+                                                        status=status,
+                                                        metric_list=metric_list)
+
+        else:
+            raise ValueError("Unknown key {}, no action will be performed.".format(key))
+
+    def configure_metric(self, endpoint, auth_token, values, verify_ssl):
+        """Create the new metric in Gnocchi."""
+        required_fields = ['resource_uuid', 'metric_name']
+        for field in required_fields:
+            if field not in values:
+                raise ValueError("Missing field: " + field)
+
+        resource_id = values['resource_uuid']
+        metric_name = values['metric_name'].lower()
+
+        # Check for an existing metric for this resource
+        metric_id = self.get_metric_id(
+            endpoint, auth_token, metric_name, resource_id, verify_ssl)
+
+        if metric_id is None:
+            # Try appending metric to existing resource
+            try:
+                base_url = "{}/v1/resource/generic/%s/metric"
+                res_url = base_url.format(endpoint) % resource_id
+                payload = {metric_name: {'archive_policy_name': 'high',
+                                         'unit': values['metric_unit']}}
+                result = Common.perform_request(
+                    res_url,
+                    auth_token,
+                    req_type="post",
+                    verify_ssl=verify_ssl,
+                    payload=json.dumps(payload, sort_keys=True))
+                # Get id of newly created metric
+                for row in json.loads(result.text):
+                    if row['name'] == metric_name:
+                        metric_id = row['id']
+                log.info("Appended metric to existing resource.")
+
+                return metric_id, resource_id
+            except Exception as exc:
+                # Gnocchi version of resource does not exist creating a new one
+                log.info("Failed to append metric to existing resource:%s",
+                         exc)
+                url = "{}/v1/resource/generic".format(endpoint)
+                metric = {'name': metric_name,
+                          'archive_policy_name': 'high',
+                          'unit': values['metric_unit'], }
+
+                resource_payload = json.dumps({'id': resource_id,
+                                               'metrics': {
+                                                   metric_name: metric}}, sort_keys=True)
+
+                resource = Common.perform_request(
+                    url,
+                    auth_token,
+                    req_type="post",
+                    payload=resource_payload,
+                    verify_ssl=verify_ssl)
+
+                # Return the newly created resource_id for creating alarms
+                new_resource_id = json.loads(resource.text)['id']
+                log.info("Created new resource for metric: %s",
+                         new_resource_id)
+
+                metric_id = self.get_metric_id(
+                    endpoint, auth_token, metric_name, new_resource_id, verify_ssl)
+
+                return metric_id, new_resource_id
+
+        else:
+            raise ValueError("Metric already exists for this resource")
+
+    def delete_metric(self, endpoint, auth_token, metric_id, verify_ssl):
+        """Delete metric."""
+        url = "{}/v1/metric/%s".format(endpoint) % metric_id
+
+        result = Common.perform_request(
+            url,
+            auth_token,
+            req_type="delete",
+            verify_ssl=verify_ssl)
+        if not str(result.status_code).startswith("2"):
+            log.warning("Failed to delete the metric.")
+            raise ValueError("Error deleting metric. Aodh API responded with code " + str(result.status_code))
+
+    def list_metrics(self, endpoint, auth_token, values, verify_ssl):
+        """List all metrics."""
+
+        # Check for a specified list
+        metric_name = None
+        if 'metric_name' in values:
+            metric_name = values['metric_name'].lower()
+
+        resource = None
+        if 'resource_uuid' in values:
+            resource = values['resource_uuid']
+
+        if resource:
+            url = "{}/v1/resource/generic/{}".format(endpoint, resource)
+            result = Common.perform_request(
+                url, auth_token, req_type="get", verify_ssl=verify_ssl)
+            resource_data = json.loads(result.text)
+            metrics = resource_data['metrics']
+
+            if metric_name:
+                if metrics.get(METRIC_MAPPINGS[metric_name]):
+                    metric_id = metrics[METRIC_MAPPINGS[metric_name]]
+                    url = "{}/v1/metric/{}".format(endpoint, metric_id)
+                    result = Common.perform_request(
+                        url, auth_token, req_type="get", verify_ssl=verify_ssl)
+                    metric_list = json.loads(result.text)
+                    log.info("Returning an %s resource list for %s metrics",
+                             metric_name, resource)
+                    return metric_list
+                else:
+                    log.info("Metric {} not found for {} resource".format(metric_name, resource))
+                    return []
+            else:
+                metric_list = []
+                for k, v in metrics.items():
+                    url = "{}/v1/metric/{}".format(endpoint, v)
+                    result = Common.perform_request(
+                        url, auth_token, req_type="get", verify_ssl=verify_ssl)
+                    metric = json.loads(result.text)
+                    metric_list.append(metric)
+                if metric_list:
+                    log.info("Return a list of %s resource metrics", resource)
+                    return metric_list
+
+                else:
+                    log.info("There are no metrics available")
+                    return []
+        else:
+            url = "{}/v1/metric?sort=name:asc".format(endpoint)
+            result = Common.perform_request(
+                url, auth_token, req_type="get", verify_ssl=verify_ssl)
+            metrics = []
+            metrics_partial = json.loads(result.text)
+            for metric in metrics_partial:
+                metrics.append(metric)
+
+            while len(json.loads(result.text)) > 0:
+                last_metric_id = metrics_partial[-1]['id']
+                url = "{}/v1/metric?sort=name:asc&marker={}".format(endpoint, last_metric_id)
+                result = Common.perform_request(
+                    url, auth_token, req_type="get", verify_ssl=verify_ssl)
+                if len(json.loads(result.text)) > 0:
+                    metrics_partial = json.loads(result.text)
+                    for metric in metrics_partial:
+                        metrics.append(metric)
+
+            if metrics is not None:
+                # Format the list response
+                if metric_name is not None:
+                    metric_list = self.response_list(
+                        metrics, metric_name=metric_name)
+                    log.info("Returning a list of %s metrics", metric_name)
+                else:
+                    metric_list = self.response_list(metrics)
+                    log.info("Returning a complete list of metrics")
+                return metric_list
+            else:
+                log.info("There are no metrics available")
+                return []
+
+    def get_metric_id(self, endpoint, auth_token, metric_name, resource_id, verify_ssl):
+        """Check if the desired metric already exists for the resource."""
+        url = "{}/v1/resource/generic/%s".format(endpoint) % resource_id
+        try:
+            # Try return the metric id if it exists
+            result = Common.perform_request(
+                url,
+                auth_token,
+                req_type="get",
+                verify_ssl=verify_ssl)
+            return json.loads(result.text)['metrics'][metric_name]
+        except KeyError as e:
+            log.error("Metric doesn't exist. No metric_id available")
+            raise e
+
+    def read_metric_data(self, endpoint, auth_token, values, verify_ssl):
+        """Collect metric measures over a specified time period."""
+        timestamps = []
+        data = []
+        # get metric_id
+        metric_id = self.get_metric_id(endpoint, auth_token, METRIC_MAPPINGS[values['metric_name']],
+                                       values['resource_uuid'], verify_ssl)
+        # Try and collect measures
+        collection_unit = values['collection_unit'].upper()
+        collection_period = values['collection_period']
+
+        # Define the start and end time based on configurations
+        # FIXME: Local timezone may differ from timezone set in Gnocchi, causing discrepancies in measures
+        stop_time = time.strftime("%Y-%m-%d") + "T" + time.strftime("%X")
+        end_time = int(round(time.time() * 1000))
+        if collection_unit == 'YEAR':
+            diff = PERIOD_MS[collection_unit]
+        else:
+            diff = collection_period * PERIOD_MS[collection_unit]
+        s_time = (end_time - diff) / 1000.0
+        start_time = datetime.datetime.fromtimestamp(s_time).strftime(
+            '%Y-%m-%dT%H:%M:%S.%f')
+        base_url = "{}/v1/metric/%(0)s/measures?start=%(1)s&stop=%(2)s"
+        url = base_url.format(endpoint) % {
+            "0": metric_id, "1": start_time, "2": stop_time}
+
+        # Perform metric data request
+        metric_data = Common.perform_request(
+            url,
+            auth_token,
+            req_type="get",
+            verify_ssl=verify_ssl)
+
+        # Generate a list of the requested timestamps and data
+        for r in json.loads(metric_data.text):
+            timestamp = r[0].replace("T", " ")
+            timestamps.append(timestamp)
+            data.append(r[2])
+
+        return timestamps, data
+
+    def response_list(self, metric_list, metric_name=None, resource=None):
+        """Create the appropriate lists for a list response."""
+        resp_list, name_list, res_list = [], [], []
+
+        # Create required lists
+        for row in metric_list:
+            # Only list OSM metrics
+            name = None
+            if row['name'] in METRIC_MAPPINGS.values():
+                for k, v in six.iteritems(METRIC_MAPPINGS):
+                    if row['name'] == v:
+                        name = k
+                metric = {"metric_name": name,
+                          "metric_uuid": row['id'],
+                          "metric_unit": row['unit'],
+                          "resource_uuid": row['resource_id']}
+                resp_list.append(metric)
+            # Generate metric_name specific list
+            if metric_name is not None and name is not None:
+                if metric_name in METRIC_MAPPINGS.keys() and row['name'] == METRIC_MAPPINGS[metric_name]:
+                    metric = {"metric_name": metric_name,
+                              "metric_uuid": row['id'],
+                              "metric_unit": row['unit'],
+                              "resource_uuid": row['resource_id']}
+                    name_list.append(metric)
+            # Generate resource specific list
+            if resource is not None and name is not None:
+                if row['resource_id'] == resource:
+                    metric = {"metric_name": name,
+                              "metric_uuid": row['id'],
+                              "metric_unit": row['unit'],
+                              "resource_uuid": row['resource_id']}
+                    res_list.append(metric)
+
+        # Join required lists
+        if metric_name is not None and resource is not None:
+            # Return intersection of res_list and name_list
+            return [i for i in res_list for j in name_list if i['metric_uuid'] == j['metric_uuid']]
+        elif metric_name is not None:
+            return name_list
+        elif resource is not None:
+            return res_list
+        else:
+            return resp_list
diff --git a/osm_mon/plugins/OpenStack/Gnocchi/metrics.py b/osm_mon/plugins/OpenStack/Gnocchi/metrics.py
deleted file mode 100644 (file)
index 7bfbc47..0000000
+++ /dev/null
@@ -1,470 +0,0 @@
-# Copyright 2017 Intel Research and Development Ireland Limited
-# *************************************************************
-
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: helena.mcgough@intel.com or adrian.hoban@intel.com
-##
-"""Carry out OpenStack metric requests via Gnocchi API."""
-
-import datetime
-import json
-import logging
-import time
-from json import JSONDecodeError
-
-import six
-import yaml
-
-from osm_mon.core.auth import AuthManager
-from osm_mon.core.message_bus.producer import KafkaProducer
-from osm_mon.plugins.OpenStack.common import Common
-from osm_mon.plugins.OpenStack.response import OpenStack_Response
-
-log = logging.getLogger(__name__)
-
-METRIC_MAPPINGS = {
-    "average_memory_utilization": "memory.usage",
-    "disk_read_ops": "disk.read.requests",
-    "disk_write_ops": "disk.write.requests",
-    "disk_read_bytes": "disk.read.bytes",
-    "disk_write_bytes": "disk.write.bytes",
-    "packets_dropped": "interface.if_dropped",
-    "packets_received": "interface.if_packets",
-    "packets_sent": "interface.if_packets",
-    "cpu_utilization": "cpu_util",
-}
-
-PERIOD_MS = {
-    "HR": 3600000,
-    "DAY": 86400000,
-    "WEEK": 604800000,
-    "MONTH": 2629746000,
-    "YEAR": 31556952000
-}
-
-
-class Metrics(object):
-    """OpenStack metric requests performed via the Gnocchi API."""
-
-    def __init__(self):
-        """Initialize the metric actions."""
-
-        # Use the Response class to generate valid json response messages
-        self._response = OpenStack_Response()
-
-        # Initializer a producer to send responses back to SO
-        self._producer = KafkaProducer("metric_response")
-
-        self._auth_manager = AuthManager()
-
-    def metric_calls(self, message, vim_uuid):
-        """Consume info from the message bus to manage metric requests."""
-        log.info("OpenStack metric action required.")
-        try:
-            values = json.loads(message.value)
-        except JSONDecodeError:
-            values = yaml.safe_load(message.value)
-
-        log.info("OpenStack metric action required.")
-
-        if 'metric_name' in values and values['metric_name'] not in METRIC_MAPPINGS.keys():
-            raise ValueError('Metric ' + values['metric_name'] + ' is not supported.')
-
-        verify_ssl = self._auth_manager.is_verify_ssl(vim_uuid)
-
-        endpoint = Common.get_endpoint("metric", vim_uuid, verify_ssl=verify_ssl)
-
-        auth_token = Common.get_auth_token(vim_uuid, verify_ssl=verify_ssl)
-
-        if message.key == "create_metric_request":
-            metric_details = values['metric_create_request']
-            status = False
-            metric_id = None
-            resource_id = None
-            try:
-                # Configure metric
-                metric_id, resource_id = self.configure_metric(endpoint, auth_token, metric_details, verify_ssl)
-                log.info("Metric successfully created")
-                status = True
-            except Exception as e:
-                log.exception("Error creating metric")
-                raise e
-            finally:
-                self._generate_and_send_response('create_metric_response',
-                                                 metric_details['correlation_id'],
-                                                 status=status,
-                                                 metric_id=metric_id,
-                                                 resource_id=resource_id)
-
-        elif message.key == "read_metric_data_request":
-            metric_id = None
-            timestamps = []
-            metric_data = []
-            status = False
-            try:
-                metric_id = self.get_metric_id(endpoint,
-                                               auth_token,
-                                               METRIC_MAPPINGS[values['metric_name']],
-                                               values['resource_uuid'],
-                                               verify_ssl)
-                # Read all metric data related to a specified metric
-                timestamps, metric_data = self.read_metric_data(endpoint, auth_token, values, verify_ssl)
-                log.info("Metric data collected successfully")
-                status = True
-            except Exception as e:
-                log.exception("Error reading metric data")
-                raise e
-            finally:
-                self._generate_and_send_response('read_metric_data_response',
-                                                 values['correlation_id'],
-                                                 status=status,
-                                                 metric_id=metric_id,
-                                                 metric_name=values['metric_name'],
-                                                 resource_id=values['resource_uuid'],
-                                                 times=timestamps,
-                                                 metrics=metric_data)
-
-        elif message.key == "delete_metric_request":
-            metric_id = None
-            status = False
-            try:
-                # delete the specified metric in the request
-                metric_id = self.get_metric_id(endpoint, auth_token, METRIC_MAPPINGS[values['metric_name']],
-                                               values['resource_uuid'], verify_ssl)
-                self.delete_metric(
-                    endpoint, auth_token, metric_id, verify_ssl)
-                log.info("Metric deleted successfully")
-                status = True
-
-            except Exception as e:
-                log.exception("Error deleting metric")
-                raise e
-            finally:
-                self._generate_and_send_response('delete_metric_response',
-                                                 values['correlation_id'],
-                                                 metric_id=metric_id,
-                                                 metric_name=values['metric_name'],
-                                                 status=status,
-                                                 resource_id=values['resource_uuid'])
-
-        elif message.key == "update_metric_request":
-            # Gnocchi doesn't support configuration updates
-            # Log and send a response back to this effect
-            log.warning("Gnocchi doesn't support metric configuration updates.")
-            req_details = values['metric_update_request']
-            metric_name = req_details['metric_name']
-            resource_id = req_details['resource_uuid']
-            metric_id = self.get_metric_id(endpoint, auth_token, metric_name, resource_id, verify_ssl)
-            self._generate_and_send_response('update_metric_response',
-                                             req_details['correlation_id'],
-                                             status=False,
-                                             resource_id=resource_id,
-                                             metric_id=metric_id)
-
-        elif message.key == "list_metric_request":
-            list_details = values['metrics_list_request']
-            metric_list = []
-            status = False
-            try:
-                metric_list = self.list_metrics(
-                    endpoint, auth_token, list_details, verify_ssl)
-                log.info("Metrics listed successfully")
-                status = True
-            except Exception as e:
-                log.exception("Error listing metrics")
-                raise e
-            finally:
-                self._generate_and_send_response('list_metric_response',
-                                                 list_details['correlation_id'],
-                                                 status=status,
-                                                 metric_list=metric_list)
-
-        else:
-            log.warning("Unknown key %s, no action will be performed.", message.key)
-
-    def configure_metric(self, endpoint, auth_token, values, verify_ssl):
-        """Create the new metric in Gnocchi."""
-        required_fields = ['resource_uuid', 'metric_name']
-        for field in required_fields:
-            if field not in values:
-                raise ValueError("Missing field: " + field)
-
-        resource_id = values['resource_uuid']
-        metric_name = values['metric_name'].lower()
-
-        # Check for an existing metric for this resource
-        metric_id = self.get_metric_id(
-            endpoint, auth_token, metric_name, resource_id, verify_ssl)
-
-        if metric_id is None:
-            # Try appending metric to existing resource
-            try:
-                base_url = "{}/v1/resource/generic/%s/metric"
-                res_url = base_url.format(endpoint) % resource_id
-                payload = {metric_name: {'archive_policy_name': 'high',
-                                         'unit': values['metric_unit']}}
-                result = Common.perform_request(
-                    res_url,
-                    auth_token,
-                    req_type="post",
-                    verify_ssl=verify_ssl,
-                    payload=json.dumps(payload, sort_keys=True))
-                # Get id of newly created metric
-                for row in json.loads(result.text):
-                    if row['name'] == metric_name:
-                        metric_id = row['id']
-                log.info("Appended metric to existing resource.")
-
-                return metric_id, resource_id
-            except Exception as exc:
-                # Gnocchi version of resource does not exist creating a new one
-                log.info("Failed to append metric to existing resource:%s",
-                         exc)
-                url = "{}/v1/resource/generic".format(endpoint)
-                metric = {'name': metric_name,
-                          'archive_policy_name': 'high',
-                          'unit': values['metric_unit'], }
-
-                resource_payload = json.dumps({'id': resource_id,
-                                               'metrics': {
-                                                   metric_name: metric}}, sort_keys=True)
-
-                resource = Common.perform_request(
-                    url,
-                    auth_token,
-                    req_type="post",
-                    payload=resource_payload,
-                    verify_ssl=verify_ssl)
-
-                # Return the newly created resource_id for creating alarms
-                new_resource_id = json.loads(resource.text)['id']
-                log.info("Created new resource for metric: %s",
-                         new_resource_id)
-
-                metric_id = self.get_metric_id(
-                    endpoint, auth_token, metric_name, new_resource_id, verify_ssl)
-
-                return metric_id, new_resource_id
-
-        else:
-            raise ValueError("Metric already exists for this resource")
-
-    def delete_metric(self, endpoint, auth_token, metric_id, verify_ssl):
-        """Delete metric."""
-        url = "{}/v1/metric/%s".format(endpoint) % metric_id
-
-        result = Common.perform_request(
-            url,
-            auth_token,
-            req_type="delete",
-            verify_ssl=verify_ssl)
-        if not str(result.status_code).startswith("2"):
-            log.warning("Failed to delete the metric.")
-            raise ValueError("Error deleting metric. Aodh API responded with code " + str(result.status_code))
-
-    def list_metrics(self, endpoint, auth_token, values, verify_ssl):
-        """List all metrics."""
-
-        # Check for a specified list
-        metric_name = None
-        if 'metric_name' in values:
-            metric_name = values['metric_name'].lower()
-
-        resource = None
-        if 'resource_uuid' in values:
-            resource = values['resource_uuid']
-
-        if resource:
-            url = "{}/v1/resource/generic/{}".format(endpoint, resource)
-            result = Common.perform_request(
-                url, auth_token, req_type="get", verify_ssl=verify_ssl)
-            resource_data = json.loads(result.text)
-            metrics = resource_data['metrics']
-
-            if metric_name:
-                if metrics.get(METRIC_MAPPINGS[metric_name]):
-                    metric_id = metrics[METRIC_MAPPINGS[metric_name]]
-                    url = "{}/v1/metric/{}".format(endpoint, metric_id)
-                    result = Common.perform_request(
-                        url, auth_token, req_type="get", verify_ssl=verify_ssl)
-                    metric_list = json.loads(result.text)
-                    log.info("Returning an %s resource list for %s metrics",
-                             metric_name, resource)
-                    return metric_list
-                else:
-                    log.info("Metric {} not found for {} resource".format(metric_name, resource))
-                    return []
-            else:
-                metric_list = []
-                for k, v in metrics.items():
-                    url = "{}/v1/metric/{}".format(endpoint, v)
-                    result = Common.perform_request(
-                        url, auth_token, req_type="get", verify_ssl=verify_ssl)
-                    metric = json.loads(result.text)
-                    metric_list.append(metric)
-                if metric_list:
-                    log.info("Return a list of %s resource metrics", resource)
-                    return metric_list
-
-                else:
-                    log.info("There are no metrics available")
-                    return []
-        else:
-            url = "{}/v1/metric?sort=name:asc".format(endpoint)
-            result = Common.perform_request(
-                url, auth_token, req_type="get", verify_ssl=verify_ssl)
-            metrics = []
-            metrics_partial = json.loads(result.text)
-            for metric in metrics_partial:
-                metrics.append(metric)
-
-            while len(json.loads(result.text)) > 0:
-                last_metric_id = metrics_partial[-1]['id']
-                url = "{}/v1/metric?sort=name:asc&marker={}".format(endpoint, last_metric_id)
-                result = Common.perform_request(
-                    url, auth_token, req_type="get", verify_ssl=verify_ssl)
-                if len(json.loads(result.text)) > 0:
-                    metrics_partial = json.loads(result.text)
-                    for metric in metrics_partial:
-                        metrics.append(metric)
-
-            if metrics is not None:
-                # Format the list response
-                if metric_name is not None:
-                    metric_list = self.response_list(
-                        metrics, metric_name=metric_name)
-                    log.info("Returning a list of %s metrics", metric_name)
-                else:
-                    metric_list = self.response_list(metrics)
-                    log.info("Returning a complete list of metrics")
-                return metric_list
-            else:
-                log.info("There are no metrics available")
-                return []
-
-    def get_metric_id(self, endpoint, auth_token, metric_name, resource_id, verify_ssl):
-        """Check if the desired metric already exists for the resource."""
-        url = "{}/v1/resource/generic/%s".format(endpoint) % resource_id
-        try:
-            # Try return the metric id if it exists
-            result = Common.perform_request(
-                url,
-                auth_token,
-                req_type="get",
-                verify_ssl=verify_ssl)
-            return json.loads(result.text)['metrics'][metric_name]
-        except KeyError as e:
-            log.error("Metric doesn't exist. No metric_id available")
-            raise e
-
-    def read_metric_data(self, endpoint, auth_token, values, verify_ssl):
-        """Collect metric measures over a specified time period."""
-        timestamps = []
-        data = []
-        # get metric_id
-        metric_id = self.get_metric_id(endpoint, auth_token, METRIC_MAPPINGS[values['metric_name']],
-                                       values['resource_uuid'], verify_ssl)
-        # Try and collect measures
-        collection_unit = values['collection_unit'].upper()
-        collection_period = values['collection_period']
-
-        # Define the start and end time based on configurations
-        # FIXME: Local timezone may differ from timezone set in Gnocchi, causing discrepancies in measures
-        stop_time = time.strftime("%Y-%m-%d") + "T" + time.strftime("%X")
-        end_time = int(round(time.time() * 1000))
-        if collection_unit == 'YEAR':
-            diff = PERIOD_MS[collection_unit]
-        else:
-            diff = collection_period * PERIOD_MS[collection_unit]
-        s_time = (end_time - diff) / 1000.0
-        start_time = datetime.datetime.fromtimestamp(s_time).strftime(
-            '%Y-%m-%dT%H:%M:%S.%f')
-        base_url = "{}/v1/metric/%(0)s/measures?start=%(1)s&stop=%(2)s"
-        url = base_url.format(endpoint) % {
-            "0": metric_id, "1": start_time, "2": stop_time}
-
-        # Perform metric data request
-        metric_data = Common.perform_request(
-            url,
-            auth_token,
-            req_type="get",
-            verify_ssl=verify_ssl)
-
-        # Generate a list of the requested timestamps and data
-        for r in json.loads(metric_data.text):
-            timestamp = r[0].replace("T", " ")
-            timestamps.append(timestamp)
-            data.append(r[2])
-
-        return timestamps, data
-
-    def response_list(self, metric_list, metric_name=None, resource=None):
-        """Create the appropriate lists for a list response."""
-        resp_list, name_list, res_list = [], [], []
-
-        # Create required lists
-        for row in metric_list:
-            # Only list OSM metrics
-            name = None
-            if row['name'] in METRIC_MAPPINGS.values():
-                for k, v in six.iteritems(METRIC_MAPPINGS):
-                    if row['name'] == v:
-                        name = k
-                metric = {"metric_name": name,
-                          "metric_uuid": row['id'],
-                          "metric_unit": row['unit'],
-                          "resource_uuid": row['resource_id']}
-                resp_list.append(metric)
-            # Generate metric_name specific list
-            if metric_name is not None and name is not None:
-                if metric_name in METRIC_MAPPINGS.keys() and row['name'] == METRIC_MAPPINGS[metric_name]:
-                    metric = {"metric_name": metric_name,
-                              "metric_uuid": row['id'],
-                              "metric_unit": row['unit'],
-                              "resource_uuid": row['resource_id']}
-                    name_list.append(metric)
-            # Generate resource specific list
-            if resource is not None and name is not None:
-                if row['resource_id'] == resource:
-                    metric = {"metric_name": name,
-                              "metric_uuid": row['id'],
-                              "metric_unit": row['unit'],
-                              "resource_uuid": row['resource_id']}
-                    res_list.append(metric)
-
-        # Join required lists
-        if metric_name is not None and resource is not None:
-            # Return intersection of res_list and name_list
-            return [i for i in res_list for j in name_list if i['metric_uuid'] == j['metric_uuid']]
-        elif metric_name is not None:
-            return name_list
-        elif resource is not None:
-            return res_list
-        else:
-            return resp_list
-
-    def _generate_and_send_response(self, key, correlation_id, **kwargs):
-        try:
-            resp_message = self._response.generate_response(
-                key, cor_id=correlation_id, **kwargs)
-            log.info("Response Message: %s", resp_message)
-            self._producer.publish_metrics_response(
-                key, resp_message)
-        except Exception as e:
-            log.exception("Response creation failed:")
-            raise e
index 5f1529a..9f5b17d 100644 (file)
@@ -21,7 +21,6 @@
 ##
 """Generate valid responses to send back to the SO."""
 
-import json
 import logging
 
 log = logging.getLogger(__name__)
@@ -29,13 +28,13 @@ log = logging.getLogger(__name__)
 schema_version = "1.0"
 
 
-class OpenStack_Response(object):
+class OpenStackResponseBuilder(object):
     """Generates responses for OpenStack plugin."""
 
     def __init__(self):
         """Initialize OpenStack Response instance."""
 
-    def generate_response(self, key, **kwargs):
+    def generate_response(self, key, **kwargs) -> dict:
         """Make call to appropriate response function."""
         if key == "list_alarm_response":
             message = self.alarm_list_response(**kwargs)
@@ -63,15 +62,15 @@ class OpenStack_Response(object):
 
         return message
 
-    def alarm_list_response(self, **kwargs):
+    def alarm_list_response(self, **kwargs) -> dict:
         """Generate the response for an alarm list request."""
         alarm_list_resp = {"schema_version": schema_version,
                            "schema_type": "list_alarm_response",
                            "correlation_id": kwargs['cor_id'],
                            "list_alarm_response": kwargs['alarm_list']}
-        return json.dumps(alarm_list_resp)
+        return alarm_list_resp
 
-    def create_alarm_response(self, **kwargs):
+    def create_alarm_response(self, **kwargs) -> dict:
         """Generate a response for a create alarm request."""
         create_alarm_resp = {"schema_version": schema_version,
                              "schema_type": "create_alarm_response",
@@ -79,9 +78,9 @@ class OpenStack_Response(object):
                                  "correlation_id": kwargs['cor_id'],
                                  "alarm_uuid": kwargs['alarm_id'],
                                  "status": kwargs['status']}}
-        return json.dumps(create_alarm_resp)
+        return create_alarm_resp
 
-    def delete_alarm_response(self, **kwargs):
+    def delete_alarm_response(self, **kwargs) -> dict:
         """Generate a response for a delete alarm request."""
         delete_alarm_resp = {"schema_version": schema_version,
                              "schema_type": "alarm_deletion_response",
@@ -89,9 +88,9 @@ class OpenStack_Response(object):
                                  "correlation_id": kwargs['cor_id'],
                                  "alarm_uuid": kwargs['alarm_id'],
                                  "status": kwargs['status']}}
-        return json.dumps(delete_alarm_resp)
+        return delete_alarm_resp
 
-    def update_alarm_response(self, **kwargs):
+    def update_alarm_response(self, **kwargs) -> dict:
         """Generate a response for an update alarm request."""
         update_alarm_resp = {"schema_version": schema_version,
                              "schema_type": "update_alarm_response",
@@ -99,9 +98,9 @@ class OpenStack_Response(object):
                                  "correlation_id": kwargs['cor_id'],
                                  "alarm_uuid": kwargs['alarm_id'],
                                  "status": kwargs['status']}}
-        return json.dumps(update_alarm_resp)
+        return update_alarm_resp
 
-    def metric_create_response(self, **kwargs):
+    def metric_create_response(self, **kwargs) -> dict:
         """Generate a response for a create metric request."""
         create_metric_resp = {"schema_version": schema_version,
                               "schema_type": "create_metric_response",
@@ -110,9 +109,9 @@ class OpenStack_Response(object):
                                   "metric_uuid": kwargs['metric_id'],
                                   "resource_uuid": kwargs['resource_id'],
                                   "status": kwargs['status']}}
-        return json.dumps(create_metric_resp)
+        return create_metric_resp
 
-    def read_metric_data_response(self, **kwargs):
+    def read_metric_data_response(self, **kwargs) -> dict:
         """Generate a response for a read metric data request."""
         read_metric_data_resp = {"schema_version": schema_version,
                                  "schema_type": "read_metric_data_response",
@@ -124,9 +123,9 @@ class OpenStack_Response(object):
                                  "metrics_data": {
                                      "time_series": kwargs['times'],
                                      "metrics_series": kwargs['metrics']}}
-        return json.dumps(read_metric_data_resp)
+        return read_metric_data_resp
 
-    def delete_metric_response(self, **kwargs):
+    def delete_metric_response(self, **kwargs) -> dict:
         """Generate a response for a delete metric request."""
         delete_metric_resp = {"schema_version": schema_version,
                               "schema_type": "delete_metric_response",
@@ -135,9 +134,9 @@ class OpenStack_Response(object):
                               "resource_uuid": kwargs['resource_id'],
                               "correlation_id": kwargs['cor_id'],
                               "status": kwargs['status']}
-        return json.dumps(delete_metric_resp)
+        return delete_metric_resp
 
-    def update_metric_response(self, **kwargs):
+    def update_metric_response(self, **kwargs) -> dict:
         """Generate a repsonse for an update metric request."""
         update_metric_resp = {"schema_version": schema_version,
                               "schema_type": "update_metric_response",
@@ -146,18 +145,18 @@ class OpenStack_Response(object):
                                   "metric_uuid": kwargs['metric_id'],
                                   "status": kwargs['status'],
                                   "resource_uuid": kwargs['resource_id']}}
-        return json.dumps(update_metric_resp)
+        return update_metric_resp
 
-    def list_metric_response(self, **kwargs):
+    def list_metric_response(self, **kwargs) -> dict:
         """Generate a response for a list metric request."""
         list_metric_resp = {"schema_version": schema_version,
                             "schema_type": "list_metric_response",
                             "correlation_id": kwargs['cor_id'],
                             "status": kwargs['status'],
                             "metrics_list": kwargs['metric_list']}
-        return json.dumps(list_metric_resp)
+        return list_metric_resp
 
-    def notify_alarm(self, **kwargs):
+    def notify_alarm(self, **kwargs) -> dict:
         """Generate a response to send alarm notifications."""
         notify_alarm_resp = {"schema_version": schema_version,
                              "schema_type": "notify_alarm",
@@ -172,4 +171,4 @@ class OpenStack_Response(object):
                                  "severity": kwargs['sev'],
                                  "status": kwargs['state'],
                                  "start_date": kwargs['date']}}
-        return json.dumps(notify_alarm_resp)
+        return notify_alarm_resp
diff --git a/osm_mon/plugins/vRealiseOps/kafka_consumer_vrops.py b/osm_mon/plugins/vRealiseOps/kafka_consumer_vrops.py
deleted file mode 100644 (file)
index f5e11e6..0000000
+++ /dev/null
@@ -1,64 +0,0 @@
-# -*- coding: utf-8 -*-
-
-##
-# Copyright 2016-2017 VMware Inc.
-# This file is part of ETSI OSM
-# All Rights Reserved.
-#
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-#
-#         http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-#
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact:  osslegalrouting@vmware.com
-##
-
-"""
-vROPs Kafka Consumer that consumes the request messages
-"""
-
-
-from kafka import KafkaConsumer
-from kafka.errors import KafkaError
-import logging as log
-
-class vROP_KafkaConsumer(object):
-    """
-        Kafka Consumer for vROPs
-    """
-
-    def __init__(self, topics=[], broker_uri=None):
-        """
-            Method to initize KafkaConsumer
-            Args:
-                broker_uri - hostname:port uri of Kafka broker
-                topics - list of topics to subscribe
-            Returns:
-               None
-        """
-
-        if broker_uri is None:
-            self.broker = '0.0.0.0:9092'
-        else:
-            self.broker = broker_uri
-
-        self.topic = topics
-        print ("vROPs Consumer started, Broker URI: {}".format(self.broker))
-        print ("Subscribed Topics {}".format(self.topic))
-        try:
-            self.vrops_consumer = KafkaConsumer(bootstrap_servers=self.broker)
-            self.vrops_consumer.subscribe(self.topic)
-        except Exception as exp:
-            msg = "fail to create consumer for topic {} with broker {} Error : {}"\
-                    .format(self.topic, self.broker, exp)
-            log.error(msg)
-            raise Exception(msg)
-
index 6ca3d40..af68dfc 100644 (file)
 """
 Monitoring metrics & creating Alarm definitions in vROPs
 """
-import pytz
-import requests
 import logging
 
+import pytz
+import requests
 import six
 from pyvcloud.vcd.client import BasicLoginCredentials
 from pyvcloud.vcd.client import Client
-API_VERSION = '5.9'
 
 from xml.etree import ElementTree as XmlElementTree
 import traceback
@@ -44,31 +43,45 @@ import datetime
 from socket import getfqdn
 
 import urllib3
-urllib3.disable_warnings(urllib3.exceptions.InsecureRequestWarning)
+
+from osm_mon.core.settings import Config
 
 sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', '..', '..'))
 from osm_mon.core.database import DatabaseManager
 
-OPERATION_MAPPING = {'GE':'GT_EQ', 'LE':'LT_EQ', 'GT':'GT', 'LT':'LT', 'EQ':'EQ'}
-severity_mano2vrops = {'WARNING':'WARNING', 'MINOR':'WARNING', 'MAJOR':"IMMEDIATE",\
-                        'CRITICAL':'CRITICAL', 'INDETERMINATE':'UNKNOWN'}
-PERIOD_MSEC = {'HR':3600000,'DAY':86400000,'WEEK':604800000,'MONTH':2678400000,'YEAR':31536000000}
+API_VERSION = '5.9'
 
-#To Do - Add actual webhook url & certificate
-#SSL_CERTIFICATE_FILE_NAME = 'vROPs_Webservice/SSL_certificate/www.vrops_webservice.com.cert'
-#webhook_url = "https://mano-dev-1:8080/notify/" #for testing
+urllib3.disable_warnings(urllib3.exceptions.InsecureRequestWarning)
+OPERATION_MAPPING = {'GE': 'GT_EQ', 'LE': 'LT_EQ', 'GT': 'GT', 'LT': 'LT', 'EQ': 'EQ'}
+severity_mano2vrops = {'WARNING': 'WARNING', 'MINOR': 'WARNING', 'MAJOR': "IMMEDIATE",
+                       'CRITICAL': 'CRITICAL', 'INDETERMINATE': 'UNKNOWN'}
+PERIOD_MSEC = {'HR': 3600000, 'DAY': 86400000, 'WEEK': 604800000, 'MONTH': 2678400000, 'YEAR': 31536000000}
+
+# To Do - Add actual webhook url & certificate
+# SSL_CERTIFICATE_FILE_NAME = 'vROPs_Webservice/SSL_certificate/www.vrops_webservice.com.cert'
+# webhook_url = "https://mano-dev-1:8080/notify/" #for testing
 webhook_url = "https://" + getfqdn() + ":8080/notify/"
 SSL_CERTIFICATE_FILE_NAME = ('vROPs_Webservice/SSL_certificate/' + getfqdn() + ".cert")
-#SSL_CERTIFICATE_FILE_NAME = 'vROPs_Webservice/SSL_certificate/10.172.137.214.cert' #for testing
+# SSL_CERTIFICATE_FILE_NAME = 'vROPs_Webservice/SSL_certificate/10.172.137.214.cert' #for testing
 
 MODULE_DIR = os.path.dirname(__file__)
 CONFIG_FILE_NAME = 'vrops_config.xml'
 CONFIG_FILE_PATH = os.path.join(MODULE_DIR, CONFIG_FILE_NAME)
 SSL_CERTIFICATE_FILE_PATH = os.path.join(MODULE_DIR, SSL_CERTIFICATE_FILE_NAME)
 
-class MonPlugin():
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
+
+logger = logging.getLogger(__name__)
+
+
+class MonPlugin:
     """MON Plugin class for vROPs telemetry plugin
     """
+
     def __init__(self, access_config=None):
         """Constructor of MON plugin
         Params:
@@ -87,37 +100,32 @@ class MonPlugin():
             check against the VIM
         """
 
-        self.logger = logging.getLogger('PluginReceiver.MonPlugin')
-        self.logger.setLevel(logging.DEBUG)
-
         if access_config is None:
-            self.logger.error("VIM Access Configuration not provided")
+            logger.error("VIM Access Configuration not provided")
             raise KeyError("VIM Access Configuration not provided")
 
         self.database_manager = DatabaseManager()
 
         self.access_config = access_config
         if not bool(access_config):
-            self.logger.error("VIM Account details are not added. Please add a VIM account")
+            logger.error("VIM Account details are not added. Please add a VIM account")
             raise KeyError("VIM Account details are not added. Please add a VIM account")
 
         try:
-            self.vrops_site =  access_config['vrops_site']
+            self.vrops_site = access_config['vrops_site']
             self.vrops_user = access_config['vrops_user']
             self.vrops_password = access_config['vrops_password']
             self.vcloud_site = access_config['vim_url']
             self.admin_username = access_config['admin_username']
             self.admin_password = access_config['admin_password']
-            #self.tenant_id = access_config['tenant_id']
+            # self.tenant_id = access_config['tenant_id']
             self.vim_uuid = access_config['vim_uuid']
 
         except KeyError as exp:
-            self.logger.error("Required VIM account details not provided: {}".format(exp))
+            logger.error("Required VIM account details not provided: {}".format(exp))
             raise KeyError("Required VIM account details not provided: {}".format(exp))
 
-
-
-    def configure_alarm(self, config_dict = {}):
+    def configure_alarm(self, config_dict={}):
         """Configures or creates a new alarm using the input parameters in config_dict
         Params:
         "alarm_name": Alarm name in string format
@@ -138,95 +146,93 @@ class MonPlugin():
 
         Returns the UUID of created alarm or None
         """
-        alarm_def = None
-        #1) get alarm & metrics parameters from plugin specific file
+        # 1) get alarm & metrics parameters from plugin specific file
         def_a_params = self.get_default_Params(config_dict['alarm_name'])
         if not def_a_params:
-            self.logger.warning("Alarm not supported: {}".format(config_dict['alarm_name']))
+            logger.warning("Alarm not supported: {}".format(config_dict['alarm_name']))
             return None
         metric_key_params = self.get_default_Params(config_dict['metric_name'])
         if not metric_key_params:
-            self.logger.warning("Metric not supported: {}".format(config_dict['metric_name']))
+            logger.warning("Metric not supported: {}".format(config_dict['metric_name']))
             return None
 
-        #1.2) Check if alarm definition already exists
-        vrops_alarm_name = def_a_params['vrops_alarm']+ '-' + config_dict['resource_uuid']
+        # 1.2) Check if alarm definition already exists
+        vrops_alarm_name = def_a_params['vrops_alarm'] + '-' + config_dict['resource_uuid']
         alert_def_list = self.get_alarm_defination_by_name(vrops_alarm_name)
         if alert_def_list:
-            self.logger.warning("Alarm already exists: {}. Try updating by update_alarm_request"\
-                            .format(vrops_alarm_name))
+            logger.warning("Alarm already exists: {}. Try updating by update_alarm_request"
+                           .format(vrops_alarm_name))
             return None
 
-        #2) create symptom definition
-        symptom_params ={'cancel_cycles': (def_a_params['cancel_period']/300)*def_a_params['cancel_cycles'],
-                        'wait_cycles': (def_a_params['period']/300)*def_a_params['evaluation'],
-                        'resource_kind_key': def_a_params['resource_kind'],
-                        'adapter_kind_key': def_a_params['adapter_kind'],
-                        'symptom_name':vrops_alarm_name,
-                        'severity': severity_mano2vrops[config_dict['severity'].upper()],
-                        'metric_key':metric_key_params['metric_key'],
-                        'operation':OPERATION_MAPPING[config_dict['operation']],
-                        'threshold_value':config_dict['threshold_value']}
+        # 2) create symptom definition
+        symptom_params = {'cancel_cycles': (def_a_params['cancel_period'] / 300) * def_a_params['cancel_cycles'],
+                          'wait_cycles': (def_a_params['period'] / 300) * def_a_params['evaluation'],
+                          'resource_kind_key': def_a_params['resource_kind'],
+                          'adapter_kind_key': def_a_params['adapter_kind'],
+                          'symptom_name': vrops_alarm_name,
+                          'severity': severity_mano2vrops[config_dict['severity'].upper()],
+                          'metric_key': metric_key_params['metric_key'],
+                          'operation': OPERATION_MAPPING[config_dict['operation']],
+                          'threshold_value': config_dict['threshold_value']}
 
         symptom_uuid = self.create_symptom(symptom_params)
         if symptom_uuid is not None:
-            self.logger.info("Symptom defined: {} with ID: {}".format(symptom_params['symptom_name'],symptom_uuid))
+            logger.info("Symptom defined: {} with ID: {}".format(symptom_params['symptom_name'], symptom_uuid))
         else:
-            self.logger.warning("Failed to create Symptom: {}".format(symptom_params['symptom_name']))
+            logger.warning("Failed to create Symptom: {}".format(symptom_params['symptom_name']))
             return None
-        #3) create alert definition
-        #To Do - Get type & subtypes for all 5 alarms
-        alarm_params = {'name':vrops_alarm_name,
-                        'description':config_dict['description']\
-                        if 'description' in config_dict and config_dict['description'] is not None else config_dict['alarm_name'],
-                        'adapterKindKey':def_a_params['adapter_kind'],
-                        'resourceKindKey':def_a_params['resource_kind'],
-                        'waitCycles':1, 'cancelCycles':1,
-                        'type':def_a_params['alarm_type'], 'subType':def_a_params['alarm_subType'],
-                        'severity':severity_mano2vrops[config_dict['severity'].upper()],
-                        'symptomDefinitionId':symptom_uuid,
-                        'impact':def_a_params['impact']}
+        # 3) create alert definition
+        # To Do - Get type & subtypes for all 5 alarms
+        alarm_params = {'name': vrops_alarm_name,
+                        'description': config_dict['description']
+                        if 'description' in config_dict and config_dict['description'] is not None else config_dict[
+                            'alarm_name'],
+                        'adapterKindKey': def_a_params['adapter_kind'],
+                        'resourceKindKey': def_a_params['resource_kind'],
+                        'waitCycles': 1, 'cancelCycles': 1,
+                        'type': def_a_params['alarm_type'], 'subType': def_a_params['alarm_subType'],
+                        'severity': severity_mano2vrops[config_dict['severity'].upper()],
+                        'symptomDefinitionId': symptom_uuid,
+                        'impact': def_a_params['impact']}
 
         alarm_def = self.create_alarm_definition(alarm_params)
         if alarm_def is None:
-            self.logger.warning("Failed to create Alert: {}".format(alarm_params['name']))
+            logger.warning("Failed to create Alert: {}".format(alarm_params['name']))
             return None
 
-        self.logger.info("Alarm defined: {} with ID: {}".format(alarm_params['name'],alarm_def))
+        logger.info("Alarm defined: {} with ID: {}".format(alarm_params['name'], alarm_def))
 
-        #4) Find vm_moref_id from vApp uuid in vCD
+        # 4) Find vm_moref_id from vApp uuid in vCD
         vm_moref_id = self.get_vm_moref_id(config_dict['resource_uuid'])
         if vm_moref_id is None:
-            self.logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(config_dict['resource_uuid']))
+            logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(config_dict['resource_uuid']))
             return None
 
-        #5) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
+        # 5) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
         resource_id = self.get_vm_resource_id(vm_moref_id)
         if resource_id is None:
-            self.logger.warning("Failed to find resource in vROPs: {}".format(config_dict['resource_uuid']))
+            logger.warning("Failed to find resource in vROPs: {}".format(config_dict['resource_uuid']))
             return None
 
-        #6) Configure alarm notification for a particular VM using it's resource_id
+        # 6) Configure alarm notification for a particular VM using it's resource_id
         notification_id = self.create_alarm_notification_rule(vrops_alarm_name, alarm_def, resource_id)
         if notification_id is None:
             return None
         else:
             alarm_def_uuid = alarm_def.split('-', 1)[1]
-            self.logger.info("Alarm definition created with notification: {} with ID: {}"\
-                    .format(alarm_params['name'],alarm_def_uuid))
-            ##self.database_manager.save_alarm(alarm_def_uuid, alarm_params['name'], self.vim_uuid)
-            self.database_manager.save_alarm(alarm_def_uuid,
-                                              self.vim_uuid,
-                                              ##alarm_params['name'],
-                                              config_dict['threshold_value'],
-                                              config_dict['operation'],
-                                              config_dict['metric_name'].lower(),
-                                              config_dict['vdu_name'].lower(),
-                                              config_dict['vnf_member_index'].lower(),
-                                              config_dict['ns_id'].lower()
-                                              )
-
-            #Return alarm definition UUID by removing 'AlertDefinition' from UUID
+            logger.info("Alarm definition created with notification: {} with ID: {}"
+                        .format(alarm_params['name'], alarm_def_uuid))
+            self.database_manager.save_alarm(alarm_params['name'],
+                                             self.vim_uuid,
+                                             config_dict['threshold_value'],
+                                             config_dict['operation'],
+                                             config_dict['metric_name'].lower(),
+                                             config_dict['vdu_name'].lower(),
+                                             config_dict['vnf_member_index'].lower(),
+                                             config_dict['ns_id'].lower()
+                                             )
+
+            # Return alarm definition UUID by removing 'AlertDefinition' from UUID
             return (alarm_def_uuid)
 
     def get_default_Params(self, metric_alarm_name):
@@ -239,9 +245,9 @@ class MonPlugin():
         try:
             source = open(CONFIG_FILE_PATH, 'r')
         except IOError as exp:
-            msg = ("Could not read Config file: {}, \nException: {}"\
-                        .format(CONFIG_FILE_PATH, exp))
-            self.logger.error(msg)
+            msg = ("Could not read Config file: {},  nException: {}"
+                   .format(CONFIG_FILE_PATH, exp))
+            logger.error(msg)
             raise IOError(msg)
 
         tree = XmlElementTree.parse(source)
@@ -249,11 +255,11 @@ class MonPlugin():
         for alarm in alarms:
             if alarm.tag.lower() == metric_alarm_name.lower():
                 for param in alarm:
-                    if param.tag in ("period", "evaluation", "cancel_period", "alarm_type",\
-                                    "cancel_cycles", "alarm_subType"):
+                    if param.tag in ("period", "evaluation", "cancel_period", "alarm_type",
+                                     "cancel_cycles", "alarm_subType"):
                         a_params[param.tag] = int(param.text)
                     elif param.tag in ("enabled", "repeat"):
-                        if(param.text.lower() == "true"):
+                        if param.text.lower() == "true":
                             a_params[param.tag] = True
                         else:
                             a_params[param.tag] = False
@@ -262,7 +268,6 @@ class MonPlugin():
         source.close()
         return a_params
 
-
     def create_symptom(self, symptom_params):
         """Create Symptom definition for an alarm
         Params:
@@ -282,37 +287,37 @@ class MonPlugin():
 
         try:
             api_url = '/suite-api/api/symptomdefinitions'
-            headers = {'Content-Type': 'application/json','Accept': 'application/json'}
+            headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
             data = {
-                        "id": None,
-                        "name": symptom_params['symptom_name'],
-                        "adapterKindKey": symptom_params['adapter_kind_key'],
-                        "resourceKindKey": symptom_params['resource_kind_key'],
-                        "waitCycles": symptom_params['wait_cycles'],
-                        "cancelCycles": symptom_params['cancel_cycles'],
-                        "state": {
-                            "severity": symptom_params['severity'],
-                            "condition": {
-                                "type": "CONDITION_HT",
-                                "key": symptom_params['metric_key'],
-                                "operator": symptom_params['operation'],
-                                "value": symptom_params['threshold_value'],
-                                "valueType": "NUMERIC",
-                                "instanced": False,
-                                "thresholdType": "STATIC"
-                            }
-                        }
+                "id": None,
+                "name": symptom_params['symptom_name'],
+                "adapterKindKey": symptom_params['adapter_kind_key'],
+                "resourceKindKey": symptom_params['resource_kind_key'],
+                "waitCycles": symptom_params['wait_cycles'],
+                "cancelCycles": symptom_params['cancel_cycles'],
+                "state": {
+                    "severity": symptom_params['severity'],
+                    "condition": {
+                        "type": "CONDITION_HT",
+                        "key": symptom_params['metric_key'],
+                        "operator": symptom_params['operation'],
+                        "value": symptom_params['threshold_value'],
+                        "valueType": "NUMERIC",
+                        "instanced": False,
+                        "thresholdType": "STATIC"
                     }
+                }
+            }
 
             resp = requests.post(self.vrops_site + api_url,
                                  auth=(self.vrops_user, self.vrops_password),
                                  headers=headers,
-                                 verify = False,
+                                 verify=False,
                                  data=json.dumps(data))
 
             if resp.status_code != 201:
-                self.logger.warning("Failed to create Symptom definition: {}, response {}"\
-                        .format(symptom_params['symptom_name'], resp.content))
+                logger.warning("Failed to create Symptom definition: {}, response {}"
+                               .format(symptom_params['symptom_name'], resp.content))
                 return None
 
             resp_data = json.loads(resp.content)
@@ -322,9 +327,8 @@ class MonPlugin():
             return symptom_id
 
         except Exception as exp:
-            self.logger.warning("Error creating symptom definition : {}\n{}"\
-            .format(exp, traceback.format_exc()))
-
+            logger.warning("Error creating symptom definition : {} n{}"
+                           .format(exp, traceback.format_exc()))
 
     def create_alarm_definition(self, alarm_params):
         """
@@ -351,42 +355,42 @@ class MonPlugin():
             api_url = '/suite-api/api/alertdefinitions'
             headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
             data = {
-                        "name": alarm_params['name'],
-                        "description": alarm_params['description'],
-                        "adapterKindKey": alarm_params['adapterKindKey'],
-                        "resourceKindKey": alarm_params['resourceKindKey'],
-                        "waitCycles": 1,
-                        "cancelCycles": 1,
-                        "type": alarm_params['type'],
-                        "subType": alarm_params['subType'],
-                        "states": [
+                "name": alarm_params['name'],
+                "description": alarm_params['description'],
+                "adapterKindKey": alarm_params['adapterKindKey'],
+                "resourceKindKey": alarm_params['resourceKindKey'],
+                "waitCycles": 1,
+                "cancelCycles": 1,
+                "type": alarm_params['type'],
+                "subType": alarm_params['subType'],
+                "states": [
+                    {
+                        "severity": alarm_params['severity'],
+                        "base-symptom-set":
                             {
-                                "severity": alarm_params['severity'],
-                                "base-symptom-set":
-                                    {
-                                        "type": "SYMPTOM_SET",
-                                        "relation": "SELF",
-                                        "aggregation": "ALL",
-                                        "symptomSetOperator": "AND",
-                                        "symptomDefinitionIds": [alarm_params['symptomDefinitionId']]
-                                    },
-                                "impact": {
-                                    "impactType": "BADGE",
-                                    "detail": alarm_params['impact']
-                                }
-                            }
-                        ]
+                                "type": "SYMPTOM_SET",
+                                "relation": "SELF",
+                                "aggregation": "ALL",
+                                "symptomSetOperator": "AND",
+                                "symptomDefinitionIds": [alarm_params['symptomDefinitionId']]
+                            },
+                        "impact": {
+                            "impactType": "BADGE",
+                            "detail": alarm_params['impact']
+                        }
                     }
+                ]
+            }
 
             resp = requests.post(self.vrops_site + api_url,
                                  auth=(self.vrops_user, self.vrops_password),
                                  headers=headers,
-                                 verify = False,
+                                 verify=False,
                                  data=json.dumps(data))
 
             if resp.status_code != 201:
-                self.logger.warning("Failed to create Alarm definition: {}, response {}"\
-                        .format(alarm_params['name'], resp.content))
+                logger.warning("Failed to create Alarm definition: {}, response {}"
+                               .format(alarm_params['name'], resp.content))
                 return None
 
             resp_data = json.loads(resp.content)
@@ -396,8 +400,7 @@ class MonPlugin():
             return alarm_uuid
 
         except Exception as exp:
-            self.logger.warning("Error creating alarm definition : {}\n{}".format(exp, traceback.format_exc()))
-
+            logger.warning("Error creating alarm definition : {} n{}".format(exp, traceback.format_exc()))
 
     def configure_rest_plugin(self):
         """
@@ -409,7 +412,7 @@ class MonPlugin():
         plugin_name = 'MON_module_REST_Plugin'
         plugin_id = self.check_if_plugin_configured(plugin_name)
 
-        #If REST plugin not configured, configure it
+        # If REST plugin not configured, configure it
         if plugin_id is not None:
             return plugin_id
         else:
@@ -417,45 +420,45 @@ class MonPlugin():
                 cert_file_string = open(SSL_CERTIFICATE_FILE_PATH, "rb").read()
             except IOError as exp:
                 msg = ("Could not read SSL certificate file: {}".format(SSL_CERTIFICATE_FILE_PATH))
-                self.logger.error(msg)
+                logger.error(msg)
                 raise IOError(msg)
             cert = load_certificate(FILETYPE_PEM, cert_file_string)
             certificate = cert.digest("sha1")
             api_url = '/suite-api/api/alertplugins'
             headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
             data = {
-                        "pluginTypeId": "RestPlugin",
-                        "name": plugin_name,
-                        "configValues": [
-                            {
-                                "name": "Url",
-                                "value": webhook_url
-                            },
-                            {
-                                "name": "Content-type",
-                                "value": "application/json"
-                            },
-                            {
-                                "name": "Certificate",
-                                "value": certificate
-                            },
-                            {
-                                "name": "ConnectionCount",
-                                "value": "20"
-                            }
-                        ]
+                "pluginTypeId": "RestPlugin",
+                "name": plugin_name,
+                "configValues": [
+                    {
+                        "name": "Url",
+                        "value": webhook_url
+                    },
+                    {
+                        "name": "Content-type",
+                        "value": "application/json"
+                    },
+                    {
+                        "name": "Certificate",
+                        "value": certificate
+                    },
+                    {
+                        "name": "ConnectionCount",
+                        "value": "20"
                     }
+                ]
+            }
 
             resp = requests.post(self.vrops_site + api_url,
                                  auth=(self.vrops_user, self.vrops_password),
                                  headers=headers,
-                                 verify = False,
+                                 verify=False,
                                  data=json.dumps(data))
 
             if resp.status_code is not 201:
-                self.logger.warning("Failed to create REST Plugin: {} for url: {}, \nresponse code: {},"\
-                            "\nresponse content: {}".format(plugin_name, webhook_url,\
-                            resp.status_code, resp.content))
+                logger.warning("Failed to create REST Plugin: {} for url: {},  nresponse code: {},"
+                               " nresponse content: {}".format(plugin_name, webhook_url,
+                                                               resp.status_code, resp.content))
                 return None
 
             resp_data = json.loads(resp.content)
@@ -463,16 +466,18 @@ class MonPlugin():
                 plugin_id = resp_data['pluginId']
 
             if plugin_id is None:
-                self.logger.warning("Failed to get REST Plugin ID for {}, url: {}".format(plugin_name, webhook_url))
+                logger.warning("Failed to get REST Plugin ID for {}, url: {}".format(plugin_name, webhook_url))
                 return None
             else:
-                self.logger.info("Created REST Plugin: {} with ID : {} for url: {}".format(plugin_name, plugin_id, webhook_url))
+                logger.info(
+                    "Created REST Plugin: {} with ID : {} for url: {}".format(plugin_name, plugin_id, webhook_url))
                 status = self.enable_rest_plugin(plugin_id, plugin_name)
                 if status is False:
-                    self.logger.warning("Failed to enable created REST Plugin: {} for url: {}".format(plugin_name, webhook_url))
+                    logger.warning(
+                        "Failed to enable created REST Plugin: {} for url: {}".format(plugin_name, webhook_url))
                     return None
                 else:
-                    self.logger.info("Enabled REST Plugin: {} for url: {}".format(plugin_name, webhook_url))
+                    logger.info("Enabled REST Plugin: {} for url: {}".format(plugin_name, webhook_url))
                     return plugin_id
 
     def check_if_plugin_configured(self, plugin_name):
@@ -480,17 +485,17 @@ class MonPlugin():
         Returns: plugin_id: if already created, None: if needs to be created
         """
         plugin_id = None
-        #Find the REST Plugin id details for - MON_module_REST_Plugin
+        # Find the REST Plugin id details for - MON_module_REST_Plugin
         api_url = '/suite-api/api/alertplugins'
         headers = {'Accept': 'application/json'}
 
         resp = requests.get(self.vrops_site + api_url,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Failed to REST GET Alarm plugin details \nResponse code: {}\nResponse content: {}"\
-            .format(resp.status_code, resp.content))
+            logger.warning("Failed to REST GET Alarm plugin details  nResponse code: {} nResponse content: {}"
+                           .format(resp.status_code, resp.content))
             return None
 
         # Look for specific plugin & parse pluginId for 'MON_module_REST_Plugin'
@@ -501,13 +506,12 @@ class MonPlugin():
                     plugin_id = notify_plugin.get('pluginId')
 
         if plugin_id is None:
-            self.logger.warning("REST plugin {} not found".format(plugin_name))
+            logger.warning("REST plugin {} not found".format(plugin_name))
             return None
         else:
-            self.logger.info("Found REST Plugin: {}".format(plugin_name))
+            logger.info("Found REST Plugin: {}".format(plugin_name))
             return plugin_id
 
-
     def enable_rest_plugin(self, plugin_id, plugin_name):
         """
         Enable the REST plugin using plugin_id
@@ -516,8 +520,8 @@ class MonPlugin():
         """
 
         if plugin_id is None or plugin_name is None:
-            self.logger.debug("enable_rest_plugin() : Plugin ID or plugin_name not provided for {} plugin"\
-                        .format(plugin_name))
+            logger.debug("enable_rest_plugin() : Plugin ID or plugin_name not provided for {} plugin"
+                         .format(plugin_name))
             return False
 
         try:
@@ -525,19 +529,19 @@ class MonPlugin():
 
             resp = requests.put(self.vrops_site + api_url,
                                 auth=(self.vrops_user, self.vrops_password),
-                                verify = False)
+                                verify=False)
 
             if resp.status_code is not 204:
-                self.logger.warning("Failed to enable REST plugin {}. \nResponse code {}\nResponse Content: {}"\
-                        .format(plugin_name, resp.status_code, resp.content))
+                logger.warning("Failed to enable REST plugin {}.  nResponse code {} nResponse Content: {}"
+                               .format(plugin_name, resp.status_code, resp.content))
                 return False
 
-            self.logger.info("Enabled REST plugin {}.".format(plugin_name))
+            logger.info("Enabled REST plugin {}.".format(plugin_name))
             return True
 
         except Exception as exp:
-            self.logger.warning("Error enabling REST plugin for {} plugin: Exception: {}\n{}"\
-                    .format(plugin_name, exp, traceback.format_exc()))
+            logger.warning("Error enabling REST plugin for {} plugin: Exception: {} n{}"
+                           .format(plugin_name, exp, traceback.format_exc()))
 
     def create_alarm_notification_rule(self, alarm_name, alarm_id, resource_id):
         """
@@ -554,45 +558,45 @@ class MonPlugin():
         notification_id = None
         plugin_name = 'MON_module_REST_Plugin'
 
-        #1) Find the REST Plugin id details for - MON_module_REST_Plugin
+        # 1) Find the REST Plugin id details for - MON_module_REST_Plugin
         plugin_id = self.check_if_plugin_configured(plugin_name)
         if plugin_id is None:
-            self.logger.warning("Failed to get REST plugin_id for : {}".format('MON_module_REST_Plugin'))
+            logger.warning("Failed to get REST plugin_id for : {}".format('MON_module_REST_Plugin'))
             return None
 
-        #2) Create Alarm notification rule
+        # 2) Create Alarm notification rule
         api_url = '/suite-api/api/notifications/rules'
         headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
         data = {
-                    "name" : notification_name,
-                    "pluginId" : plugin_id,
-                    "resourceFilter": {
-                        "matchResourceIdOnly": True,
-                        "resourceId": resource_id
-                        },
-                    "alertDefinitionIdFilters" : {
-                    "values" : [ alarm_id ]
-                    }
-                }
+            "name": notification_name,
+            "pluginId": plugin_id,
+            "resourceFilter": {
+                "matchResourceIdOnly": True,
+                "resourceId": resource_id
+            },
+            "alertDefinitionIdFilters": {
+                "values": [alarm_id]
+            }
+        }
 
         resp = requests.post(self.vrops_site + api_url,
                              auth=(self.vrops_user, self.vrops_password),
                              headers=headers,
-                             verify = False,
+                             verify=False,
                              data=json.dumps(data))
 
         if resp.status_code is not 201:
-            self.logger.warning("Failed to create Alarm notification rule {} for {} alarm."\
-                        "\nResponse code: {}\nResponse content: {}"\
-                        .format(notification_name, alarm_name, resp.status_code, resp.content))
+            logger.warning("Failed to create Alarm notification rule {} for {} alarm."
+                           " nResponse code: {} nResponse content: {}"
+                           .format(notification_name, alarm_name, resp.status_code, resp.content))
             return None
 
-        #parse notification id from response
+        # parse notification id from response
         resp_data = json.loads(resp.content)
         if resp_data.get('id') is not None:
             notification_id = resp_data['id']
 
-        self.logger.info("Created Alarm notification rule {} for {} alarm.".format(notification_name, alarm_name))
+        logger.info("Created Alarm notification rule {} for {} alarm.".format(notification_name, alarm_name))
         return notification_id
 
     def get_vm_moref_id(self, vapp_uuid):
@@ -604,14 +608,12 @@ class MonPlugin():
                 vm_details = self.get_vapp_details_rest(vapp_uuid)
                 if vm_details and "vm_vcenter_info" in vm_details:
                     vm_moref_id = vm_details["vm_vcenter_info"].get("vm_moref_id", None)
-
-            self.logger.info("Found vm_moref_id: {} for vApp UUID: {}".format(vm_moref_id, vapp_uuid))
-            return vm_moref_id
+                    logger.info("Found vm_moref_id: {} for vApp UUID: {}".format(vm_moref_id, vapp_uuid))
+                    return vm_moref_id
 
         except Exception as exp:
-            self.logger.warning("Error occurred while getting VM moref ID for VM : {}\n{}"\
-                        .format(exp, traceback.format_exc()))
-
+            logger.warning("Error occurred while getting VM moref ID for VM : {} n{}"
+                           .format(exp, traceback.format_exc()))
 
     def get_vapp_details_rest(self, vapp_uuid=None):
         """
@@ -632,31 +634,31 @@ class MonPlugin():
 
         vca = self.connect_as_admin()
         if not vca:
-            self.logger.warning("Failed to connect to vCD")
+            logger.warning("Failed to connect to vCD")
             return parsed_respond
 
         url_list = [self.vcloud_site, '/api/vApp/vapp-', vapp_uuid]
         get_vapp_restcall = ''.join(url_list)
 
         if vca._session:
-            headers = {'Accept':'application/*+xml;version=' + API_VERSION,
+            headers = {'Accept': 'application/*+xml;version=' + API_VERSION,
                        'x-vcloud-authorization': vca._session.headers['x-vcloud-authorization']}
             response = requests.get(get_vapp_restcall,
                                     headers=headers,
                                     verify=False)
 
             if response.status_code != 200:
-                self.logger.warning("REST API call {} failed. Return status code {}"\
-                            .format(get_vapp_restcall, response.content))
+                logger.warning("REST API call {} failed. Return status code {}"
+                               .format(get_vapp_restcall, response.content))
                 return parsed_respond
 
             try:
                 xmlroot_respond = XmlElementTree.fromstring(response.content)
 
                 namespaces = {'vm': 'http://www.vmware.com/vcloud/v1.5',
-                              "vmext":"http://www.vmware.com/vcloud/extension/v1.5",
-                              "xmlns":"http://www.vmware.com/vcloud/v1.5"
-                             }
+                              "vmext": "http://www.vmware.com/vcloud/extension/v1.5",
+                              "xmlns": "http://www.vmware.com/vcloud/v1.5"
+                              }
 
                 # parse children section for other attrib
                 children_section = xmlroot_respond.find('vm:Children/', namespaces)
@@ -668,15 +670,14 @@ class MonPlugin():
                         vmext = vim_info.find('vmext:VmVimObjectRef', namespaces)
                         if vmext is not None:
                             vm_vcenter_info["vm_moref_id"] = vmext.find('vmext:MoRef', namespaces).text
-                        parsed_respond["vm_vcenter_info"]= vm_vcenter_info
+                        parsed_respond["vm_vcenter_info"] = vm_vcenter_info
 
-            except Exception as exp :
-                self.logger.warning("Error occurred calling rest api for getting vApp details: {}\n{}"\
-                            .format(exp, traceback.format_exc()))
+            except Exception as exp:
+                logger.warning("Error occurred calling rest api for getting vApp details: {} n{}"
+                               .format(exp, traceback.format_exc()))
 
         return parsed_respond
 
-
     def connect_as_admin(self):
         """ Method connect as pvdc admin user to vCloud director.
             There are certain action that can be done only by provider vdc admin user.
@@ -686,20 +687,19 @@ class MonPlugin():
                 The return vca object that letter can be used to connect to vcloud direct as admin for provider vdc
         """
 
-        self.logger.debug("Logging into vCD org as admin.")
+        logger.debug("Logging into vCD org as admin.")
 
         try:
             host = self.vcloud_site
             org = 'System'
             client_as_admin = Client(host, verify_ssl_certs=False)
-            client_as_admin.set_credentials(BasicLoginCredentials(self.admin_username, org,\
+            client_as_admin.set_credentials(BasicLoginCredentials(self.admin_username, org,
                                                                   self.admin_password))
-        except Exception as e:
-            self.logger.warning("Can't connect to a vCloud director as: {} with exception {}"\
-                             .format(self.admin_username, e))
-
-        return client_as_admin
+            return client_as_admin
 
+        except Exception as e:
+            logger.warning("Can't connect to a vCloud director as: {} with exception {}"
+                           .format(self.admin_username, e))
 
     def get_vm_resource_id(self, vm_moref_id):
         """ Find resource ID in vROPs using vm_moref_id
@@ -712,12 +712,12 @@ class MonPlugin():
 
         resp = requests.get(self.vrops_site + api_url,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Failed to get resource details from vROPs for {}"\
-                             "\nResponse code:{}\nResponse Content: {}"\
-                             .format(vm_moref_id, resp.status_code, resp.content))
+            logger.warning("Failed to get resource details from vROPs for {}"
+                           " nResponse code:{} nResponse Content: {}"
+                           .format(vm_moref_id, resp.status_code, resp.content))
             return None
 
         vm_resource_id = None
@@ -731,20 +731,19 @@ class MonPlugin():
                         if resource_details.get('resourceIdentifiers') is not None:
                             resource_identifiers = resource_details['resourceIdentifiers']
                             for resource_identifier in resource_identifiers:
-                                if resource_identifier['identifierType']['name']=='VMEntityObjectID':
+                                if resource_identifier['identifierType']['name'] == 'VMEntityObjectID':
                                     if resource_identifier.get('value') is not None and \
-                                        resource_identifier['value']==vm_moref_id:
+                                            resource_identifier['value'] == vm_moref_id:
                                         vm_resource_id = resource['identifier']
-                                        self.logger.info("Found VM resource ID: {} for vm_moref_id: {}"\
-                                                         .format(vm_resource_id, vm_moref_id))
+                                        logger.info("Found VM resource ID: {} for vm_moref_id: {}"
+                                                    .format(vm_resource_id, vm_moref_id))
 
         except Exception as exp:
-            self.logger.warning("get_vm_resource_id: Error in parsing {}\n{}"\
-                             .format(exp, traceback.format_exc()))
+            logger.warning("get_vm_resource_id: Error in parsing {} n{}"
+                           .format(exp, traceback.format_exc()))
 
         return vm_resource_id
 
-
     def get_metrics_data(self, metric={}):
         """Get an individual metric's data of a resource.
         Params:
@@ -768,79 +767,80 @@ class MonPlugin():
         return_data['schema_type'] = 'read_metric_data_response'
         return_data['vim_uuid'] = metric['vim_uuid']
         return_data['metric_name'] = metric['metric_name']
-        #To do - No metric_uuid in vROPs, thus returning '0'
+        # To do - No metric_uuid in vROPs, thus returning '0'
         return_data['metric_uuid'] = '0'
         return_data['correlation_id'] = metric['correlation_id']
         return_data['resource_uuid'] = metric['resource_uuid']
-        return_data['metrics_data'] = {'time_series':[], 'metrics_series':[]}
-        #To do - Need confirmation about uuid & id
+        return_data['metrics_data'] = {'time_series': [], 'metrics_series': []}
+        # To do - Need confirmation about uuid & id
         ##if 'tenant_uuid' in metric and metric['tenant_uuid'] is not None:
         ##    return_data['tenant_uuid'] = metric['tenant_uuid']
         ##else:
         ##    return_data['tenant_uuid'] = None
         return_data['unit'] = None
-        #return_data['tenant_id'] = self.tenant_id
-        #self.logger.warning("return_data: {}".format(return_data))
+        # return_data['tenant_id'] = self.tenant_id
+        # logger.warning("return_data: {}".format(return_data))
 
-        #1) Get metric details from plugin specific file & format it into vROPs metrics
+        # 1) Get metric details from plugin specific file & format it into vROPs metrics
         metric_key_params = self.get_default_Params(metric['metric_name'])
 
         if not metric_key_params:
-            self.logger.warning("Metric not supported: {}".format(metric['metric_name']))
-            #To Do: Return message
+            logger.warning("Metric not supported: {}".format(metric['metric_name']))
+            # To Do: Return message
             return return_data
 
         return_data['unit'] = metric_key_params['unit']
 
-        #2) Find the resource id in vROPs based on OSM resource_uuid
-        #2.a) Find vm_moref_id from vApp uuid in vCD
+        # 2) Find the resource id in vROPs based on OSM resource_uuid
+        # 2.a) Find vm_moref_id from vApp uuid in vCD
         vm_moref_id = self.get_vm_moref_id(metric['resource_uuid'])
         if vm_moref_id is None:
-            self.logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(metric['resource_uuid']))
+            logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(metric['resource_uuid']))
             return return_data
-        #2.b) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
+        # 2.b) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
         resource_id = self.get_vm_resource_id(vm_moref_id)
         if resource_id is None:
-            self.logger.warning("Failed to find resource in vROPs: {}".format(metric['resource_uuid']))
+            logger.warning("Failed to find resource in vROPs: {}".format(metric['resource_uuid']))
             return return_data
 
-        #3) Calculate begin & end time for period & period unit
+        # 3) Calculate begin & end time for period & period unit
         end_time = int(round(time.time() * 1000))
         if metric['collection_unit'] == 'YR':
             time_diff = PERIOD_MSEC[metric['collection_unit']]
         else:
-            time_diff = metric['collection_period']* PERIOD_MSEC[metric['collection_unit']]
+            time_diff = metric['collection_period'] * PERIOD_MSEC[metric['collection_unit']]
         begin_time = end_time - time_diff
 
-        #4) Get the metrics data
-        self.logger.info("metric_key_params['metric_key'] = {}".format(metric_key_params['metric_key']))
-        self.logger.info("end_time: {}, begin_time: {}".format(end_time, begin_time))
+        # 4) Get the metrics data
+        logger.info("metric_key_params['metric_key'] = {}".format(metric_key_params['metric_key']))
+        logger.info("end_time: {}, begin_time: {}".format(end_time, begin_time))
 
-        url_list = ['/suite-api/api/resources/', resource_id, '/stats?statKey=',\
-                    metric_key_params['metric_key'], '&begin=', str(begin_time),'&end=',str(end_time)]
+        url_list = ['/suite-api/api/resources/', resource_id, '/stats?statKey=',
+                    metric_key_params['metric_key'], '&begin=', str(begin_time), '&end=', str(end_time)]
         api_url = ''.join(url_list)
         headers = {'Accept': 'application/json'}
 
         resp = requests.get(self.vrops_site + api_url,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Failed to retrieve Metric data from vROPs for {}\nResponse code:{}\nResponse Content: {}"\
+            logger.warning(
+                "Failed to retrieve Metric data from vROPs for {} nResponse code:{} nResponse Content: {}"
                     .format(metric['metric_name'], resp.status_code, resp.content))
             return return_data
 
-        #5) Convert to required format
+        # 5) Convert to required format
         metrics_data = {}
         json_data = json.loads(resp.content)
-        for resp_key,resp_val in six.iteritems(json_data):
+        for resp_key, resp_val in six.iteritems(json_data):
             if resp_key == 'values':
                 data = json_data['values'][0]
-                for data_k,data_v in six.iteritems(data):
+                for data_k, data_v in six.iteritems(data):
                     if data_k == 'stat-list':
                         stat_list = data_v
-                        for stat_list_k,stat_list_v in six.iteritems(stat_list):
-                            for stat_keys,stat_vals in six.iteritems(stat_list_v[0]):
+                        for stat_list_k, stat_list_v in six.iteritems(stat_list):
+                            for stat_keys, stat_vals in six.iteritems(stat_list_v[0]):
                                 if stat_keys == 'timestamps':
                                     metrics_data['time_series'] = stat_list_v[0]['timestamps']
                                 if stat_keys == 'data':
@@ -854,27 +854,27 @@ class MonPlugin():
         """Update alarm configuration (i.e. Symptom & alarm) as per request
         """
         if new_alarm_config.get('alarm_uuid') is None:
-            self.logger.warning("alarm_uuid is required to update an Alarm")
+            logger.warning("alarm_uuid is required to update an Alarm")
             return None
-        #1) Get Alarm details from it's uuid & find the symptom definition
+        # 1) Get Alarm details from it's uuid & find the symptom definition
         alarm_details_json, alarm_details = self.get_alarm_defination_details(new_alarm_config['alarm_uuid'])
         if alarm_details_json is None:
             return None
 
         try:
-            #2) Update the symptom definition
+            # 2) Update the symptom definition
             if alarm_details['alarm_id'] is not None and alarm_details['symptom_definition_id'] is not None:
                 symptom_defination_id = alarm_details['symptom_definition_id']
             else:
-                self.logger.info("Symptom Definition ID not found for {}".format(new_alarm_config['alarm_uuid']))
+                logger.info("Symptom Definition ID not found for {}".format(new_alarm_config['alarm_uuid']))
                 return None
 
             symptom_uuid = self.update_symptom_defination(symptom_defination_id, new_alarm_config)
 
-            #3) Update the alarm definition & Return UUID if successful update
+            # 3) Update the alarm definition & Return UUID if successful update
             if symptom_uuid is None:
-                self.logger.info("Symptom Definition details not found for {}"\
-                                .format(new_alarm_config['alarm_uuid']))
+                logger.info("Symptom Definition details not found for {}"
+                            .format(new_alarm_config['alarm_uuid']))
                 return None
             else:
                 alarm_uuid = self.reconfigure_alarm(alarm_details_json, new_alarm_config)
@@ -883,13 +883,13 @@ class MonPlugin():
                 else:
                     return alarm_uuid
         except:
-            self.logger.error("Exception while updating alarm: {}".format(traceback.format_exc()))
+            logger.error("Exception while updating alarm: {}".format(traceback.format_exc()))
 
     def get_alarm_defination_details(self, alarm_uuid):
         """Get alarm details based on alarm UUID
         """
         if alarm_uuid is None:
-            self.logger.warning("get_alarm_defination_details: Alarm UUID not provided")
+            logger.warning("get_alarm_defination_details: Alarm UUID not provided")
             return None, None
 
         alarm_details = {}
@@ -899,11 +899,11 @@ class MonPlugin():
 
         resp = requests.get(self.vrops_site + api_url + alarm_uuid,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Alarm to be updated not found: {}\nResponse code:{}\nResponse Content: {}"\
-                    .format(alarm_uuid, resp.status_code, resp.content))
+            logger.warning("Alarm to be updated not found: {} nResponse code:{} nResponse Content: {}"
+                           .format(alarm_uuid, resp.status_code, resp.content))
             return None, None
 
         try:
@@ -915,14 +915,14 @@ class MonPlugin():
                 alarm_details['resource_kind'] = json_data['resourceKindKey']
                 alarm_details['type'] = json_data['type']
                 alarm_details['sub_type'] = json_data['subType']
-                alarm_details['symptom_definition_id'] = json_data['states'][0]['base-symptom-set']['symptomDefinitionIds'][0]
+                alarm_details['symptom_definition_id'] = \
+                    json_data['states'][0]['base-symptom-set']['symptomDefinitionIds'][0]
         except Exception as exp:
-            self.logger.warning("Exception while retrieving alarm definition details: {}".format(exp))
+            logger.warning("Exception while retrieving alarm definition details: {}".format(exp))
             return None, None
 
         return json_data, alarm_details
 
-
     def get_alarm_defination_by_name(self, alarm_name):
         """Get alarm details based on alarm name
         """
@@ -930,7 +930,7 @@ class MonPlugin():
         alert_match_list = []
 
         if alarm_name is None:
-            self.logger.warning("get_alarm_defination_by_name: Alarm name not provided")
+            logger.warning("get_alarm_defination_by_name: Alarm name not provided")
             return alert_match_list
 
         json_data = {}
@@ -939,11 +939,11 @@ class MonPlugin():
 
         resp = requests.get(self.vrops_site + api_url,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("get_alarm_defination_by_name: Error in response: {}\nResponse code:{}"\
-                    "\nResponse Content: {}".format(alarm_name, resp.status_code, resp.content))
+            logger.warning("get_alarm_defination_by_name: Error in response: {} nResponse code:{}"
+                           " nResponse Content: {}".format(alarm_name, resp.status_code, resp.content))
             return alert_match_list
 
         try:
@@ -952,21 +952,20 @@ class MonPlugin():
                 alerts_list = json_data['alertDefinitions']
                 alert_match_list = list(filter(lambda alert: alert['name'] == alarm_name, alerts_list))
                 status = False if not alert_match_list else True
-                #self.logger.debug("Found alert_match_list: {}for larm_name: {},\nstatus: {}".format(alert_match_list, alarm_name,status))
+                # logger.debug("Found alert_match_list: {}for larm_name: {}, nstatus: {}".format(alert_match_list, alarm_name,status))
 
             return alert_match_list
 
         except Exception as exp:
-            self.logger.warning("Exception while searching alarm definition: {}".format(exp))
+            logger.warning("Exception while searching alarm definition: {}".format(exp))
             return alert_match_list
 
-
     def update_symptom_defination(self, symptom_uuid, new_alarm_config):
         """Update symptom definition based on new alarm input configuration
         """
-        #1) Get symptom definition details
+        # 1) Get symptom definition details
         symptom_details = self.get_symptom_defination_details(symptom_uuid)
-        #print "\n\nsymptom_details: {}".format(symptom_details)
+        # print " n nsymptom_details: {}".format(symptom_details)
         if symptom_details is None:
             return None
 
@@ -976,48 +975,46 @@ class MonPlugin():
             symptom_details['state']['condition']['operator'] = OPERATION_MAPPING[new_alarm_config['operation']]
         if 'threshold_value' in new_alarm_config and new_alarm_config['threshold_value'] is not None:
             symptom_details['state']['condition']['value'] = new_alarm_config['threshold_value']
-        #Find vrops metric key from metric_name, if required
+        # Find vrops metric key from metric_name, if required
         """
         if 'metric_name' in new_alarm_config and new_alarm_config['metric_name'] is not None:
             metric_key_params = self.get_default_Params(new_alarm_config['metric_name'])
             if not metric_key_params:
-                self.logger.warning("Metric not supported: {}".format(config_dict['metric_name']))
+                logger.warning("Metric not supported: {}".format(config_dict['metric_name']))
                 return None
             symptom_details['state']['condition']['key'] = metric_key_params['metric_key']
         """
-        self.logger.info("Fetched Symptom details : {}".format(symptom_details))
+        logger.info("Fetched Symptom details : {}".format(symptom_details))
 
         api_url = '/suite-api/api/symptomdefinitions'
-        headers = {'Content-Type': 'application/json', 'Accept':'application/json'}
+        headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
         data = json.dumps(symptom_details)
         resp = requests.put(self.vrops_site + api_url,
-                             auth=(self.vrops_user, self.vrops_password),
-                             headers=headers,
-                             verify = False,
-                             data=data)
+                            auth=(self.vrops_user, self.vrops_password),
+                            headers=headers,
+                            verify=False,
+                            data=data)
 
         if resp.status_code != 200:
-            self.logger.warning("Failed to update Symptom definition: {}, response {}"\
-                    .format(symptom_uuid, resp.content))
+            logger.warning("Failed to update Symptom definition: {}, response {}"
+                           .format(symptom_uuid, resp.content))
             return None
 
-
         if symptom_uuid is not None:
-            self.logger.info("Symptom definition updated {} for alarm: {}"\
-                    .format(symptom_uuid, new_alarm_config['alarm_uuid']))
+            logger.info("Symptom definition updated {} for alarm: {}"
+                        .format(symptom_uuid, new_alarm_config['alarm_uuid']))
             return symptom_uuid
         else:
-            self.logger.warning("Failed to update Symptom Definition {} for : {}"\
-                    .format(symptom_uuid, new_alarm_config['alarm_uuid']))
+            logger.warning("Failed to update Symptom Definition {} for : {}"
+                           .format(symptom_uuid, new_alarm_config['alarm_uuid']))
             return None
 
-
     def get_symptom_defination_details(self, symptom_uuid):
         """Get symptom definition details
         """
         symptom_details = {}
         if symptom_uuid is None:
-            self.logger.warning("get_symptom_defination_details: Symptom UUID not provided")
+            logger.warning("get_symptom_defination_details: Symptom UUID not provided")
             return None
 
         api_url = '/suite-api/api/symptomdefinitions/'
@@ -1025,18 +1022,17 @@ class MonPlugin():
 
         resp = requests.get(self.vrops_site + api_url + symptom_uuid,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Symptom definition not found {} \nResponse code:{}\nResponse Content: {}"\
-                    .format(symptom_uuid, resp.status_code, resp.content))
+            logger.warning("Symptom definition not found {}  nResponse code:{} nResponse Content: {}"
+                           .format(symptom_uuid, resp.status_code, resp.content))
             return None
 
         symptom_details = json.loads(resp.content)
-        #print "New symptom Details: {}".format(symptom_details)
+        # print "New symptom Details: {}".format(symptom_details)
         return symptom_details
 
-
     def reconfigure_alarm(self, alarm_details_json, new_alarm_config):
         """Reconfigure alarm definition as per input
         """
@@ -1046,52 +1042,52 @@ class MonPlugin():
             alarm_details_json['description'] = new_alarm_config['description']
 
         api_url = '/suite-api/api/alertdefinitions'
-        headers = {'Content-Type': 'application/json', 'Accept':'application/json'}
+        headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
         data = json.dumps(alarm_details_json)
         resp = requests.put(self.vrops_site + api_url,
-                             auth=(self.vrops_user, self.vrops_password),
-                             headers=headers,
-                             verify = False,
-                             data=data)
+                            auth=(self.vrops_user, self.vrops_password),
+                            headers=headers,
+                            verify=False,
+                            data=data)
 
         if resp.status_code != 200:
-            self.logger.warning("Failed to update Alarm definition: {}, response code {}, response content: {}"\
-                    .format(alarm_details_json['id'], resp.status_code, resp.content))
+            logger.warning("Failed to update Alarm definition: {}, response code {}, response content: {}"
+                           .format(alarm_details_json['id'], resp.status_code, resp.content))
             return None
         else:
             parsed_alarm_details = json.loads(resp.content)
             alarm_def_uuid = parsed_alarm_details['id'].split('-', 1)[1]
-            self.logger.info("Successfully updated Alarm definition: {}".format(alarm_def_uuid))
+            logger.info("Successfully updated Alarm definition: {}".format(alarm_def_uuid))
             return alarm_def_uuid
 
     def delete_alarm_configuration(self, delete_alarm_req_dict):
         """Delete complete alarm configuration
         """
         if delete_alarm_req_dict['alarm_uuid'] is None:
-            self.logger.info("delete_alarm_configuration: Alarm UUID not provided")
+            logger.info("delete_alarm_configuration: Alarm UUID not provided")
             return None
-        #1)Get alarm & symptom definition details
+        # 1)Get alarm & symptom definition details
         alarm_details_json, alarm_details = self.get_alarm_defination_details(delete_alarm_req_dict['alarm_uuid'])
         if alarm_details is None or alarm_details_json is None:
             return None
 
-        #2) Delete alarm notification
+        # 2) Delete alarm notification
         rule_id = self.delete_notification_rule(alarm_details['alarm_name'])
         if rule_id is None:
             return None
 
-        #3) Delete alarm configuration
+        # 3) Delete alarm configuration
         alarm_id = self.delete_alarm_defination(alarm_details['alarm_id'])
         if alarm_id is None:
             return None
 
-        #4) Delete alarm symptom
+        # 4) Delete alarm symptom
         symptom_id = self.delete_symptom_definition(alarm_details['symptom_definition_id'])
         if symptom_id is None:
             return None
         else:
-            self.logger.info("Completed deleting alarm configuration: {}"\
-                    .format(delete_alarm_req_dict['alarm_uuid']))
+            logger.info("Completed deleting alarm configuration: {}"
+                        .format(delete_alarm_req_dict['alarm_uuid']))
             return delete_alarm_req_dict['alarm_uuid']
 
     def delete_notification_rule(self, alarm_name):
@@ -1102,15 +1098,15 @@ class MonPlugin():
             return None
         else:
             api_url = '/suite-api/api/notifications/rules/'
-            headers = {'Accept':'application/json'}
+            headers = {'Accept': 'application/json'}
             resp = requests.delete(self.vrops_site + api_url + rule_id,
-                                auth=(self.vrops_user, self.vrops_password),
-                                verify = False, headers = headers)
+                                   auth=(self.vrops_user, self.vrops_password),
+                                   verify=False, headers=headers)
             if resp.status_code is not 204:
-                self.logger.warning("Failed to delete notification rules for {}".format(alarm_name))
+                logger.warning("Failed to delete notification rules for {}".format(alarm_name))
                 return None
             else:
-                self.logger.info("Deleted notification rules for {}".format(alarm_name))
+                logger.info("Deleted notification rules for {}".format(alarm_name))
                 return rule_id
 
     def get_notification_rule_id_by_alarm_name(self, alarm_name):
@@ -1118,14 +1114,14 @@ class MonPlugin():
         """
         alarm_notify_id = 'notify_' + alarm_name
         api_url = '/suite-api/api/notifications/rules'
-        headers = {'Content-Type': 'application/json', 'Accept':'application/json'}
+        headers = {'Content-Type': 'application/json', 'Accept': 'application/json'}
         resp = requests.get(self.vrops_site + api_url,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Failed to get notification rules details for {}"\
-                    .format(alarm_name))
+            logger.warning("Failed to get notification rules details for {}"
+                           .format(alarm_name))
             return None
 
         notifications = json.loads(resp.content)
@@ -1134,45 +1130,44 @@ class MonPlugin():
             for dict in notifications_list:
                 if dict['name'] is not None and dict['name'] == alarm_notify_id:
                     notification_id = dict['id']
-                    self.logger.info("Found Notification id to be deleted: {} for {}"\
-                            .format(notification_id, alarm_name))
+                    logger.info("Found Notification id to be deleted: {} for {}"
+                                .format(notification_id, alarm_name))
                     return notification_id
 
-            self.logger.warning("Notification id to be deleted not found for {}"\
-                            .format(alarm_name))
+            logger.warning("Notification id to be deleted not found for {}"
+                           .format(alarm_name))
             return None
 
     def delete_alarm_defination(self, alarm_id):
         """Delete created Alarm definition
         """
         api_url = '/suite-api/api/alertdefinitions/'
-        headers = {'Accept':'application/json'}
+        headers = {'Accept': 'application/json'}
         resp = requests.delete(self.vrops_site + api_url + alarm_id,
-                            auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                               auth=(self.vrops_user, self.vrops_password),
+                               verify=False, headers=headers)
         if resp.status_code is not 204:
-            self.logger.warning("Failed to delete alarm definition {}".format(alarm_id))
+            logger.warning("Failed to delete alarm definition {}".format(alarm_id))
             return None
         else:
-            self.logger.info("Deleted alarm definition {}".format(alarm_id))
+            logger.info("Deleted alarm definition {}".format(alarm_id))
             return alarm_id
 
     def delete_symptom_definition(self, symptom_id):
         """Delete symptom definition
         """
         api_url = '/suite-api/api/symptomdefinitions/'
-        headers = {'Accept':'application/json'}
+        headers = {'Accept': 'application/json'}
         resp = requests.delete(self.vrops_site + api_url + symptom_id,
-                            auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                               auth=(self.vrops_user, self.vrops_password),
+                               verify=False, headers=headers)
         if resp.status_code is not 204:
-            self.logger.warning("Failed to delete symptom definition {}".format(symptom_id))
+            logger.warning("Failed to delete symptom definition {}".format(symptom_id))
             return None
         else:
-            self.logger.info("Deleted symptom definition {}".format(symptom_id))
+            logger.info("Deleted symptom definition {}".format(symptom_id))
             return symptom_id
 
-
     def verify_metric_support(self, metric_info):
         """Verify, if Metric is supported by vROPs plugin, verify metric unit & return status
             Returns:
@@ -1180,108 +1175,107 @@ class MonPlugin():
         """
         status = False
         if 'metric_name' not in metric_info:
-            self.logger.debug("Metric name not provided: {}".format(metric_info))
+            logger.debug("Metric name not provided: {}".format(metric_info))
             return status
         metric_key_params = self.get_default_Params(metric_info['metric_name'].lower())
         if not metric_key_params:
-            self.logger.warning("Metric not supported: {}".format(metric_info['metric_name']))
+            logger.warning("Metric not supported: {}".format(metric_info['metric_name']))
             return status
         else:
-            #If Metric is supported, verify optional metric unit & return status
+            # If Metric is supported, verify optional metric unit & return status
             if 'metric_unit' in metric_info:
                 if metric_key_params.get('unit') == metric_info['metric_unit']:
-                    self.logger.info("Metric is supported with unit: {}".format(metric_info['metric_name']))
+                    logger.info("Metric is supported with unit: {}".format(metric_info['metric_name']))
                     status = True
                 else:
-                    self.logger.debug("Metric supported but there is unit mismatch for: {}."\
-                                    "Supported unit: {}"\
-                                    .format(metric_info['metric_name'],metric_key_params['unit']))
+                    logger.debug("Metric supported but there is unit mismatch for: {}."
+                                 "Supported unit: {}"
+                                 .format(metric_info['metric_name'], metric_key_params['unit']))
                     status = True
         return status
 
     def get_triggered_alarms_list(self, list_alarm_input):
         """Get list of triggered alarms on a resource based on alarm input request.
         """
-        #TO Do - Need to add filtering of alarms based on Severity & alarm name
+        # TO Do - Need to add filtering of alarms based on Severity & alarm name
 
         triggered_alarms_list = []
         if list_alarm_input.get('resource_uuid') is None:
-            self.logger.warning("Resource UUID is required to get triggered alarms list")
+            logger.warning("Resource UUID is required to get triggered alarms list")
             return triggered_alarms_list
 
-        #1)Find vROPs resource ID using RO resource UUID
+        # 1)Find vROPs resource ID using RO resource UUID
         vrops_resource_id = self.get_vrops_resourceid_from_ro_uuid(list_alarm_input['resource_uuid'])
         if vrops_resource_id is None:
             return triggered_alarms_list
 
-        #2)Get triggered alarms on particular resource
-        triggered_alarms_list = self.get_triggered_alarms_on_resource(list_alarm_input['resource_uuid'], vrops_resource_id)
+        # 2)Get triggered alarms on particular resource
+        triggered_alarms_list = self.get_triggered_alarms_on_resource(list_alarm_input['resource_uuid'],
+                                                                      vrops_resource_id)
         return triggered_alarms_list
 
     def get_vrops_resourceid_from_ro_uuid(self, ro_resource_uuid):
         """Fetch vROPs resource ID using resource UUID from RO/SO
         """
-        #1) Find vm_moref_id from vApp uuid in vCD
+        # 1) Find vm_moref_id from vApp uuid in vCD
         vm_moref_id = self.get_vm_moref_id(ro_resource_uuid)
         if vm_moref_id is None:
-            self.logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(ro_resource_uuid))
+            logger.warning("Failed to find vm morefid for vApp in vCD: {}".format(ro_resource_uuid))
             return None
 
-        #2) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
+        # 2) Based on vm_moref_id, find VM's corresponding resource_id in vROPs to set notification
         vrops_resource_id = self.get_vm_resource_id(vm_moref_id)
         if vrops_resource_id is None:
-            self.logger.warning("Failed to find resource in vROPs: {}".format(ro_resource_uuid))
+            logger.warning("Failed to find resource in vROPs: {}".format(ro_resource_uuid))
             return None
         return vrops_resource_id
 
-
     def get_triggered_alarms_on_resource(self, ro_resource_uuid, vrops_resource_id):
         """Get triggered alarms on particular resource & return list of dictionary of alarms
         """
         resource_alarms = []
         api_url = '/suite-api/api/alerts?resourceId='
-        headers = {'Accept':'application/json'}
+        headers = {'Accept': 'application/json'}
         resp = requests.get(self.vrops_site + api_url + vrops_resource_id,
                             auth=(self.vrops_user, self.vrops_password),
-                            verify = False, headers = headers)
+                            verify=False, headers=headers)
 
         if resp.status_code is not 200:
-            self.logger.warning("Failed to get triggered alarms for {}"\
-                    .format(ro_resource_uuid))
+            logger.warning("Failed to get triggered alarms for {}"
+                           .format(ro_resource_uuid))
             return None
 
         all_alerts = json.loads(resp.content)
         if 'alerts' in all_alerts:
             if not all_alerts['alerts']:
-                self.logger.info("No alarms present on resource {}".format(ro_resource_uuid))
+                logger.info("No alarms present on resource {}".format(ro_resource_uuid))
                 return resource_alarms
             all_alerts_list = all_alerts['alerts']
             for alarm in all_alerts_list:
-                #self.logger.info("Triggered Alarm {}".format(alarm))
-                if alarm['alertDefinitionName'] is not None and\
-                    len(alarm['alertDefinitionName'].split('-', 1)) == 2:
-                        if alarm['alertDefinitionName'].split('-', 1)[1] == ro_resource_uuid:
-                            alarm_instance = {}
-                            alarm_instance['alarm_uuid'] = alarm['alertDefinitionId'].split('-', 1)[1]
-                            alarm_instance['resource_uuid'] = ro_resource_uuid
-                            alarm_instance['alarm_instance_uuid'] = alarm['alertId']
-                            alarm_instance['vim_type'] = 'VMware'
-                            #find severity of alarm
-                            severity = None
-                            for key,value in six.iteritems(severity_mano2vrops):
-                                if value == alarm['alertLevel']:
-                                    severity = key
-                            if severity is None:
-                                severity = 'INDETERMINATE'
-                            alarm_instance['severity'] = severity
-                            alarm_instance['status'] = alarm['status']
-                            alarm_instance['start_date'] = self.convert_date_time(alarm['startTimeUTC'])
-                            alarm_instance['update_date'] = self.convert_date_time(alarm['updateTimeUTC'])
-                            alarm_instance['cancel_date'] = self.convert_date_time(alarm['cancelTimeUTC'])
-                            self.logger.info("Triggered Alarm on resource {}".format(alarm_instance))
-                            resource_alarms.append(alarm_instance)
+                # logger.info("Triggered Alarm {}".format(alarm))
+                if alarm['alertDefinitionName'] is not None and \
+                        len(alarm['alertDefinitionName'].split('-', 1)) == 2:
+                    if alarm['alertDefinitionName'].split('-', 1)[1] == ro_resource_uuid:
+                        alarm_instance = {'alarm_uuid': alarm['alertDefinitionId'].split('-', 1)[1],
+                                          'resource_uuid': ro_resource_uuid,
+                                          'alarm_instance_uuid': alarm['alertId'],
+                                          'vim_type': 'VMware'}
+                        # find severity of alarm
+                        severity = None
+                        for key, value in six.iteritems(severity_mano2vrops):
+                            if value == alarm['alertLevel']:
+                                severity = key
+                        if severity is None:
+                            severity = 'INDETERMINATE'
+                        alarm_instance['severity'] = severity
+                        alarm_instance['status'] = alarm['status']
+                        alarm_instance['start_date'] = self.convert_date_time(alarm['startTimeUTC'])
+                        alarm_instance['update_date'] = self.convert_date_time(alarm['updateTimeUTC'])
+                        alarm_instance['cancel_date'] = self.convert_date_time(alarm['cancelTimeUTC'])
+                        logger.info("Triggered Alarm on resource {}".format(alarm_instance))
+                        resource_alarms.append(alarm_instance)
         if not resource_alarms:
-            self.logger.info("No alarms present on resource {}".format(ro_resource_uuid))
+            logger.info("No alarms present on resource {}".format(ro_resource_uuid))
         return resource_alarms
 
     def convert_date_time(self, date_time):
@@ -1289,7 +1283,6 @@ class MonPlugin():
         """
         date_time_formatted = '0000-00-00T00:00:00'
         if date_time != 0:
-            complete_datetime = datetime.datetime.fromtimestamp(date_time/1000.0, tz=pytz.utc).isoformat('T')
-            date_time_formatted = complete_datetime.split('.',1)[0]
+            complete_datetime = datetime.datetime.fromtimestamp(date_time / 1000.0, tz=pytz.utc).isoformat('T')
+            date_time_formatted = complete_datetime.split('.', 1)[0]
         return date_time_formatted
-
index 130c4bd..ce9e1f8 100644 (file)
@@ -31,139 +31,112 @@ import logging
 import os
 import sys
 import traceback
-#Core producer
+from io import UnsupportedOperation
+
 import six
 
+from osm_mon.core.settings import Config
 from osm_mon.plugins.vRealiseOps.mon_plugin_vrops import MonPlugin
 
 sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', '..', '..'))
-from osm_mon.core.message_bus.producer import KafkaProducer
 
 from osm_mon.core.auth import AuthManager
 
-#from core.message_bus.producer import KafkaProducer
 from xml.etree import ElementTree as XmlElementTree
 
-
 schema_version = "1.0"
 req_config_params = ('vrops_site', 'vrops_user', 'vrops_password',
-                    'vcloud-site','admin_username','admin_password',
-                    'vcenter_ip','vcenter_port','vcenter_user','vcenter_password',
-                    'vim_tenant_name','orgname')
+                     'vcloud-site', 'admin_username', 'admin_password',
+                     'vcenter_ip', 'vcenter_port', 'vcenter_user', 'vcenter_password',
+                     'vim_tenant_name', 'orgname')
 MODULE_DIR = os.path.dirname(__file__)
 CONFIG_FILE_NAME = 'vrops_config.xml'
 CONFIG_FILE_PATH = os.path.join(MODULE_DIR, CONFIG_FILE_NAME)
 
-def set_logger():
-    """Set Logger
-    """
-    BASE_DIR = os.path.dirname(os.path.dirname(__file__))
-    logger = logging.getLogger()
-    formatter = logging.Formatter('%(asctime)s - %(name)s - %(levelname)s - %(message)s')
-    handler = logging.FileHandler(os.path.join(BASE_DIR,"mon_vrops_log.log"))
-    handler.setFormatter(formatter)
-    logger.addHandler(handler)
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
+
+logger = logging.getLogger(__name__)
 
 
-class PluginReceiver():
+class PluginReceiver:
     """MON Plugin receiver receiving request messages & responding using producer for vROPs
     telemetry plugin
     """
+
     def __init__(self):
         """Constructor of PluginReceiver
         """
+        self._cfg = Config.instance()
+
+    def handle_alarm_requests(self, key: str, values: dict, vim_uuid: str):
+        values['vim_uuid'] = vim_uuid
+        if key == "create_alarm_request":
+            config_alarm_info = values
+            alarm_uuid = self.create_alarm(config_alarm_info)
+            logger.info("Alarm created with alarm uuid: {}".format(alarm_uuid))
+            # Publish message using producer
+            return self.publish_create_alarm_status(alarm_uuid, config_alarm_info)
+        elif key == "update_alarm_request":
+            update_alarm_info = values
+            alarm_uuid = self.update_alarm(update_alarm_info)
+            logger.info("Alarm definition updated : alarm uuid: {}".format(alarm_uuid))
+            # Publish message using producer
+            return self.publish_update_alarm_status(alarm_uuid, update_alarm_info)
+        elif key == "delete_alarm_request":
+            delete_alarm_info = values
+            alarm_uuid = self.delete_alarm(delete_alarm_info)
+            logger.info("Alarm definition deleted : alarm uuid: {}".format(alarm_uuid))
+            # Publish message using producer
+            return self.publish_delete_alarm_status(alarm_uuid, delete_alarm_info)
+        elif key == "list_alarm_request":
+            request_input = values
+            triggered_alarm_list = self.list_alarms(request_input)
+            # Publish message using producer
+            return self.publish_list_alarm_response(triggered_alarm_list, request_input)
+        else:
+            raise UnsupportedOperation("Unknown key, no action will be performed")
+
+    def handle_metric_requests(self, key: str, values: dict, vim_uuid: str):
+        values['vim_uuid'] = vim_uuid
+        if key == "read_metric_data_request":
+            metric_request_info = values
+            access_config = self.get_vim_access_config(metric_request_info['vim_uuid'])
+            mon_plugin_obj = MonPlugin(access_config)
+            metrics_data = mon_plugin_obj.get_metrics_data(metric_request_info)
+            logger.info("Collected Metrics Data: {}".format(metrics_data))
+            # Publish message using producer
+            return self.publish_metrics_data_status(metrics_data)
+        elif key == "create_metric_request":
+            metric_info = values
+            metric_status = self.verify_metric(metric_info)
+            # Publish message using producer
+            return self.publish_create_metric_response(metric_info, metric_status)
+        elif key == "update_metric_request":
+            metric_info = values
+            metric_status = self.verify_metric(metric_info)
+            # Publish message using producer
+            return self.publish_update_metric_response(metric_info, metric_status)
+        elif key == "delete_metric_request":
+            metric_info = values
+            # Deleting Metric Data is not allowed. Publish status as False
+            logger.warning("Deleting Metric is not allowed by VMware vROPs plugin: {}"
+                           .format(metric_info['metric_name']))
+            # Publish message using producer
+            return self.publish_delete_metric_response(metric_info)
 
-
-        self.logger = logging.getLogger('PluginReceiver')
-        self.logger.setLevel(logging.DEBUG)
-        set_logger()
-
-        #Core producer
-        self.producer_alarms = KafkaProducer('alarm_response')
-        self.producer_metrics = KafkaProducer('metric_response')
-        self.producer_access_credentials = KafkaProducer('vim_access_credentials_response')
-
-
-    def consume(self, message, vim_uuid):
-        """Consume the message, act on it & respond
-        """
-        try:
-            self.logger.info("Message received for VIM: {} :\nMessage Topic={}:{}:{}:\n"\
-                             "Message Key={}\nMessage Value={}"\
-                             .format(vim_uuid, message.topic, message.partition, message.offset,\
-                                     message.key, message.value))
-            message_values = json.loads(message.value)
-            #Adding vim_uuid to message
-            message_values['vim_uuid'] = vim_uuid
-            self.logger.info("Action required for: {}".format(message.topic))
-            if message.topic == 'alarm_request':
-                if message.key == "create_alarm_request":
-                    config_alarm_info = message_values
-                    alarm_uuid = self.create_alarm(config_alarm_info)
-                    self.logger.info("Alarm created with alarm uuid: {}".format(alarm_uuid))
-                    #Publish message using producer
-                    self.publish_create_alarm_status(alarm_uuid, config_alarm_info)
-                elif message.key == "update_alarm_request":
-                    update_alarm_info = message_values
-                    alarm_uuid = self.update_alarm(update_alarm_info)
-                    self.logger.info("Alarm definition updated : alarm uuid: {}".format(alarm_uuid))
-                    #Publish message using producer
-                    self.publish_update_alarm_status(alarm_uuid, update_alarm_info)
-                elif message.key == "delete_alarm_request":
-                    delete_alarm_info = message_values
-                    alarm_uuid = self.delete_alarm(delete_alarm_info)
-                    self.logger.info("Alarm definition deleted : alarm uuid: {}".format(alarm_uuid))
-                    #Publish message using producer
-                    self.publish_delete_alarm_status(alarm_uuid, delete_alarm_info)
-                elif message.key == "list_alarm_request":
-                    request_input = message_values
-                    triggered_alarm_list = self.list_alarms(request_input)
-                    #Publish message using producer
-                    self.publish_list_alarm_response(triggered_alarm_list, request_input)
-            elif message.topic == 'metric_request':
-                if message.key == "read_metric_data_request":
-                    metric_request_info = message_values
-                    access_config = None
-                    access_config = self.get_vim_access_config(metric_request_info['vim_uuid'])
-                    mon_plugin_obj = MonPlugin(access_config)
-                    metrics_data = mon_plugin_obj.get_metrics_data(metric_request_info)
-                    self.logger.info("Collected Metrics Data: {}".format(metrics_data))
-                    #Publish message using producer
-                    self.publish_metrics_data_status(metrics_data)
-                elif message.key == "create_metric_request":
-                    metric_info = message_values
-                    metric_status = self.verify_metric(metric_info)
-                    #Publish message using producer
-                    self.publish_create_metric_response(metric_info, metric_status)
-                elif message.key == "update_metric_request":
-                    metric_info = message_values
-                    metric_status = self.verify_metric(metric_info)
-                    #Publish message using producer
-                    self.publish_update_metric_response(metric_info, metric_status)
-                elif message.key == "delete_metric_request":
-                    metric_info = message_values
-                    #Deleting Metric Data is not allowed. Publish status as False
-                    self.logger.warning("Deleting Metric is not allowed by VMware vROPs plugin: {}"\
-                                        .format(metric_info['metric_name']))
-                    #Publish message using producer
-                    self.publish_delete_metric_response(metric_info)
-            elif message.topic == 'access_credentials':
-                if message.key == "vim_access_credentials":
-                    access_info = message_values
-                    access_update_status = self.update_access_credentials(access_info['access_config'])
-                    self.publish_access_update_response(access_update_status, access_info)
-
-        except:
-            self.logger.error("Exception in vROPs plugin receiver: {}".format(traceback.format_exc()))
-
+        else:
+            raise UnsupportedOperation("Unknown key, no action will be performed")
 
     def create_alarm(self, config_alarm_info):
         """Create alarm using vROPs plugin
         """
-        access_config = None
         access_config = self.get_vim_access_config(config_alarm_info['vim_uuid'])
         mon_plugin = MonPlugin(access_config)
-        plugin_uuid = mon_plugin.configure_rest_plugin()
+        mon_plugin.configure_rest_plugin()
         alarm_uuid = mon_plugin.configure_alarm(config_alarm_info['alarm_create_request'])
         return alarm_uuid
 
@@ -172,24 +145,23 @@ class PluginReceiver():
         """
         topic = 'alarm_response'
         msg_key = 'create_alarm_response'
-        response_msg = {"schema_version":schema_version,
-                         "schema_type":"create_alarm_response",
-                         "vim_uuid":config_alarm_info["vim_uuid"],
-                         "alarm_create_response":
-                            {"correlation_id":config_alarm_info["alarm_create_request"]["correlation_id"],
-                             "alarm_uuid":alarm_uuid,
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "create_alarm_response",
+                        "vim_uuid": config_alarm_info["vim_uuid"],
+                        "alarm_create_response":
+                            {"correlation_id": config_alarm_info["alarm_create_request"]["correlation_id"],
+                             "alarm_uuid": alarm_uuid,
                              "status": True if alarm_uuid else False
-                            }
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_alarms.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+                             }
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def update_alarm(self, update_alarm_info):
         """Update already created alarm
         """
-        access_config = None
         access_config = self.get_vim_access_config(update_alarm_info['vim_uuid'])
         mon_plugin = MonPlugin(access_config)
         alarm_uuid = mon_plugin.update_alarm_configuration(update_alarm_info['alarm_update_request'])
@@ -200,25 +172,24 @@ class PluginReceiver():
         """
         topic = 'alarm_response'
         msg_key = 'update_alarm_response'
-        response_msg = {"schema_version":schema_version,
-                         "schema_type":"update_alarm_response",
-                         "vim_uuid":update_alarm_info["vim_uuid"],
-                         "alarm_update_response":
-                            {"correlation_id":update_alarm_info["alarm_update_request"]["correlation_id"],
-                             "alarm_uuid":update_alarm_info["alarm_update_request"]["alarm_uuid"] \
-                             if update_alarm_info["alarm_update_request"].get('alarm_uuid') is not None else None,
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "update_alarm_response",
+                        "vim_uuid": update_alarm_info["vim_uuid"],
+                        "alarm_update_response":
+                            {"correlation_id": update_alarm_info["alarm_update_request"]["correlation_id"],
+                             "alarm_uuid": update_alarm_info["alarm_update_request"]["alarm_uuid"] \
+                                 if update_alarm_info["alarm_update_request"].get('alarm_uuid') is not None else None,
                              "status": True if alarm_uuid else False
-                            }
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_alarms.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+                             }
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def delete_alarm(self, delete_alarm_info):
         """Delete alarm configuration
         """
-        access_config = None
         access_config = self.get_vim_access_config(delete_alarm_info['vim_uuid'])
         mon_plugin = MonPlugin(access_config)
         alarm_uuid = mon_plugin.delete_alarm_configuration(delete_alarm_info['alarm_delete_request'])
@@ -229,36 +200,33 @@ class PluginReceiver():
         """
         topic = 'alarm_response'
         msg_key = 'delete_alarm_response'
-        response_msg = {"schema_version":schema_version,
-                         "schema_type":"delete_alarm_response",
-                         "vim_uuid":delete_alarm_info['vim_uuid'],
-                         "alarm_deletion_response":
-                            {"correlation_id":delete_alarm_info["alarm_delete_request"]["correlation_id"],
-                             "alarm_uuid":delete_alarm_info["alarm_delete_request"]["alarm_uuid"],
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "delete_alarm_response",
+                        "vim_uuid": delete_alarm_info['vim_uuid'],
+                        "alarm_deletion_response":
+                            {"correlation_id": delete_alarm_info["alarm_delete_request"]["correlation_id"],
+                             "alarm_uuid": delete_alarm_info["alarm_delete_request"]["alarm_uuid"],
                              "status": True if alarm_uuid else False
-                            }
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_alarms.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+                             }
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
 
+        return response_msg
 
     def publish_metrics_data_status(self, metrics_data):
         """Publish the requested metric data using producer
         """
         topic = 'metric_response'
         msg_key = 'read_metric_data_response'
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, metrics_data))
-        #Core producer
-        self.producer_metrics.publish(key=msg_key, value=json.dumps(metrics_data), topic=topic)
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, metrics_data))
 
+        return metrics_data
 
     def verify_metric(self, metric_info):
         """Verify if metric is supported or not
         """
-        access_config = None
         access_config = self.get_vim_access_config(metric_info['vim_uuid'])
         mon_plugin = MonPlugin(access_config)
         if 'metric_create_request' in metric_info:
@@ -272,44 +240,44 @@ class PluginReceiver():
         """
         topic = 'metric_response'
         msg_key = 'create_metric_response'
-        response_msg = {"schema_version":schema_version,
-                         "schema_type":"create_metric_response",
-                         ##"vim_uuid":metric_info['vim_uuid'],
-                         ##"correlation_id":metric_info['correlation_id'],
-                         "metric_create_response":
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "create_metric_response",
+                        ##"vim_uuid":metric_info['vim_uuid'],
+                        ##"correlation_id":metric_info['correlation_id'],
+                        "metric_create_response":
                             {
-                             ##"metric_uuid":'0',
-                             ##"resource_uuid":metric_info['metric_create']['resource_uuid'],
-                             ##"vim_uuid":metric_info['vim_uuid'], #May be required. TODO - Confirm
-                             "correlation_id":metric_info['correlation_id'],
-                             "status":metric_status
+                                ##"metric_uuid":'0',
+                                ##"resource_uuid":metric_info['metric_create']['resource_uuid'],
+                                ##"vim_uuid":metric_info['vim_uuid'], #May be required. TODO - Confirm
+                                "correlation_id": metric_info['correlation_id'],
+                                "status": metric_status
                             }
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_metrics.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def publish_update_metric_response(self, metric_info, metric_status):
         """Publish update metric response
         """
         topic = 'metric_response'
         msg_key = 'update_metric_response'
-        response_msg = {"schema_version":schema_version,
-                        "schema_type":"metric_update_response",
-                        "vim_uuid":metric_info['vim_uuid'],
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "metric_update_response",
+                        "vim_uuid": metric_info['vim_uuid'],
                         "metric_update_response":
                             {
-                             "metric_uuid":'0',
-                             "correlation_id":metric_info['correlation_id'],
-                             "resource_uuid":metric_info['metric_create']['resource_uuid'],
-                             "status":metric_status
+                                "metric_uuid": '0',
+                                "correlation_id": metric_info['correlation_id'],
+                                "resource_uuid": metric_info['metric_create']['resource_uuid'],
+                                "status": metric_status
                             }
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_metrics.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def publish_delete_metric_response(self, metric_info):
         """Publish delete metric response
@@ -321,64 +289,60 @@ class PluginReceiver():
         else:
             tenant_uuid = None
 
-        response_msg = {"schema_version":schema_version,
-                        "schema_type":"delete_metric_response",
-                        "vim_uuid":metric_info['vim_uuid'],
-                        "correlation_id":metric_info['correlation_id'],
-                        "metric_name":metric_info['metric_name'],
-                        "metric_uuid":'0',
-                        "resource_uuid":metric_info['resource_uuid'],
-                        "tenant_uuid":tenant_uuid,
-                        "status":False
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_metrics.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "delete_metric_response",
+                        "vim_uuid": metric_info['vim_uuid'],
+                        "correlation_id": metric_info['correlation_id'],
+                        "metric_name": metric_info['metric_name'],
+                        "metric_uuid": '0',
+                        "resource_uuid": metric_info['resource_uuid'],
+                        "tenant_uuid": tenant_uuid,
+                        "status": False
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def list_alarms(self, list_alarm_input):
         """Collect list of triggered alarms based on input
         """
-        access_config = None
         access_config = self.get_vim_access_config(list_alarm_input['vim_uuid'])
         mon_plugin = MonPlugin(access_config)
         triggered_alarms = mon_plugin.get_triggered_alarms_list(list_alarm_input['alarm_list_request'])
         return triggered_alarms
 
-
     def publish_list_alarm_response(self, triggered_alarm_list, list_alarm_input):
         """Publish list of triggered alarms
         """
         topic = 'alarm_response'
         msg_key = 'list_alarm_response'
-        response_msg = {"schema_version":schema_version,
-                        "schema_type":"list_alarm_response",
-                        "vim_type":"VMware",
-                        "vim_uuid":list_alarm_input['vim_uuid'],
-                        "correlation_id":list_alarm_input['alarm_list_request']['correlation_id'],
-                        "list_alarm_response":triggered_alarm_list
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core producer
-        self.producer_alarms.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
-
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "list_alarm_response",
+                        "vim_type": "VMware",
+                        "vim_uuid": list_alarm_input['vim_uuid'],
+                        "correlation_id": list_alarm_input['alarm_list_request']['correlation_id'],
+                        "list_alarm_response": triggered_alarm_list
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"
+                    .format(topic, msg_key, response_msg))
+
+        return response_msg
 
     def update_access_credentials(self, access_info):
         """Verify if all the required access config params are provided and
            updates access config in default vrops config file
         """
         update_status = False
-        wr_status = False
-        #Check if all the required config params are passed in request
-        if not all (keys in access_info for keys in req_config_params):
-            self.logger.debug("All required Access Config Parameters not provided")
-            self.logger.debug("List of required Access Config Parameters: {}".format(req_config_params))
-            self.logger.debug("List of given Access Config Parameters: {}".format(access_info))
+        # Check if all the required config params are passed in request
+        if not all(keys in access_info for keys in req_config_params):
+            logger.debug("All required Access Config Parameters not provided")
+            logger.debug("List of required Access Config Parameters: {}".format(req_config_params))
+            logger.debug("List of given Access Config Parameters: {}".format(access_info))
             return update_status
 
         wr_status = self.write_access_config(access_info)
-        return wr_status    #True/False
+        return wr_status  True/False
 
     def write_access_config(self, access_info):
         """Write access configuration to vROPs config file.
@@ -391,39 +355,36 @@ class PluginReceiver():
             for config in root:
                 if config.tag == 'Access_Config':
                     for param in config:
-                        for key,val in six.iteritems(access_info):
+                        for key, val in six.iteritems(access_info):
                             if param.tag == key:
-                                #print param.tag, val
+                                # print param.tag, val
                                 param.text = val
 
             tree.write(CONFIG_FILE_PATH)
             wr_status = True
         except Exception as exp:
-            self.logger.warning("Failed to update Access Config Parameters: {}".format(exp))
+            logger.warning("Failed to update Access Config Parameters: {}".format(exp))
 
         return wr_status
 
-
     def publish_access_update_response(self, access_update_status, access_info_req):
         """Publish access update response
         """
         topic = 'access_credentials'
         msg_key = 'vim_access_credentials_response'
-        response_msg = {"schema_version":schema_version,
-                        "schema_type":"vim_access_credentials_response",
-                        "correlation_id":access_info_req['access_config']['correlation_id'],
-                        "status":access_update_status
-                       }
-        self.logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}"\
-                .format(topic, msg_key, response_msg))
-        #Core Add producer
-        self.producer_access_credentials.publish(key=msg_key, value=json.dumps(response_msg), topic=topic)
-
+        response_msg = {"schema_version": schema_version,
+                        "schema_type": "vim_access_credentials_response",
+                        "correlation_id": access_info_req['access_config']['correlation_id'],
+                        "status": access_update_status
+                        }
+        logger.info("Publishing response:\nTopic={}\nKey={}\nValue={}" \
+                    .format(topic, msg_key, response_msg))
+        # Core Add producer
+        return response_msg
 
     def get_vim_access_config(self, vim_uuid):
         """Get VIM access configuration & account details from path: VIM_ACCOUNTS_FILE_PATH
         """
-        vim_account_details = None
         vim_account = {}
         auth_manager = AuthManager()
         vim_account_details = auth_manager.get_credentials(vim_uuid)
@@ -458,9 +419,10 @@ class PluginReceiver():
                 if vim_config['orgname'] is not None:
                     vim_account['orgname'] = vim_config['orgname']
         except Exception as exp:
-            self.logger.error("VIM account details not sufficient: {}".format(exp))
+            logger.error("VIM account details not sufficient: {}".format(exp))
         return vim_account
 
+
 """
 def main():
     #log.basicConfig(filename='mon_vrops_log.log',level=log.DEBUG)
@@ -471,4 +433,3 @@ def main():
 if __name__ == "__main__":
     main()
 """
-
index fd51449..d7cf33c 100755 (executable)
  Webservice for vRealize Operations (vROPs) to post/notify alarms details.
 
 """
+from osm_mon.core.settings import Config
+
 __author__ = "Arpita Kate"
 __date__ = "$15-Sept-2017 16:09:29$"
 __version__ = '0.1'
 
-
-from bottle import (ServerAdapter, route, run, server_names, redirect, default_app,
-                     request, response, template, debug, TEMPLATE_PATH , static_file)
-from socket import getfqdn
-from datetime import datetime
-from xml.etree import ElementTree as ET
+import json
 import logging
 import os
-import json
 import sys
+from datetime import datetime
+from socket import getfqdn
+
 import requests
-sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', '..', '..','..'))
-from osm_mon.core.message_bus.producer import KafkaProducer
-#from core.message_bus.producer import KafkaProducer
+from bottle import (ServerAdapter, route, run, server_names, default_app,
+                    request, response)
+
+sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', '..', '..', '..'))
+from osm_mon.core.message_bus.producer import Producer
+
+# from core.message_bus.producer import KafkaProducer
 
 sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', '..', '..'))
 from osm_mon.core.database import DatabaseManager
@@ -55,34 +58,35 @@ except ImportError:
     from cherrypy.wsgiserver import CherryPyWSGIServer as WSGIServer
     from cherrypy.wsgiserver.ssl_pyopenssl import pyOpenSSLAdapter
 
-#Set Constants
+# Set Constants
 BASE_DIR = os.path.dirname(os.path.dirname(__file__))
 CERT_DIR = os.path.join(BASE_DIR, "SSL_certificate")
 certificate_name = getfqdn() + ".cert"
 key_name = getfqdn() + ".key"
 CERTIFICATE = os.path.join(CERT_DIR, certificate_name)
 KEY = os.path.join(CERT_DIR, key_name)
-#CERTIFICATE = os.path.join(CERT_DIR, "www.vrops_webservice.com.cert")
-#KEY = os.path.join(CERT_DIR, "www.vrops_webservice.com.key")
+# CERTIFICATE = os.path.join(CERT_DIR, "www.vrops_webservice.com.cert")
+# KEY = os.path.join(CERT_DIR, "www.vrops_webservice.com.key")
 CONFIG_FILE = os.path.join(BASE_DIR, '../vrops_config.xml')
-#Severity Mapping from vROPs to OSM
+# Severity Mapping from vROPs to OSM
 VROPS_SEVERITY_TO_OSM_MAPPING = {
-                "ALERT_CRITICALITY_LEVEL_CRITICAL":"CRITICAL",
-                "ALERT_CRITICALITY_LEVEL_WARNING":"WARNING",
-                "ALERT_CRITICALITY_LEVEL_IMMEDIATE":"MAJOR",
-                "ALERT_CRITICALITY_LEVEL_INFO":"INDETERMINATE",
-                "ALERT_CRITICALITY_LEVEL_AUTO":"INDETERMINATE",
-                "ALERT_CRITICALITY_LEVEL_UNKNOWN":"INDETERMINATE",
-                "ALERT_CRITICALITY_LEVEL_NONE":"INDETERMINATE"
-            }
-
-#Set logger
-logger = logging.getLogger('vROPs_Webservice')
-formatter = logging.Formatter('%(asctime)s - %(name)s - %(levelname)s - %(message)s')
-hdlr = logging.FileHandler(os.path.join(BASE_DIR,"vrops_webservice.log"))
-hdlr.setFormatter(formatter)
-logger.addHandler(hdlr)
-logger.setLevel(logging.DEBUG)
+    "ALERT_CRITICALITY_LEVEL_CRITICAL": "CRITICAL",
+    "ALERT_CRITICALITY_LEVEL_WARNING": "WARNING",
+    "ALERT_CRITICALITY_LEVEL_IMMEDIATE": "MAJOR",
+    "ALERT_CRITICALITY_LEVEL_INFO": "INDETERMINATE",
+    "ALERT_CRITICALITY_LEVEL_AUTO": "INDETERMINATE",
+    "ALERT_CRITICALITY_LEVEL_UNKNOWN": "INDETERMINATE",
+    "ALERT_CRITICALITY_LEVEL_NONE": "INDETERMINATE"
+}
+
+# Set logger
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
+
+logger = logging.getLogger(__name__)
 
 
 def format_datetime(str_date):
@@ -96,14 +100,15 @@ def format_datetime(str_date):
     date_fromat = "%Y-%m-%dT%H:%M:%S"
     formated_datetime = None
     try:
-        datetime_obj = datetime.fromtimestamp(float(str_date)/1000.)
+        datetime_obj = datetime.fromtimestamp(float(str_date) / 1000.)
         formated_datetime = datetime_obj.strftime(date_fromat)
     except Exception as exp:
         logger.error('Exception: {} occured while converting date {} into format {}'.format(
-                           exp,str_date, date_fromat))
+            exp, str_date, date_fromat))
 
     return formated_datetime
 
+
 def get_alarm_config(alarm_name):
     """
         Method to get configuration parameters
@@ -114,19 +119,20 @@ def get_alarm_config(alarm_name):
     """
     vim_account = {}
     database_manager = DatabaseManager()
-    vim_account_details = database_manager.get_credentials_for_alarm_name(alarm_name,'VMware')
+    alarm = database_manager.get_alarm(alarm_name, 'VMware')
+    vim_account_details = alarm.credentials
 
     try:
         if vim_account_details is not None:
             vim_account['name'] = vim_account_details.name
             vim_account['vim_tenant_name'] = vim_account_details.tenant_name
             vim_account['vim_type'] = vim_account_details.type
-            vim_account['vim_uuid'] = vim_account_details._id
+            vim_account['vim_uuid'] = vim_account_details.uuid
             vim_account['vim_url'] = vim_account_details.url
             vim_account['org_user'] = vim_account_details.user
             vim_account['org_password'] = vim_account_details.password
 
-            vim_config = json.loads(vim_account_details.config) 
+            vim_config = json.loads(vim_account_details.config)
             vim_account['tenant_id'] = vim_config['tenant_id']
             vim_account['admin_username'] = vim_config['admin_username']
             vim_account['admin_password'] = vim_config['admin_password']
@@ -159,29 +165,29 @@ def get_alarm_definationID(alarm_instance_uuid, access_config):
                 alarm defination ID
     """
     alarm_definationID = None
-    if alarm_instance_uuid :
+    if alarm_instance_uuid:
         try:
-            #access_config = get_alarm_config()
+            # access_config = get_alarm_config()
             headers = {'Accept': 'application/json'}
-            api_url = '{}/suite-api/api/alerts/{}'\
-                        .format(access_config.get('vrops_site'), alarm_instance_uuid)
+            api_url = '{}/suite-api/api/alerts/{}' \
+                .format(access_config.get('vrops_site'), alarm_instance_uuid)
             api_response = requests.get(api_url,
-                                        auth=(access_config.get('vrops_user'),\
+                                        auth=(access_config.get('vrops_user'),
                                               access_config.get('vrops_password')),
-                                        verify = False,
-                                        headers = headers
+                                        verify=False,
+                                        headers=headers
                                         )
 
-            if  api_response.status_code == 200:
+            if api_response.status_code == 200:
                 data = api_response.json()
                 if data.get("alertDefinitionId") is not None:
                     alarm_definationID = '-'.join(data.get("alertDefinitionId").split('-')[1:])
             else:
-                logger.error("Failed to get alert definition ID for alarm {}"\
+                logger.error("Failed to get alert definition ID for alarm {}"
                              .format(alarm_instance_uuid))
         except Exception as exp:
-            logger.error("Exception occured while getting alert definition ID for alarm : {}"\
-                          .format(exp, alarm_instance_uuid))
+            logger.error("Exception occured while getting alert definition ID for alarm : {}"
+                         .format(exp, alarm_instance_uuid))
 
     return alarm_definationID
 
@@ -195,23 +201,23 @@ def notify_alarm(alarmID):
         Returns:
            response code
     """
-    logger.info("Request:{} from:{} {} {} "\
+    logger.info("Request:{} from:{} {} {} " \
                 .format(request, request.remote_addr, request.method, request.url))
     response.headers['Content-Type'] = 'application/json'
     try:
         postdata = json.loads(request.body.read())
         notify_details = {}
         vim_access_config = get_alarm_config(postdata.get('alertName'))
-        #Parse noditfy data
-        notify_details['vim_uuid'] = vim_access_config.get('vim_uuid') 
-        notify_details['alarm_uuid'] = get_alarm_definationID(postdata.get('alertId'),\
+        # Parse notify data
+        notify_details['vim_uuid'] = vim_access_config.get('vim_uuid')
+        notify_details['alarm_uuid'] = get_alarm_definationID(postdata.get('alertId'),
                                                               vim_access_config)
         notify_details['description'] = postdata.get('info')
         notify_details['alarm_instance_uuid'] = alarmID
         notify_details['resource_uuid'] = '-'.join(postdata.get('alertName').split('-')[1:])
-        notify_details['tenant_uuid'] =  vim_access_config.get('tenant_id')
+        notify_details['tenant_uuid'] = vim_access_config.get('tenant_id')
         notify_details['vim_type'] = "VMware"
-        notify_details['severity'] = VROPS_SEVERITY_TO_OSM_MAPPING.get(postdata.get('criticality'),\
+        notify_details['severity'] = VROPS_SEVERITY_TO_OSM_MAPPING.get(postdata.get('criticality'),
                                                                        'INDETERMINATE')
         notify_details['status'] = postdata.get('status')
         if postdata.get('startDate'):
@@ -224,20 +230,21 @@ def notify_alarm(alarmID):
         alarm_details = {'schema_version': 1.0,
                          'schema_type': "notify_alarm",
                          'notify_details': notify_details
-                        }
+                         }
         alarm_data = json.dumps(alarm_details)
         logger.info("Alarm details: {}".format(alarm_data))
 
-        #Publish Alarm details
-        kafkaMsgProducer = KafkaProducer("alarm_response")
-        kafkaMsgProducer.publish(topic='alarm_response', key='notify_alarm', value=alarm_data)
+        # Publish Alarm details
+        producer = Producer()
+        producer.send(topic='alarm_response', key=notify_alarm, value=alarm_data)
+        producer.flush()
 
-        #return 201 on Success
+        # return 201 on Success
         response.status = 201
 
     except Exception as exp:
         logger.error('Exception: {} occured while notifying alarm {}.'.format(exp, alarmID))
-        #return 500 on Error
+        # return 500 on Error
         response.status = 500
 
     return response
@@ -256,7 +263,7 @@ class SSLWebServer(ServerAdapter):
         server.ssl_adapter = pyOpenSSLAdapter(
             certificate=CERTIFICATE,
             private_key=KEY,
-           # certificate_chain="intermediate_cert.crt"
+            # certificate_chain="intermediate_cert.crt"
         )
 
         try:
@@ -268,8 +275,8 @@ class SSLWebServer(ServerAdapter):
 
 
 if __name__ == "__main__":
-    #Start SSL Web Service
+    # Start SSL Web Service
     logger.info("Start vROPs Web Service")
     app = default_app()
     server_names['sslwebserver'] = SSLWebServer
-    run(app=app,host=getfqdn(), port=8080, server='sslwebserver')
+    run(app=app, host=getfqdn(), port=8080, server='sslwebserver')
index d81308a..cd7731b 100644 (file)
 # For those usages not covered by the Apache License, Version 2.0 please
 # contact: bdiaz@whitestack.com or glavado@whitestack.com
 ##
+import logging
+import sys
+
+from osm_mon.core.settings import Config
+
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
+log = logging.getLogger(__name__)
index 278c8c2..fbec56c 100644 (file)
@@ -33,9 +33,8 @@ from kafka.errors import KafkaError
 
 from osm_mon.core.auth import AuthManager
 from osm_mon.core.database import DatabaseManager, VimCredentials
-from osm_mon.core.message_bus.producer import KafkaProducer as Producer
 from osm_mon.plugins.OpenStack import response
-from osm_mon.plugins.OpenStack.Aodh import alarming
+from osm_mon.plugins.OpenStack.Aodh import alarm_handler
 from osm_mon.plugins.OpenStack.common import Common
 
 log = logging.getLogger(__name__)
@@ -44,7 +43,6 @@ mock_creds = VimCredentials()
 mock_creds.config = '{}'
 
 
-@mock.patch.object(Producer, "publish_alarm_request", mock.Mock())
 @mock.patch.object(DatabaseManager, "save_alarm", mock.Mock())
 @mock.patch.object(Common, "get_auth_token", mock.Mock())
 @mock.patch.object(Common, "get_endpoint", mock.Mock())
@@ -64,7 +62,7 @@ class AlarmIntegrationTest(unittest.TestCase):
         except KafkaError:
             self.skipTest('Kafka server not present.')
         # Set up common and alarming class instances
-        self.alarms = alarming.Alarming()
+        self.alarms = alarm_handler.OpenstackAlarmHandler()
         self.openstack_auth = Common()
 
     def tearDown(self):
@@ -73,8 +71,8 @@ class AlarmIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarming.Alarming, "update_alarm")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(alarm_handler.OpenstackAlarmHandler, "update_alarm")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_update_alarm_req(self, resp, update_alarm, get_creds, perf_req):
         """Test Aodh update alarm request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -93,7 +91,7 @@ class AlarmIntegrationTest(unittest.TestCase):
             if message.key == "update_alarm_request":
                 # Mock a valid alarm update
                 update_alarm.return_value = "alarm_id"
-                self.alarms.alarming(message, 'test_id')
+                self.alarms.handle_message(message, 'test_id')
 
                 # A response message is generated and sent via MON's producer
                 resp.assert_called_with(
@@ -105,8 +103,8 @@ class AlarmIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarming.Alarming, "configure_alarm")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(alarm_handler.OpenstackAlarmHandler, "configure_alarm")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_create_alarm_req(self, resp, config_alarm, get_creds, perf_req):
         """Test Aodh create alarm request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -131,7 +129,7 @@ class AlarmIntegrationTest(unittest.TestCase):
             if message.key == "create_alarm_request":
                 # Mock a valid alarm creation
                 config_alarm.return_value = "alarm_id"
-                self.alarms.alarming(message, 'test_id')
+                self.alarms.handle_message(message, 'test_id')
 
                 # A response message is generated and sent via MON's produce
                 resp.assert_called_with(
@@ -143,8 +141,8 @@ class AlarmIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarming.Alarming, "list_alarms")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(alarm_handler.OpenstackAlarmHandler, "list_alarms")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_list_alarm_req(self, resp, list_alarm, get_creds, perf_req):
         """Test Aodh list alarm request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -162,7 +160,7 @@ class AlarmIntegrationTest(unittest.TestCase):
             if message.key == "list_alarm_request":
                 # Mock an empty list generated by the request
                 list_alarm.return_value = []
-                self.alarms.alarming(message, 'test_id')
+                self.alarms.handle_message(message, 'test_id')
 
                 # Response message is generated
                 resp.assert_called_with(
@@ -174,8 +172,8 @@ class AlarmIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarming.Alarming, "delete_alarm")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(alarm_handler.OpenstackAlarmHandler, "delete_alarm")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_delete_alarm_req(self, resp, del_alarm, get_creds, perf_req):
         """Test Aodh delete alarm request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -191,7 +189,7 @@ class AlarmIntegrationTest(unittest.TestCase):
 
         for message in self.req_consumer:
             if message.key == "delete_alarm_request":
-                self.alarms.alarming(message, 'test_id')
+                self.alarms.handle_message(message, 'test_id')
 
                 # Response message is generated and sent by MON's producer
                 resp.assert_called_with(
@@ -202,7 +200,7 @@ class AlarmIntegrationTest(unittest.TestCase):
         self.fail("No message received in consumer")
 
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarming.Alarming, "update_alarm_state")
+    @mock.patch.object(alarm_handler.OpenstackAlarmHandler, "update_alarm_state")
     def test_ack_alarm_req(self, ack_alarm, get_creds):
         """Test Aodh acknowledge alarm request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -216,7 +214,7 @@ class AlarmIntegrationTest(unittest.TestCase):
 
         for message in self.req_consumer:
             if message.key == "acknowledge_alarm":
-                self.alarms.alarming(message, 'test_id')
+                self.alarms.handle_message(message, 'test_id')
                 ack_alarm.assert_called_with(mock.ANY, mock.ANY, 'alarm_id', True)
                 return
 
index 344ccbd..578c8b1 100644 (file)
@@ -31,7 +31,6 @@ from kafka.errors import KafkaError
 
 from osm_mon.core.auth import AuthManager
 from osm_mon.core.database import VimCredentials
-from osm_mon.core.message_bus.producer import KafkaProducer as Producer
 
 from kafka import KafkaConsumer
 from kafka import KafkaProducer
@@ -40,7 +39,7 @@ import mock
 
 from osm_mon.plugins.OpenStack import response
 
-from osm_mon.plugins.OpenStack.Gnocchi import metrics
+from osm_mon.plugins.OpenStack.Gnocchi import metric_handler
 
 from osm_mon.plugins.OpenStack.common import Common
 
@@ -50,13 +49,12 @@ mock_creds = VimCredentials()
 mock_creds.config = '{}'
 
 
-@mock.patch.object(Producer, "publish_alarm_request", mock.Mock())
 @mock.patch.object(Common, "get_auth_token", mock.Mock())
 @mock.patch.object(Common, "get_endpoint", mock.Mock())
 class MetricIntegrationTest(unittest.TestCase):
     def setUp(self):
         # Set up common and alarming class instances
-        self.metric_req = metrics.Metrics()
+        self.metric_req = metric_handler.OpenstackMetricHandler()
         self.openstack_auth = Common()
 
         try:
@@ -75,8 +73,8 @@ class MetricIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(metrics.Metrics, "configure_metric")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(metric_handler.OpenstackMetricHandler, "configure_metric")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_create_metric_req(self, resp, config_metric, get_creds, perf_req):
         """Test Gnocchi create metric request message from producer."""
         # Set-up message, producer and consumer for tests
@@ -95,7 +93,7 @@ class MetricIntegrationTest(unittest.TestCase):
             if message.key == "create_metric_request":
                 # A valid metric is created
                 config_metric.return_value = "metric_id", "resource_id"
-                self.metric_req.metric_calls(message, 'test_id')
+                self.metric_req.handle_request(message, 'test_id')
 
                 # A response message is generated and sent by MON's producer
                 resp.assert_called_with(
@@ -107,8 +105,8 @@ class MetricIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(metrics.Metrics, "delete_metric")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(metric_handler.OpenstackMetricHandler, "delete_metric")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_delete_metric_req(self, resp, del_metric, get_creds, perf_req):
         """Test Gnocchi delete metric request message from producer."""
         # Set-up message, producer and consumer for tests
@@ -129,7 +127,7 @@ class MetricIntegrationTest(unittest.TestCase):
             if message.key == "delete_metric_request":
                 # Metric has been deleted
                 del_metric.return_value = True
-                self.metric_req.metric_calls(message, 'test_id')
+                self.metric_req.handle_request(message, 'test_id')
 
                 # A response message is generated and sent by MON's producer
                 resp.assert_called_with(
@@ -142,8 +140,8 @@ class MetricIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(metrics.Metrics, "read_metric_data")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(metric_handler.OpenstackMetricHandler, "read_metric_data")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_read_metric_data_req(self, resp, read_data, get_creds, perf_req):
         """Test Gnocchi read metric data request message from producer."""
         # Set-up message, producer and consumer for tests
@@ -165,7 +163,7 @@ class MetricIntegrationTest(unittest.TestCase):
             if message.key == "read_metric_data_request":
                 # Mock empty lists generated by the request message
                 read_data.return_value = [], []
-                self.metric_req.metric_calls(message, 'test_id')
+                self.metric_req.handle_request(message, 'test_id')
 
                 # A response message is generated and sent by MON's producer
                 resp.assert_called_with(
@@ -178,8 +176,8 @@ class MetricIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(metrics.Metrics, "list_metrics")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(metric_handler.OpenstackMetricHandler, "list_metrics")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_list_metrics_req(self, resp, list_metrics, get_creds, perf_req):
         """Test Gnocchi list metrics request message from producer."""
         # Set-up message, producer and consumer for tests
@@ -200,7 +198,7 @@ class MetricIntegrationTest(unittest.TestCase):
             if message.key == "list_metric_request":
                 # Mock an empty list generated by the request
                 list_metrics.return_value = []
-                self.metric_req.metric_calls(message, 'test_id')
+                self.metric_req.handle_request(message, 'test_id')
 
                 # A response message is generated and sent by MON's producer
                 resp.assert_called_with(
@@ -211,8 +209,8 @@ class MetricIntegrationTest(unittest.TestCase):
 
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(metrics.Metrics, "get_metric_id")
-    @mock.patch.object(response.OpenStack_Response, "generate_response")
+    @mock.patch.object(metric_handler.OpenstackMetricHandler, "get_metric_id")
+    @mock.patch.object(response.OpenStackResponseBuilder, "generate_response")
     def test_update_metrics_req(self, resp, get_id, get_creds, perf_req):
         """Test Gnocchi update metric request message from KafkaProducer."""
         # Set-up message, producer and consumer for tests
@@ -232,7 +230,7 @@ class MetricIntegrationTest(unittest.TestCase):
             if message.key == "update_metric_request":
                 # Gnocchi doesn't support metric updates
                 get_id.return_value = "metric_id"
-                self.metric_req.metric_calls(message, 'test_id')
+                self.metric_req.handle_request(message, 'test_id')
 
                 # Response message is generated and sent via MON's producer
                 # No metric update has taken place
index 6fedf69..8aa2c9f 100644 (file)
@@ -21,7 +21,6 @@
 ##
 """Tests for all common OpenStack methods."""
 
-
 from __future__ import unicode_literals
 import json
 import logging
@@ -31,14 +30,14 @@ from threading import Thread
 
 import mock
 import requests
+from kafka import KafkaProducer
 from six.moves.BaseHTTPServer import BaseHTTPRequestHandler
 from six.moves.BaseHTTPServer import HTTPServer
 
-from osm_mon.core.message_bus.producer import KafkaProducer
 from osm_mon.core.settings import Config
-from osm_mon.plugins.OpenStack.Aodh.alarming import Alarming
+from osm_mon.plugins.OpenStack.Aodh.alarm_handler import OpenstackAlarmHandler
 from osm_mon.plugins.OpenStack.common import Common
-from osm_mon.plugins.OpenStack.response import OpenStack_Response
+from osm_mon.plugins.OpenStack.response import OpenStackResponseBuilder
 
 log = logging.getLogger(__name__)
 
@@ -86,12 +85,10 @@ class MockNotifierHandler(BaseHTTPRequestHandler):
 
     def notify_alarm(self, values):
         """Mock the notify_alarm functionality to generate a valid response."""
-        config = Config.instance()
-        config.read_environ()
-        self._alarming = Alarming()
+        cfg = Config.instance()
+        self._alarming = OpenstackAlarmHandler()
         self._common = Common()
-        self._response = OpenStack_Response()
-        self._producer = KafkaProducer('alarm_response')
+        self._response = OpenStackResponseBuilder()
         alarm_id = values['alarm_id']
 
         auth_token = Common.get_auth_token('test_id')
@@ -120,10 +117,8 @@ class MockNotifierHandler(BaseHTTPRequestHandler):
                         resource_id=resource_id,
                         sev=values['severity'], date=a_date,
                         state=values['current'], vim_type="OpenStack")
-                    self._producer.publish_alarm_response(
-                        'notify_alarm', resp_message)
                 except Exception:
-                    pass
+                    log.exception("Error generating response")
 
 
 def get_free_port():
@@ -155,12 +150,11 @@ def test_do_get():
 
 
 class AlarmNotificationTest(unittest.TestCase):
-    @mock.patch.object(KafkaProducer, "publish_alarm_response")
-    @mock.patch.object(OpenStack_Response, "generate_response")
+    @mock.patch.object(OpenStackResponseBuilder, "generate_response")
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(Common, "get_endpoint")
     @mock.patch.object(Common, "get_auth_token")
-    def test_post_notify_alarm(self, auth, endpoint, perf_req, resp, notify):
+    def test_post_notify_alarm(self, auth, endpoint, perf_req, resp):
         """Integration test for notify_alarm."""
         url = 'http://localhost:{port}/users'.format(port=mock_server_port)
         payload = {"severity": "critical",
@@ -187,6 +181,3 @@ class AlarmNotificationTest(unittest.TestCase):
             "notify_alarm", alarm_id="my_alarm_id", resource_id="my_resource_id",
             sev="critical", date='dd-mm-yyyy 00:00', state="current_state",
             vim_type="OpenStack")
-
-        # Response message is sent back to the SO via MON's producer
-        notify.assert_called_with("notify_alarm", mock.ANY)
index e69de29..cd7731b 100644 (file)
@@ -0,0 +1,34 @@
+# -*- coding: utf-8 -*-
+
+# Copyright 2018 Whitestack, LLC
+# *************************************************************
+
+# This file is part of OSM Monitoring module
+# All Rights Reserved to Whitestack, LLC
+
+# Licensed under the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License. You may obtain
+# a copy of the License at
+
+#         http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+# License for the specific language governing permissions and limitations
+# under the License.
+
+# For those usages not covered by the Apache License, Version 2.0 please
+# contact: bdiaz@whitestack.com or glavado@whitestack.com
+##
+import logging
+import sys
+
+from osm_mon.core.settings import Config
+
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
+log = logging.getLogger(__name__)
index f767c47..02cec8b 100644 (file)
 """Tests for all alarm request message keys."""
 
 import json
-
 import logging
-
 import unittest
+from io import UnsupportedOperation
 
 import mock
 
 from osm_mon.core.auth import AuthManager
 from osm_mon.core.database import VimCredentials, DatabaseManager
-from osm_mon.core.message_bus.producer import KafkaProducer
-from osm_mon.plugins.OpenStack.Aodh import alarming as alarm_req
+from osm_mon.plugins.OpenStack.Aodh import alarm_handler as alarm_req
+from osm_mon.plugins.OpenStack.Aodh.alarm_handler import OpenstackAlarmHandler
 from osm_mon.plugins.OpenStack.common import Common
 
 log = logging.getLogger(__name__)
@@ -51,14 +50,13 @@ class Message(object):
         self.value = json.dumps({'mock_value': 'mock_details'})
 
 
-@mock.patch.object(KafkaProducer, 'publish', mock.Mock())
 class TestAlarmKeys(unittest.TestCase):
     """Integration test for alarm request keys."""
 
     def setUp(self):
         """Setup the tests for alarm request keys."""
         super(TestAlarmKeys, self).setUp()
-        self.alarming = alarm_req.Alarming()
+        self.alarming = alarm_req.OpenstackAlarmHandler()
         self.alarming.common = Common()
 
     @mock.patch.object(AuthManager, 'get_credentials')
@@ -67,11 +65,10 @@ class TestAlarmKeys(unittest.TestCase):
     def test_alarming_authentication(self, get_token, get_endpoint, get_creds):
         """Test getting an auth_token and endpoint for alarm requests."""
         # if auth_token is None environment variables are used to authenticate
-        message = Message()
-
         get_creds.return_value = mock_creds
 
-        self.alarming.alarming(message, 'test_id')
+        with self.assertRaises(UnsupportedOperation):
+            self.alarming.handle_message('', {}, 'test_id')
 
         get_token.assert_called_with('test_id', verify_ssl=True)
         get_endpoint.assert_any_call('alarming', 'test_id', verify_ssl=True)
@@ -79,59 +76,49 @@ class TestAlarmKeys(unittest.TestCase):
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
     @mock.patch.object(Common, 'get_auth_token', mock.Mock())
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarm_req.Alarming, 'delete_alarm')
+    @mock.patch.object(alarm_req.OpenstackAlarmHandler, 'delete_alarm')
     def test_delete_alarm_key(self, del_alarm, get_creds):
         """Test the functionality for a create alarm request."""
-        # Mock a message value and key
-        message = Message()
-        message.key = 'delete_alarm_request'
-        message.value = json.dumps({'alarm_delete_request': {
+        value = {'alarm_delete_request': {
             'correlation_id': 1,
             'alarm_uuid': 'my_alarm_id'
-        }})
+        }}
 
         get_creds.return_value = mock_creds
         del_alarm.return_value = {}
 
         # Call the alarming functionality and check delete request
-        self.alarming.alarming(message, 'test_id')
+        self.alarming.handle_message('delete_alarm_request', value, 'test_id')
         del_alarm.assert_called_with(mock.ANY, mock.ANY, 'my_alarm_id', True)
 
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
     @mock.patch.object(Common, 'get_auth_token', mock.Mock())
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarm_req.Alarming, 'list_alarms')
+    @mock.patch.object(alarm_req.OpenstackAlarmHandler, 'list_alarms')
     def test_list_alarm_key(self, list_alarm, get_creds):
         """Test the functionality for a list alarm request."""
-        # Mock a message with list alarm key and value
-        message = Message()
-        message.key = 'list_alarm_request'
-        message.value = json.dumps({'alarm_list_request': {'correlation_id': 1}})
+        value = {'alarm_list_request': {'correlation_id': 1}}
 
         get_creds.return_value = mock_creds
 
         list_alarm.return_value = []
 
         # Call the alarming functionality and check list functionality
-        self.alarming.alarming(message, 'test_id')
+        self.alarming.handle_message('list_alarm_request', value, 'test_id')
         list_alarm.assert_called_with(mock.ANY, mock.ANY, {'correlation_id': 1}, True)
 
     @mock.patch.object(Common, 'get_auth_token', mock.Mock())
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarm_req.Alarming, 'update_alarm_state')
+    @mock.patch.object(alarm_req.OpenstackAlarmHandler, 'update_alarm_state')
     def test_ack_alarm_key(self, ack_alarm, get_creds):
         """Test the functionality for an acknowledge alarm request."""
-        # Mock a message with acknowledge alarm key and value
-        message = Message()
-        message.key = 'acknowledge_alarm'
-        message.value = json.dumps({'ack_details':
-                                        {'alarm_uuid': 'my_alarm_id'}})
+        value = {'ack_details': {'alarm_uuid': 'my_alarm_id'}}
 
         get_creds.return_value = mock_creds
 
         # Call alarming functionality and check acknowledge functionality
-        self.alarming.alarming(message, 'test_id')
+        self.alarming.handle_message('acknowledge_alarm_request', value, 'test_id')
         ack_alarm.assert_called_with(mock.ANY, mock.ANY, 'my_alarm_id', True)
 
     @mock.patch.object(Common, 'get_auth_token', mock.Mock())
@@ -139,25 +126,22 @@ class TestAlarmKeys(unittest.TestCase):
     @mock.patch.object(DatabaseManager, 'save_alarm', mock.Mock())
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, 'get_credentials')
-    @mock.patch.object(alarm_req.Alarming, 'configure_alarm')
+    @mock.patch.object(alarm_req.OpenstackAlarmHandler, 'configure_alarm')
     def test_config_alarm_key(self, config_alarm, get_creds, perf_req):
         """Test the functionality for a create alarm request."""
-        # Mock a message with config alarm key and value
-        message = Message()
-        message.key = 'create_alarm_request'
-        message.value = json.dumps({'alarm_create_request': {'correlation_id': 1, 'threshold_value': 50,
-                                                             'operation': 'GT', 'metric_name': 'cpu_utilization',
-                                                             'vdu_name': 'vdu',
-                                                             'vnf_member_index': '1',
-                                                             'ns_id': '1',
-                                                             'resource_uuid': '123'}})
+        value = {'alarm_create_request': {'correlation_id': 1, 'threshold_value': 50,
+                                          'operation': 'GT', 'metric_name': 'cpu_utilization',
+                                          'vdu_name': 'vdu',
+                                          'vnf_member_index': '1',
+                                          'ns_id': '1',
+                                          'resource_uuid': '123'}}
         mock_perf_req_return_value = {"metrics": {"cpu_util": 123}}
         perf_req.return_value = type('obj', (object,), {'text': json.dumps(mock_perf_req_return_value, sort_keys=True)})
         get_creds.return_value = mock_creds
 
         # Call alarming functionality and check config alarm call
         config_alarm.return_value = 'my_alarm_id'
-        self.alarming.alarming(message, 'test_id')
+        self.alarming.handle_message('create_alarm_request', value, 'test_id')
         config_alarm.assert_called_with(mock.ANY, mock.ANY, {'correlation_id': 1, 'threshold_value': 50,
                                                              'operation': 'GT',
                                                              'metric_name': 'cpu_utilization',
index 7368f13..67486e7 100644 (file)
@@ -28,7 +28,7 @@ import unittest
 import mock
 
 from osm_mon.core.settings import Config
-from osm_mon.plugins.OpenStack.Aodh import alarming as alarm_req
+from osm_mon.plugins.OpenStack.Aodh import alarm_handler as alarm_req
 from osm_mon.plugins.OpenStack.common import Common
 
 log = logging.getLogger(__name__)
@@ -55,7 +55,7 @@ class TestAlarming(unittest.TestCase):
     def setUp(self):
         """Setup for tests."""
         super(TestAlarming, self).setUp()
-        self.alarming = alarm_req.Alarming()
+        self.alarming = alarm_req.OpenstackAlarmHandler()
 
     @mock.patch.object(Common, "perform_request")
     def test_config_invalid_alarm_req(self, perf_req):
@@ -152,6 +152,9 @@ class TestAlarming(unittest.TestCase):
     @mock.patch.object(Common, "perform_request")
     def test_ack_alarm_req(self, perf_req):
         """Test update alarm state for acknowledge alarm request."""
+        resp = Response({})
+        perf_req.return_value = resp
+
         self.alarming.update_alarm_state(alarm_endpoint, auth_token, "my_alarm_id", True)
 
         perf_req.assert_called_with(
@@ -185,7 +188,7 @@ class TestAlarming(unittest.TestCase):
         perf_req.assert_called_with(mock.ANY, auth_token, req_type="get")
         self.assertEqual(perf_req.call_count, 1)
 
-    @mock.patch.object(alarm_req.Alarming, "check_payload")
+    @mock.patch.object(alarm_req.OpenstackAlarmHandler, "check_payload")
     @mock.patch.object(Common, "perform_request")
     def test_update_alarm_valid(self, perf_req, check_pay):
         """Test valid update alarm request."""
index de2d13b..b71ca72 100644 (file)
 """Tests for all metric request message keys."""
 
 import json
-
 import logging
-
 import unittest
 
 import mock
 
 from osm_mon.core.auth import AuthManager
-from osm_mon.plugins.OpenStack.Gnocchi import metrics as metric_req
-
+from osm_mon.plugins.OpenStack.Gnocchi import metric_handler as metric_req
 from osm_mon.plugins.OpenStack.common import Common
 
 log = logging.getLogger(__name__)
@@ -72,10 +69,10 @@ class TestMetricCalls(unittest.TestCase):
     def setUp(self):
         """Setup the tests for metric request keys."""
         super(TestMetricCalls, self).setUp()
-        self.metrics = metric_req.Metrics()
+        self.metrics = metric_req.OpenstackMetricHandler()
         self.metrics._common = Common()
 
-    @mock.patch.object(metric_req.Metrics, "get_metric_id")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "get_metric_id")
     @mock.patch.object(Common, "perform_request")
     def test_invalid_config_metric_req(
             self, perf_req, get_metric):
@@ -104,7 +101,7 @@ class TestMetricCalls(unittest.TestCase):
 
         perf_req.assert_not_called()
 
-    @mock.patch.object(metric_req.Metrics, "get_metric_id")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "get_metric_id")
     @mock.patch.object(Common, "perform_request")
     @mock.patch.object(AuthManager, "get_credentials")
     def test_valid_config_metric_req(
@@ -150,7 +147,7 @@ class TestMetricCalls(unittest.TestCase):
         with self.assertRaises(ValueError):
             self.metrics.delete_metric(endpoint, auth_token, "metric_id", verify_ssl=False)
 
-    @mock.patch.object(metric_req.Metrics, "response_list")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "response_list")
     @mock.patch.object(Common, "perform_request")
     def test_complete_list_metric_req(self, perf_req, resp_list):
         """Test the complete list metric function."""
@@ -163,7 +160,7 @@ class TestMetricCalls(unittest.TestCase):
             "<ANY>/v1/metric?sort=name:asc", auth_token, req_type="get", verify_ssl=False)
         resp_list.assert_called_with([{u'id': u'test_id'}])
 
-    @mock.patch.object(metric_req.Metrics, "response_list")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "response_list")
     @mock.patch.object(Common, "perform_request")
     def test_resource_list_metric_req(self, perf_req, resp_list):
         """Test the resource list metric function."""
@@ -175,7 +172,7 @@ class TestMetricCalls(unittest.TestCase):
         perf_req.assert_any_call(
             "<ANY>/v1/metric/test_id", auth_token, req_type="get", verify_ssl=False)
 
-    @mock.patch.object(metric_req.Metrics, "response_list")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "response_list")
     @mock.patch.object(Common, "perform_request")
     def test_name_list_metric_req(self, perf_req, resp_list):
         """Test the metric_name list metric function."""
@@ -189,7 +186,7 @@ class TestMetricCalls(unittest.TestCase):
         resp_list.assert_called_with(
             [{u'id': u'test_id'}], metric_name="disk_write_bytes")
 
-    @mock.patch.object(metric_req.Metrics, "response_list")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "response_list")
     @mock.patch.object(Common, "perform_request")
     def test_combined_list_metric_req(self, perf_req, resp_list):
         """Test the combined resource and metric list metric function."""
@@ -214,7 +211,7 @@ class TestMetricCalls(unittest.TestCase):
         perf_req.assert_called_with(
             "<ANY>/v1/resource/generic/r_id", auth_token, req_type="get", verify_ssl=False)
 
-    @mock.patch.object(metric_req.Metrics, "get_metric_id")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "get_metric_id")
     @mock.patch.object(Common, "perform_request")
     def test_valid_read_data_req(self, perf_req, get_metric):
         """Test the read metric data function, for a valid call."""
index f66be72..2fa31a6 100644 (file)
@@ -28,8 +28,8 @@ import unittest
 import mock
 
 from osm_mon.core.auth import AuthManager
-from osm_mon.core.message_bus.producer import KafkaProducer
-from osm_mon.plugins.OpenStack.Gnocchi import metrics as metric_req
+from osm_mon.plugins.OpenStack.Gnocchi import metric_handler as metric_req
+from osm_mon.plugins.OpenStack.Gnocchi.metric_handler import OpenstackMetricHandler
 from osm_mon.plugins.OpenStack.common import Common
 
 log = logging.getLogger(__name__)
@@ -54,26 +54,22 @@ class Message(object):
         self.value = json.dumps({"mock_message": "message_details"})
 
 
-@mock.patch.object(KafkaProducer, 'publish', mock.Mock())
 class TestMetricReq(unittest.TestCase):
     """Integration test for metric request keys."""
 
     def setUp(self):
         """Setup the tests for metric request keys."""
         super(TestMetricReq, self).setUp()
-        self.metrics = metric_req.Metrics()
+        self.metrics = metric_req.OpenstackMetricHandler()
 
     @mock.patch.object(Common, "get_auth_token", mock.Mock())
     @mock.patch.object(Common, "get_endpoint", mock.Mock())
-    @mock.patch.object(metric_req.Metrics, "delete_metric")
-    @mock.patch.object(metric_req.Metrics, "get_metric_id")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "delete_metric")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "get_metric_id")
     @mock.patch.object(AuthManager, "get_credentials")
     def test_delete_metric_key(self, get_creds, get_metric_id, del_metric):
         """Test the functionality for a delete metric request."""
-        # Mock a message value and key
-        message = Message()
-        message.key = "delete_metric_request"
-        message.value = json.dumps({"metric_name": "disk_write_ops", "resource_uuid": "my_r_id", "correlation_id": 1})
+        value = {"metric_name": "disk_write_ops", "resource_uuid": "my_r_id", "correlation_id": 1}
 
         get_creds.return_value = type('obj', (object,), {
             'config': '{"insecure":true}'
@@ -82,19 +78,16 @@ class TestMetricReq(unittest.TestCase):
 
         # Call the metric functionality and check delete request
         get_metric_id.return_value = "my_metric_id"
-        self.metrics.metric_calls(message, 'test_id')
+        self.metrics.handle_request('delete_metric_request', value, 'test_id')
         del_metric.assert_called_with(mock.ANY, mock.ANY, "my_metric_id", False)
 
     @mock.patch.object(Common, "get_auth_token", mock.Mock())
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
-    @mock.patch.object(metric_req.Metrics, "list_metrics")
+    @mock.patch.object(metric_req.OpenstackMetricHandler, "list_metrics")
     @mock.patch.object(AuthManager, "get_credentials")
     def test_list_metric_key(self, get_creds, list_metrics):
         """Test the functionality for a list metric request."""
-        # Mock a message with list metric key and value
-        message = Message()
-        message.key = "list_metric_request"
-        message.value = json.dumps({"metrics_list_request": {"correlation_id": 1}})
+        value = {"metrics_list_request": {"correlation_id": 1}}
 
         get_creds.return_value = type('obj', (object,), {
             'config': '{"insecure":true}'
@@ -103,7 +96,7 @@ class TestMetricReq(unittest.TestCase):
         list_metrics.return_value = []
 
         # Call the metric functionality and check list functionality
-        self.metrics.metric_calls(message, 'test_id')
+        self.metrics.handle_request('list_metric_request', value, 'test_id')
         list_metrics.assert_called_with(mock.ANY, mock.ANY, {"correlation_id": 1}, False)
 
     @mock.patch.object(Common, "get_auth_token", mock.Mock())
@@ -112,13 +105,10 @@ class TestMetricReq(unittest.TestCase):
     @mock.patch.object(Common, "perform_request")
     def test_update_metric_key(self, perf_req, get_creds):
         """Test the functionality for an update metric request."""
-        # Mock a message with update metric key and value
-        message = Message()
-        message.key = "update_metric_request"
-        message.value = json.dumps({"metric_update_request":
-                                        {"correlation_id": 1,
-                                         "metric_name": "my_metric",
-                                         "resource_uuid": "my_r_id"}})
+        value = {"metric_update_request":
+                     {"correlation_id": 1,
+                      "metric_name": "my_metric",
+                      "resource_uuid": "my_r_id"}}
 
         get_creds.return_value = type('obj', (object,), {
             'config': '{"insecure":true}'
@@ -130,35 +120,29 @@ class TestMetricReq(unittest.TestCase):
 
         # Call metric functionality and confirm no function is called
         # Gnocchi does not support updating a metric configuration
-        self.metrics.metric_calls(message, 'test_id')
+        self.metrics.handle_request('update_metric_request', value, 'test_id')
 
     @mock.patch.object(Common, "get_auth_token", mock.Mock())
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
-    @mock.patch.object(metric_req.Metrics, "configure_metric")
+    @mock.patch.object(OpenstackMetricHandler, "configure_metric")
     @mock.patch.object(AuthManager, "get_credentials")
     def test_config_metric_key(self, get_credentials, config_metric):
         """Test the functionality for a create metric request."""
-        # Mock a message with create metric key and value
-        message = Message()
-        message.key = "create_metric_request"
-        message.value = json.dumps({"metric_create_request": {"correlation_id": 123}})
+        value = {"metric_create_request": {"correlation_id": 123}}
         get_credentials.return_value = type('obj', (object,), {'config': '{"insecure":true}'})
         # Call metric functionality and check config metric
         config_metric.return_value = "metric_id", "resource_id"
-        self.metrics.metric_calls(message, 'test_id')
+        self.metrics.handle_request('create_metric_request', value, 'test_id')
         config_metric.assert_called_with(mock.ANY, mock.ANY, {"correlation_id": 123}, False)
 
     @mock.patch.object(Common, "get_auth_token", mock.Mock())
     @mock.patch.object(Common, 'get_endpoint', mock.Mock())
-    @mock.patch.object(metric_req.Metrics, "read_metric_data")
+    @mock.patch.object(OpenstackMetricHandler, "read_metric_data")
     @mock.patch.object(AuthManager, "get_credentials")
     @mock.patch.object(Common, "perform_request")
     def test_read_data_key(self, perf_req, get_creds, read_data):
         """Test the functionality for a read metric data request."""
-        # Mock a message with a read data key and value
-        message = Message()
-        message.key = "read_metric_data_request"
-        message.value = json.dumps({"correlation_id": 123, "metric_name": "cpu_utilization", "resource_uuid": "uuid"})
+        value = {"correlation_id": 123, "metric_name": "cpu_utilization", "resource_uuid": "uuid"}
 
         get_creds.return_value = type('obj', (object,), {
             'config': '{"insecure":true}'
@@ -170,6 +154,6 @@ class TestMetricReq(unittest.TestCase):
 
         # Call metric functionality and check read data metrics
         read_data.return_value = "time_stamps", "data_values"
-        self.metrics.metric_calls(message, 'test_id')
+        self.metrics.handle_request('read_metric_data_request', value, 'test_id')
         read_data.assert_called_with(
-            mock.ANY, mock.ANY, json.loads(message.value), False)
+            mock.ANY, mock.ANY, value, False)
index e2695d4..a420c70 100644 (file)
@@ -25,9 +25,9 @@ import json
 import unittest
 
 import mock
+from kafka import KafkaProducer
 
 from osm_mon.core.database import DatabaseManager, Alarm
-from osm_mon.core.message_bus.producer import KafkaProducer
 from osm_mon.plugins.OpenStack.Aodh.notifier import NotifierHandler
 
 post_data = {"severity": "critical",
@@ -76,6 +76,8 @@ class MockNotifierHandler(NotifierHandler):
         pass
 
 
+@mock.patch.object(KafkaProducer, "__init__", lambda *args, **kwargs: None)
+@mock.patch.object(KafkaProducer, "flush", mock.Mock())
 class TestNotifier(unittest.TestCase):
     """Test the NotifierHandler class for requests from aodh."""
 
@@ -100,7 +102,7 @@ class TestNotifier(unittest.TestCase):
         set_head.assert_called_once()
         notify.assert_called_with(post_data)
 
-    @mock.patch.object(KafkaProducer, "publish_alarm_response")
+    @mock.patch.object(NotifierHandler, "_publish_response")
     @mock.patch.object(DatabaseManager, "get_alarm")
     def test_notify_alarm_valid_alarm(
             self, get_alarm, notify):
@@ -113,10 +115,9 @@ class TestNotifier(unittest.TestCase):
         get_alarm.return_value = mock_alarm
 
         self.handler.notify_alarm(post_data)
+        notify.assert_called_with('notify_alarm', mock.ANY)
 
-        notify.assert_called_with("notify_alarm", mock.ANY)
-
-    @mock.patch.object(KafkaProducer, "publish_alarm_response")
+    @mock.patch.object(NotifierHandler, "_publish_response")
     @mock.patch.object(DatabaseManager, "get_alarm")
     def test_notify_alarm_invalid_alarm(
             self, get_alarm, notify):
index 6cf4e3f..1377bc0 100644 (file)
@@ -22,7 +22,6 @@
 """Test that the correct responses are generated for each message."""
 
 import logging
-
 import unittest
 
 import mock
@@ -38,7 +37,7 @@ class TestOpenStackResponse(unittest.TestCase):
     def setUp(self):
         """Setup for testing OpenStack plugin responses."""
         super(TestOpenStackResponse, self).setUp()
-        self.plugin_resp = resp.OpenStack_Response()
+        self.plugin_resp = resp.OpenStackResponseBuilder()
 
     def test_invalid_key(self):
         """Test if an invalid key is entered for a response."""
@@ -46,70 +45,70 @@ class TestOpenStackResponse(unittest.TestCase):
         self.assertEqual(message, None)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "alarm_list_response")
+        resp.OpenStackResponseBuilder, "alarm_list_response")
     def test_list_alarm_resp(self, alarm_list_resp):
         """Test out a function call for a list alarm response."""
         message = self.plugin_resp.generate_response("list_alarm_response")
         self.assertEqual(alarm_list_resp.return_value, message)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "list_metric_response")
+        resp.OpenStackResponseBuilder, "list_metric_response")
     def test_list_metric_resp(self, metric_list_resp):
         """Test list metric response function call."""
         message = self.plugin_resp.generate_response("list_metric_response")
         self.assertEqual(message, metric_list_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "delete_alarm_response")
+        resp.OpenStackResponseBuilder, "delete_alarm_response")
     def test_delete_alarm_resp(self, del_alarm_resp):
         """Test delete alarm response function call."""
         message = self.plugin_resp.generate_response("delete_alarm_response")
         self.assertEqual(message, del_alarm_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "delete_metric_response")
+        resp.OpenStackResponseBuilder, "delete_metric_response")
     def test_delete_metric_resp(self, del_metric_resp):
         """Test the response functionality of delete metric response."""
         message = self.plugin_resp.generate_response("delete_metric_response")
         self.assertEqual(message, del_metric_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "create_alarm_response")
+        resp.OpenStackResponseBuilder, "create_alarm_response")
     def test_create_alarm_resp(self, config_alarm_resp):
         """Test create alarm response function call."""
         message = self.plugin_resp.generate_response("create_alarm_response")
         self.assertEqual(message, config_alarm_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "metric_create_response")
+        resp.OpenStackResponseBuilder, "metric_create_response")
     def test_create_metric_resp(self, config_metric_resp):
         """Test create metric response function call."""
         message = self.plugin_resp.generate_response("create_metric_response")
         self.assertEqual(message, config_metric_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "update_alarm_response")
+        resp.OpenStackResponseBuilder, "update_alarm_response")
     def test_update_alarm_resp(self, up_alarm_resp):
         """Test update alarm response function call."""
         message = self.plugin_resp.generate_response("update_alarm_response")
         self.assertEqual(message, up_alarm_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "update_metric_response")
+        resp.OpenStackResponseBuilder, "update_metric_response")
     def test_update_metric_resp(self, up_metric_resp):
         """Test update metric response function call."""
         message = self.plugin_resp.generate_response("update_metric_response")
         self.assertEqual(message, up_metric_resp.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "notify_alarm")
+        resp.OpenStackResponseBuilder, "notify_alarm")
     def test_notify_alarm(self, notify_alarm):
         """Test notify alarm response function call."""
         message = self.plugin_resp.generate_response("notify_alarm")
         self.assertEqual(message, notify_alarm.return_value)
 
     @mock.patch.object(
-        resp.OpenStack_Response, "read_metric_data_response")
+        resp.OpenStackResponseBuilder, "read_metric_data_response")
     def test_read_metric_data_resp(self, read_data_resp):
         """Test read metric data response function call."""
         message = self.plugin_resp.generate_response(
index e2d83ed..64d5d51 100644 (file)
 """VMware MON plugin tests."""
 
 import logging
+import sys
 
-# Initialise a logger for tests
-logging.basicConfig(filename='vmware_mon_tests.log',
-                    format='%(asctime)s %(message)s',
-                    datefmt='%m/%d/%Y %I:%M:%S %p', filemode='a',
-                    level=logging.INFO)
+from osm_mon.core.settings import Config
+
+cfg = Config.instance()
+logging.basicConfig(stream=sys.stdout,
+                    format='%(asctime)s - %(name)s - %(levelname)s - %(message)s',
+                    datefmt='%m/%d/%Y %I:%M:%S %p',
+                    level=logging.getLevelName(cfg.OSMMON_LOG_LEVEL))
 log = logging.getLogger(__name__)
 
index 30e9a3d..2d10d1b 100644 (file)
 
 """ Mock tests for VMware vROPs Mon plugin """
 
+import os
 import sys
-
-import json
-
-import logging
-
 import unittest
 
 import mock
-
 import requests
 
-import os
-
-sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)),"..","..",".."))
+sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), "..", "..", ".."))
 
 from osm_mon.plugins.vRealiseOps import mon_plugin_vrops as monPlugin
 
-from pyvcloud.vcd.client import Client,BasicLoginCredentials
+from pyvcloud.vcd.client import Client
 
 
 class TestMonPlugin(unittest.TestCase):
@@ -51,29 +44,28 @@ class TestMonPlugin(unittest.TestCase):
         """Setup the tests for Mon Plugin class methods"""
         super(TestMonPlugin, self).setUp()
 
-        self.m_vim_access_config = {'vrops_site':'abc',
-            'vrops_user':'user',
-            'vrops_password':'passwd',
-            'vim_url':'vcd_url',
-            'admin_username':'admin',
-            'admin_password':'admin_passwd',
-            'vim_uuid':'1',
-            'tenant_id':'org_vdc_1'}
+        self.m_vim_access_config = {'vrops_site': 'abc',
+                                    'vrops_user': 'user',
+                                    'vrops_password': 'passwd',
+                                    'vim_url': 'vcd_url',
+                                    'admin_username': 'admin',
+                                    'admin_password': 'admin_passwd',
+                                    'vim_uuid': '1',
+                                    'tenant_id': 'org_vdc_1'}
         self.mon_plugin = monPlugin.MonPlugin(self.m_vim_access_config)
         # create client object
         self.vca = Client('test', verify_ssl_certs=False)
         # create session
         self.session = requests.Session()
 
-
     def test_get_default_Params_valid_metric_alarm_name(self):
         """Test get default params method"""
 
         # Mock valid metric_alarm_name and response
         metric_alarm_name = "Average_Memory_Usage_Above_Threshold"
         expected_return = {'impact': 'risk', 'cancel_cycles': 2, 'adapter_kind': 'VMWARE',
-                           'repeat': False,'cancel_period': 300, 'alarm_type': 16,
-                           'vrops_alarm': 'Avg_Mem_Usage_Above_Thr','enabled': True, 'period': 300,
+                           'repeat': False, 'cancel_period': 300, 'alarm_type': 16,
+                           'vrops_alarm': 'Avg_Mem_Usage_Above_Thr', 'enabled': True, 'period': 300,
                            'resource_kind': 'VirtualMachine', 'alarm_subType': 19,
                            'action': 'acknowledge', 'evaluation': 2, 'unit': '%'}
 
@@ -83,7 +75,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     def test_get_default_Params_invalid_metric_alarm_name(self):
         """Test get default params method invalid metric alarm"""
 
@@ -97,7 +88,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_symptom_valid_req_response(self, m_post):
         """Test create symptom method-valid request"""
@@ -105,20 +95,20 @@ class TestMonPlugin(unittest.TestCase):
         # Mock valid symptom params and mock responses
         symptom_param = {'threshold_value': 0, 'cancel_cycles': 1, 'adapter_kind_key': 'VMWARE',
                          'resource_kind_key': 'VirtualMachine', 'severity': 'CRITICAL',
-                         'symptom_name':\
-                         'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                         'symptom_name': \
+                             'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                          'operation': 'GT', 'wait_cycles': 1, 'metric_key': 'cpu|usage_average'}
 
         m_post.return_value.status_code = 201
         m_post.return_value.content = \
-                         '{"id":"SymptomDefinition-351c23b4-bc3c-4c7b-b4af-1ad90a673c5d",\
-                         "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
-                         "adapterKindKey":"VMWARE","resourceKindKey":"VirtualMachine",\
-                         "waitCycles":1,"cancelCycles":1,\
-                         "state":{"severity":"CRITICAL","condition":{"type":"CONDITION_HT",\
-                         "key":"cpu|usage_average","operator":"GT","value":"0.0",\
-                         "valueType":"NUMERIC",\
-                         "instanced":false,"thresholdType":"STATIC"}}}'
+            '{"id":"SymptomDefinition-351c23b4-bc3c-4c7b-b4af-1ad90a673c5d",\
+            "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
+            "adapterKindKey":"VMWARE","resourceKindKey":"VirtualMachine",\
+            "waitCycles":1,"cancelCycles":1,\
+            "state":{"severity":"CRITICAL","condition":{"type":"CONDITION_HT",\
+            "key":"cpu|usage_average","operator":"GT","value":"0.0",\
+            "valueType":"NUMERIC",\
+            "instanced":false,"thresholdType":"STATIC"}}}'
 
         expected_return = "SymptomDefinition-351c23b4-bc3c-4c7b-b4af-1ad90a673c5d"
 
@@ -131,7 +121,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_symptom_invalid_req_response(self, m_post):
         """Test create symptom method-invalid response"""
@@ -139,8 +128,8 @@ class TestMonPlugin(unittest.TestCase):
         # Mock valid symptom params and invalid  mock responses
         symptom_param = {'threshold_value': 0, 'cancel_cycles': 1, 'adapter_kind_key': 'VMWARE',
                          'resource_kind_key': 'VirtualMachine', 'severity': 'CRITICAL',
-                         'symptom_name':\
-                         'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                         'symptom_name': \
+                             'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                          'operation': 'GT', 'wait_cycles': 1, 'metric_key': 'cpu|usage_average'}
 
         m_post.return_value.status_code = 404
@@ -157,7 +146,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_symptom_incorrect_data(self, m_post):
         """Test create symptom method-incorrect data"""
@@ -165,8 +153,8 @@ class TestMonPlugin(unittest.TestCase):
         # Mock valid symptom params and invalid  mock responses
         symptom_param = {'threshold_value': 0, 'cancel_cycles': 1, 'adapter_kind_key': 'VMWARE',
                          'resource_kind_key': 'VirtualMachine', 'severity': 'CRITICAL',
-                         'symptom_name':\
-                         'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                         'symptom_name': \
+                             'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                          'operation': 'GT', 'metric_key': 'cpu|usage_average'}
 
         expected_return = None
@@ -180,7 +168,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_alarm_definition_valid_req_response(self, m_post):
         """Test create alarm definition method-valid response"""
@@ -191,21 +178,21 @@ class TestMonPlugin(unittest.TestCase):
                        'severity': 'CRITICAL', 'impact': 'risk', 'adapterKindKey': 'VMWARE',
                        'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                        'resourceKindKey': 'VirtualMachine', 'type': 16,
-                       'symptomDefinitionId':\
-                       'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
+                       'symptomDefinitionId': \
+                           'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
 
         m_post.return_value.status_code = 201
         m_post.return_value.content = \
-                       '{"id":"AlertDefinition-d4f21e4b-770a-45d6-b298-022eaf489115",\
-                       "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
-                       "description":"CPU_Utilization_Above_Threshold","adapterKindKey":"VMWARE",\
-                       "resourceKindKey":"VirtualMachine","waitCycles":1,"cancelCycles":1,\
-                       "type":16,"subType":19,\
-                       "states":[{"severity":"CRITICAL","base-symptom-set":{"type":"SYMPTOM_SET",\
-                       "relation":"SELF","aggregation":"ALL","symptomSetOperator":"AND",\
-                       "symptomDefinitionIds":\
-                       ["SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df"]},\
-                       "impact":{"impactType":"BADGE","detail":"risk"}}]}'
+            '{"id":"AlertDefinition-d4f21e4b-770a-45d6-b298-022eaf489115",\
+            "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
+            "description":"CPU_Utilization_Above_Threshold","adapterKindKey":"VMWARE",\
+            "resourceKindKey":"VirtualMachine","waitCycles":1,"cancelCycles":1,\
+            "type":16,"subType":19,\
+            "states":[{"severity":"CRITICAL","base-symptom-set":{"type":"SYMPTOM_SET",\
+            "relation":"SELF","aggregation":"ALL","symptomSetOperator":"AND",\
+            "symptomDefinitionIds":\
+            ["SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df"]},\
+            "impact":{"impactType":"BADGE","detail":"risk"}}]}'
 
         expected_return = "AlertDefinition-d4f21e4b-770a-45d6-b298-022eaf489115"
 
@@ -218,7 +205,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_alarm_definition_invalid_req_response(self, m_post):
         """Test create alarm definition method-invalid response"""
@@ -229,8 +215,8 @@ class TestMonPlugin(unittest.TestCase):
                        'severity': 'CRITICAL', 'impact': 'risk', 'adapterKindKey': 'VMWARE',
                        'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                        'resourceKindKey': 'VirtualMachine', 'type': 16,
-                       'symptomDefinitionId':\
-                       'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
+                       'symptomDefinitionId': \
+                           'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
 
         m_post.return_value.status_code = 404
         m_post.return_value.content = '404 Not Found'
@@ -246,7 +232,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     def test_create_alarm_definition_incorrect_data(self, m_post):
         """Test create alarm definition method-incorrect data"""
@@ -255,8 +240,8 @@ class TestMonPlugin(unittest.TestCase):
         alarm_param = {'description': 'CPU_Utilization_Above_Threshold', 'cancelCycles': 1,
                        'subType': 19, 'waitCycles': 1, 'type': 16,
                        'severity': 'CRITICAL', 'impact': 'risk', 'adapterKindKey': 'VMWARE',
-                       'symptomDefinitionId':\
-                       'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
+                       'symptomDefinitionId': \
+                           'SymptomDefinition-25278b06-bff8-4409-a141-9b4e064235df'}
         expected_return = None
 
         # call create symptom method under test
@@ -268,7 +253,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -277,48 +261,47 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_valid_req(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
+    def test_configure_alarm_valid_req(self, m_get_default_Params,
+                                       m_get_alarm_defination_by_name,
+                                       m_create_symptom,
+                                       m_create_alarm_definition,
+                                       m_get_vm_moref_id,
+                                       m_get_vm_resource_id,
+                                       m_create_alarm_notification_rule,
                                        m_save_alarm):
-
         """Test configure alarm valid request creating alarm"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                        'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                        'statistic': 'AVERAGE', 'metric_name': 'cpu_utilization',
-                       'vdu_name':'vdu1','vnf_member_index':'index1','ns_id':'nsd1',
+                       'vdu_name': 'vdu1', 'vnf_member_index': 'index1', 'ns_id': 'nsd1',
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': \
+                              'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
 
-        #alarm parameters to  be passed for alarm creation
+        # alarm parameters to  be passed for alarm creation
         alarm_params = {'description': 'CPU_Utilization_Above_Threshold',
                         'cancelCycles': 1, 'subType': 19,
                         'waitCycles': 1, 'severity': 'CRITICAL',
                         'impact': 'risk', 'adapterKindKey': 'VMWARE',
                         'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                         'resourceKindKey': 'VirtualMachine',
-                        'symptomDefinitionId':\
-                        'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
+                        'symptomDefinitionId': \
+                            'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
                         'type': 16}
 
         vm_moref_id = 'vm-6626'
@@ -327,7 +310,7 @@ class TestMonPlugin(unittest.TestCase):
         resource_id = 'ac87622f-b761-40a0-b151-00872a2a456e'
         alarm_def_uuid = '0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -339,29 +322,29 @@ class TestMonPlugin(unittest.TestCase):
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
 
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804'
-        m_create_alarm_definition.return_value =\
-        'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
+        m_create_alarm_definition.return_value = \
+            'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
         m_get_vm_moref_id.return_value = vm_moref_id
         m_get_vm_resource_id.return_value = 'ac87622f-b761-40a0-b151-00872a2a456e'
         m_create_alarm_notification_rule.return_value = 'f37900e7-dd01-4383-b84c-08f519530d71'
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
         m_create_alarm_definition.assert_called_with(alarm_params)
         m_get_vm_moref_id.assert_called_with(config_dict['resource_uuid'])
         m_get_vm_resource_id.assert_called_with(vm_moref_id)
-        m_create_alarm_notification_rule.assert_called_with(vrops_alarm_name,\
-                                                            alarm_def,\
+        m_create_alarm_notification_rule.assert_called_with(vrops_alarm_name,
+                                                            alarm_def,
                                                             resource_id)
-        m_save_alarm.assert_called_with(alarm_def_uuid,'1',
+        m_save_alarm.assert_called_with(vrops_alarm_name, '1',
                                         config_dict['threshold_value'],
                                         config_dict['operation'],
                                         config_dict['metric_name'],
@@ -369,10 +352,9 @@ class TestMonPlugin(unittest.TestCase):
                                         config_dict['vnf_member_index'],
                                         config_dict['ns_id'])
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -381,17 +363,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_invalid_alarm_name_req(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_invalid_alarm_name_req(self, m_get_default_Params,
+                                                    m_get_alarm_defination_by_name,
+                                                    m_create_symptom,
+                                                    m_create_alarm_definition,
+                                                    m_get_vm_moref_id,
+                                                    m_get_vm_resource_id,
+                                                    m_create_alarm_notification_rule,
+                                                    m_save_alarm):
         """Test configure alarm invalid test: for invalid alarm name"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -402,13 +384,13 @@ class TestMonPlugin(unittest.TestCase):
 
         alarm_def_uuid = None
 
-        #Mock default Parameters return value to None
+        # Mock default Parameters return value to None
         m_get_default_Params.return_value = {}
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         m_get_default_Params.assert_called_with(config_dict['alarm_name'])
         m_get_alarm_defination_by_name.assert_not_called()
         m_create_symptom.assert_not_called()
@@ -418,10 +400,9 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value i.e. None
+        # Verify return value with expected value i.e. None
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -430,17 +411,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_invalid_metric_name_req(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_invalid_metric_name_req(self, m_get_default_Params,
+                                                     m_get_alarm_defination_by_name,
+                                                     m_create_symptom,
+                                                     m_create_alarm_definition,
+                                                     m_get_vm_moref_id,
+                                                     m_get_vm_resource_id,
+                                                     m_create_alarm_notification_rule,
+                                                     m_save_alarm):
         """Test configure alarm invalid test: for invalid metric name"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -451,7 +432,7 @@ class TestMonPlugin(unittest.TestCase):
 
         alarm_def_uuid = None
 
-        #Mock default Parameters return values for metrics to None
+        # Mock default Parameters return values for metrics to None
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -463,10 +444,10 @@ class TestMonPlugin(unittest.TestCase):
                                             {}
                                             ]
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_not_called()
         m_create_symptom.assert_not_called()
@@ -476,10 +457,9 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value i.e. None
+        # Verify return value with expected value i.e. None
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -488,17 +468,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_invalid_already_exists(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_invalid_already_exists(self, m_get_default_Params,
+                                                    m_get_alarm_defination_by_name,
+                                                    m_create_symptom,
+                                                    m_create_alarm_definition,
+                                                    m_get_vm_moref_id,
+                                                    m_get_vm_resource_id,
+                                                    m_create_alarm_notification_rule,
+                                                    m_save_alarm):
         """Test configure alarm invalid test: for alarm that already exists"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -510,7 +490,7 @@ class TestMonPlugin(unittest.TestCase):
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -521,14 +501,13 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return value
+        # set mocked function return value
         m_get_alarm_defination_by_name.return_value = ['mocked_alarm_CPU_Utilization_Above_Thr']
 
-
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_not_called()
@@ -537,10 +516,9 @@ class TestMonPlugin(unittest.TestCase):
         m_get_vm_resource_id.assert_not_called()
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -549,17 +527,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_failed_symptom_creation(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_failed_symptom_creation(self, m_get_default_Params,
+                                                     m_get_alarm_defination_by_name,
+                                                     m_create_symptom,
+                                                     m_create_alarm_definition,
+                                                     m_get_vm_moref_id,
+                                                     m_get_vm_resource_id,
+                                                     m_create_alarm_notification_rule,
+                                                     m_save_alarm):
         """Test configure alarm: failed to create symptom"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -568,21 +546,20 @@ class TestMonPlugin(unittest.TestCase):
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -593,14 +570,14 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = None
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
@@ -610,10 +587,9 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -622,17 +598,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_failed_alert_creation(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_failed_alert_creation(self, m_get_default_Params,
+                                                   m_get_alarm_defination_by_name,
+                                                   m_create_symptom,
+                                                   m_create_alarm_definition,
+                                                   m_get_vm_moref_id,
+                                                   m_get_vm_resource_id,
+                                                   m_create_alarm_notification_rule,
+                                                   m_save_alarm):
         """Test configure alarm: failed to create alert in vROPs"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -641,33 +617,31 @@ class TestMonPlugin(unittest.TestCase):
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
 
-        #alarm parameters to  be passed for alarm creation
+        # alarm parameters to  be passed for alarm creation
         alarm_params = {'description': 'CPU_Utilization_Above_Threshold',
                         'cancelCycles': 1, 'subType': 19,
                         'waitCycles': 1, 'severity': 'CRITICAL',
                         'impact': 'risk', 'adapterKindKey': 'VMWARE',
                         'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                         'resourceKindKey': 'VirtualMachine',
-                        'symptomDefinitionId':\
-                        'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
+                        'symptomDefinitionId': 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
                         'type': 16}
 
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -678,15 +652,15 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804'
         m_create_alarm_definition.return_value = None
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
@@ -696,10 +670,9 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -708,17 +681,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_failed_to_get_vm_moref_id(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_failed_to_get_vm_moref_id(self, m_get_default_Params,
+                                                       m_get_alarm_defination_by_name,
+                                                       m_create_symptom,
+                                                       m_create_alarm_definition,
+                                                       m_get_vm_moref_id,
+                                                       m_get_vm_resource_id,
+                                                       m_create_alarm_notification_rule,
+                                                       m_save_alarm):
         """Test configure alarm: failed to get vm_moref_id"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -727,33 +700,33 @@ class TestMonPlugin(unittest.TestCase):
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': \
+                              'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
 
-        #alarm parameters to  be passed for alarm creation
+        # alarm parameters to  be passed for alarm creation
         alarm_params = {'description': 'CPU_Utilization_Above_Threshold',
                         'cancelCycles': 1, 'subType': 19,
                         'waitCycles': 1, 'severity': 'CRITICAL',
                         'impact': 'risk', 'adapterKindKey': 'VMWARE',
                         'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                         'resourceKindKey': 'VirtualMachine',
-                        'symptomDefinitionId':\
-                        'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
+                        'symptomDefinitionId': \
+                            'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
                         'type': 16}
 
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -764,17 +737,17 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804'
-        m_create_alarm_definition.return_value =\
-        'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
+        m_create_alarm_definition.return_value = \
+            'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
         m_get_vm_moref_id.return_value = None
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
@@ -784,10 +757,9 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_not_called()
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -796,17 +768,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_failed_to_get_vm_resource_id(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_failed_to_get_vm_resource_id(self, m_get_default_Params,
+                                                          m_get_alarm_defination_by_name,
+                                                          m_create_symptom,
+                                                          m_create_alarm_definition,
+                                                          m_get_vm_moref_id,
+                                                          m_get_vm_resource_id,
+                                                          m_create_alarm_notification_rule,
+                                                          m_save_alarm):
         """Test configure alarm: failed to get vm resource_id"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -815,34 +787,34 @@ class TestMonPlugin(unittest.TestCase):
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': \
+                              'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
 
-        #alarm parameters to  be passed for alarm creation
+        # alarm parameters to  be passed for alarm creation
         alarm_params = {'description': 'CPU_Utilization_Above_Threshold',
                         'cancelCycles': 1, 'subType': 19,
                         'waitCycles': 1, 'severity': 'CRITICAL',
                         'impact': 'risk', 'adapterKindKey': 'VMWARE',
                         'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                         'resourceKindKey': 'VirtualMachine',
-                        'symptomDefinitionId':\
-                        'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
+                        'symptomDefinitionId': \
+                            'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
                         'type': 16}
 
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         vm_moref_id = 'vm-6626'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -853,19 +825,19 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804'
-        m_create_alarm_definition.return_value =\
-        'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
+        m_create_alarm_definition.return_value = \
+            'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
         m_get_vm_moref_id.return_value = vm_moref_id
         m_get_vm_resource_id.return_value = None
         m_save_alarm.assert_not_called()
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
@@ -874,10 +846,9 @@ class TestMonPlugin(unittest.TestCase):
         m_get_vm_resource_id.assert_called_with(vm_moref_id)
         m_create_alarm_notification_rule.assert_not_called()
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.DatabaseManager, 'save_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'create_alarm_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
@@ -886,17 +857,17 @@ class TestMonPlugin(unittest.TestCase):
     @mock.patch.object(monPlugin.MonPlugin, 'create_symptom')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_by_name')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_configure_alarm_failed_to_create_alarm_notification_rule(self, m_get_default_Params,\
-                                       m_get_alarm_defination_by_name,\
-                                       m_create_symptom,\
-                                       m_create_alarm_definition,\
-                                       m_get_vm_moref_id,\
-                                       m_get_vm_resource_id,\
-                                       m_create_alarm_notification_rule,\
-                                       m_save_alarm):
+    def test_configure_alarm_failed_to_create_alarm_notification_rule(self, m_get_default_Params,
+                                                                      m_get_alarm_defination_by_name,
+                                                                      m_create_symptom,
+                                                                      m_create_alarm_definition,
+                                                                      m_get_vm_moref_id,
+                                                                      m_get_vm_resource_id,
+                                                                      m_create_alarm_notification_rule,
+                                                                      m_save_alarm):
         """Test configure alarm: failed to create alarm notification rule"""
 
-        #Mock input configuration dictionary
+        # Mock input configuration dictionary
         config_dict = {'threshold_value': 0, 'severity': 'CRITICAL',
                        'alarm_name': 'CPU_Utilization_Above_Threshold',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
@@ -905,27 +876,27 @@ class TestMonPlugin(unittest.TestCase):
                        'operation': 'GT', 'unit': '%',
                        'description': 'CPU_Utilization_Above_Threshold'}
 
-        #symptom parameters to be passed for symptom creation
+        # symptom parameters to be passed for symptom creation
         symptom_params = {'threshold_value': 0,
                           'cancel_cycles': 1,
                           'adapter_kind_key': 'VMWARE',
                           'resource_kind_key': 'VirtualMachine',
                           'severity': 'CRITICAL',
-                          'symptom_name':\
-                          'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                          'symptom_name': \
+                              'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                           'operation': 'GT',
                           'wait_cycles': 1,
                           'metric_key': 'cpu|usage_average'}
 
-        #alarm parameters to  be passed for alarm creation
+        # alarm parameters to  be passed for alarm creation
         alarm_params = {'description': 'CPU_Utilization_Above_Threshold',
                         'cancelCycles': 1, 'subType': 19,
                         'waitCycles': 1, 'severity': 'CRITICAL',
                         'impact': 'risk', 'adapterKindKey': 'VMWARE',
                         'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                         'resourceKindKey': 'VirtualMachine',
-                        'symptomDefinitionId':\
-                        'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
+                        'symptomDefinitionId': \
+                            'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804',
                         'type': 16}
 
         vrops_alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
@@ -934,7 +905,7 @@ class TestMonPlugin(unittest.TestCase):
         resource_id = 'ac87622f-b761-40a0-b151-00872a2a456e'
         alarm_def_uuid = None
 
-        #Mock default Parameters for alarm & metric configuration
+        # Mock default Parameters for alarm & metric configuration
         m_get_default_Params.side_effect = [{'impact': 'risk', 'cancel_cycles': 1,
                                              'adapter_kind': 'VMWARE', 'repeat': False,
                                              'cancel_period': 300, 'alarm_type': 16,
@@ -945,19 +916,19 @@ class TestMonPlugin(unittest.TestCase):
                                              'evaluation': 1, 'unit': 'msec'},
                                             {'metric_key': 'cpu|usage_average', 'unit': '%'}
                                             ]
-        #set mocked function return values
+        # set mocked function return values
         m_get_alarm_defination_by_name.return_value = []
         m_create_symptom.return_value = 'SymptomDefinition-2e8f9ddc-9f7b-4cd6-b85d-7d7fe3a8a804'
-        m_create_alarm_definition.return_value =\
-        'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
+        m_create_alarm_definition.return_value = \
+            'AlertDefinition-0f3cdcb3-4e1b-4a0b-86d0-66d4b3f65220'
         m_get_vm_moref_id.return_value = vm_moref_id
         m_get_vm_resource_id.return_value = 'ac87622f-b761-40a0-b151-00872a2a456e'
         m_create_alarm_notification_rule.return_value = None
 
-        #Call configure_alarm method under test
+        # Call configure_alarm method under test
         return_value = self.mon_plugin.configure_alarm(config_dict)
 
-        #Verify that mocked methods are called with correct parameters
+        # Verify that mocked methods are called with correct parameters
         self.assertEqual(m_get_default_Params.call_count, 2)
         m_get_alarm_defination_by_name.assert_called_with(vrops_alarm_name)
         m_create_symptom.assert_called_with(symptom_params)
@@ -967,17 +938,16 @@ class TestMonPlugin(unittest.TestCase):
         m_create_alarm_notification_rule.assert_called_with(vrops_alarm_name, alarm_def, resource_id)
         m_save_alarm.assert_not_called()
 
-        #Verify return value with expected value of alarm_def_uuid
+        # Verify return value with expected value of alarm_def_uuid
         self.assertEqual(return_value, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_alarm_defination_details_valid_rest_req_response(self, m_get):
         """Test get_alarm_defination_details: For a valid REST request response"""
 
         alarm_uuid = '9a6d8a14-9f25-4d81-bf91-4d773497444d'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{"id":"AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d",\
                             "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
@@ -990,47 +960,47 @@ class TestMonPlugin(unittest.TestCase):
                             ["SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278"]},\
                             "impact":{"impactType":"BADGE","detail":"risk"}}]}'
 
-        expected_alarm_details = {'adapter_kind': 'VMWARE','symptom_definition_id':\
-                                  'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278',
-                                  'alarm_name':\
-                                  'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+        expected_alarm_details = {'adapter_kind': 'VMWARE', 'symptom_definition_id': \
+            'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278',
+                                  'alarm_name': \
+                                      'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                                   'alarm_id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
                                   'resource_kind': 'VirtualMachine', 'type': 16, 'sub_type': 19}
 
         expected_alarm_details_json = {'states':
-                                       [{'impact':
-                                         {'impactType':'BADGE', 'detail':'risk'},'severity':'CRITICAL',
-                                         'base-symptom-set': {'symptomDefinitionIds':\
-                                         ['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
-                                         'relation': 'SELF', 'type': 'SYMPTOM_SET',
-                                         'aggregation':'ALL', 'symptomSetOperator': 'AND'}}],
+                                           [{'impact':
+                                                 {'impactType': 'BADGE', 'detail': 'risk'}, 'severity': 'CRITICAL',
+                                             'base-symptom-set': {'symptomDefinitionIds': \
+                                                 [
+                                                     'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
+                                                 'relation': 'SELF', 'type': 'SYMPTOM_SET',
+                                                 'aggregation': 'ALL', 'symptomSetOperator': 'AND'}}],
                                        'adapterKindKey': 'VMWARE',
                                        'description': 'CPU_Utilization_Above_Threshold',
-                                       'type': 16, 'cancelCycles': 1, 
+                                       'type': 16, 'cancelCycles': 1,
                                        'resourceKindKey': 'VirtualMachine',
                                        'subType': 19, 'waitCycles': 1,
                                        'id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
-                                       'name':\
-                                       'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'}
+                                       'name': \
+                                           'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'}
 
-        #Call get_alarm_defination_details method under test
+        # Call get_alarm_defination_details method under test
         alarm_details_json, alarm_details = self.mon_plugin.get_alarm_defination_details(alarm_uuid)
 
-        #Verify that mocked method is called
+        # Verify that mocked method is called
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_alarm_details, alarm_details)
         self.assertEqual(expected_alarm_details_json, alarm_details_json)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_alarm_defination_details_invalid_rest_req_response(self, m_get):
         """Test get_alarm_defination_details: For an invalid REST request response"""
 
         alarm_uuid = '9a6d8a14-9f25-4d81-bf91-4d773497444d'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 404
         m_get.return_value.content = '{"message": "No such AlertDefinition - \
                                         AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444.",\
@@ -1039,24 +1009,23 @@ class TestMonPlugin(unittest.TestCase):
         expected_alarm_details = None
         expected_alarm_details_json = None
 
-        #Call get_alarm_defination_details method under test
+        # Call get_alarm_defination_details method under test
         alarm_details_json, alarm_details = self.mon_plugin.get_alarm_defination_details(alarm_uuid)
 
-        #verify that mocked method is called
+        # verify that mocked method is called
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_alarm_details, alarm_details)
         self.assertEqual(expected_alarm_details_json, alarm_details_json)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_alarm_defination_by_name_valid_rest_req_response(self, m_get):
         """Test get_alarm_defination_by_name: For a valid REST request response"""
 
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{"pageInfo": {"totalCount": 1,"page": 0,"pageSize": 1000},\
                                     "links": [\
@@ -1081,46 +1050,45 @@ class TestMonPlugin(unittest.TestCase):
                                             "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278"]}}]\
                                         }]}'
 
-        #Expected return match list
+        # Expected return match list
         Exp_alert_match_list = [{'states':
-                                   [{'impact': {'impactType': 'BADGE', 'detail': 'risk'},
-                                     'severity': 'CRITICAL',
-                                     'base-symptom-set': {
-                                         'symptomDefinitionIds': \
-                                         ['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
-                                         'relation': 'SELF',
-                                         'type': 'SYMPTOM_SET',
-                                         'aggregation': 'ALL',
-                                         'symptomSetOperator': 'AND'}
-                                     }],
-                                   'adapterKindKey': 'VMWARE',
-                                   'description': 'CPU_Utilization_Above_Threshold',
-                                   'type': 16,
-                                   'cancelCycles': 1,
-                                   'resourceKindKey': 'VirtualMachine',
-                                   'subType': 19, 'waitCycles': 1,
-                                   'id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
-                                   'name': \
-                                   'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
-                                   }]
-
-        #Call get_alarm_defination_by_name method under test
+                                     [{'impact': {'impactType': 'BADGE', 'detail': 'risk'},
+                                       'severity': 'CRITICAL',
+                                       'base-symptom-set': {
+                                           'symptomDefinitionIds': \
+                                               ['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
+                                           'relation': 'SELF',
+                                           'type': 'SYMPTOM_SET',
+                                           'aggregation': 'ALL',
+                                           'symptomSetOperator': 'AND'}
+                                       }],
+                                 'adapterKindKey': 'VMWARE',
+                                 'description': 'CPU_Utilization_Above_Threshold',
+                                 'type': 16,
+                                 'cancelCycles': 1,
+                                 'resourceKindKey': 'VirtualMachine',
+                                 'subType': 19, 'waitCycles': 1,
+                                 'id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
+                                 'name': \
+                                     'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
+                                 }]
+
+        # Call get_alarm_defination_by_name method under test
         alert_match_list = self.mon_plugin.get_alarm_defination_by_name(alarm_name)
 
-        #Verify that mocked method is called
+        # Verify that mocked method is called
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(Exp_alert_match_list, alert_match_list)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_alarm_defination_by_name_no_valid_alarm_found(self, m_get):
         """Test get_alarm_defination_by_name: With no valid alarm found in returned list"""
 
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda5'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{"pageInfo": {"totalCount": 1,"page": 0,"pageSize": 1000},\
                                     "links": [\
@@ -1145,50 +1113,49 @@ class TestMonPlugin(unittest.TestCase):
                                             "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278"]}}]\
                                         }]}'
 
-        #Expected return match list
+        # Expected return match list
         Exp_alert_match_list = []
 
-        #Call get_alarm_defination_by_name method under test
+        # Call get_alarm_defination_by_name method under test
         alert_match_list = self.mon_plugin.get_alarm_defination_by_name(alarm_name)
 
-        #Verify that mocked method is called
+        # Verify that mocked method is called
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(Exp_alert_match_list, alert_match_list)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     @mock.patch.object(monPlugin.MonPlugin, 'get_symptom_defination_details')
-    def test_update_symptom_defination_valid_symptom_req_response(self,\
-                                                                  m_get_symptom_defination_details,\
+    def test_update_symptom_defination_valid_symptom_req_response(self,
+                                                                  m_get_symptom_defination_details,
                                                                   m_put):
         """Test update_symptom_defination: With valid REST response, update symptom"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_defination_id = 'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'
-        new_alarm_config = {'severity':"CRITICAL",
+        new_alarm_config = {'severity': "CRITICAL",
                             'operation': 'GT',
-                            'threshold_value':5,
-                            'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'
+                            'threshold_value': 5,
+                            'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'
                             }
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_symptom_defination_details.return_value = {
-                            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",
-                            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",
-                            "adapterKindKey": "VMWARE",
-                            "resourceKindKey": "VirtualMachine",
-                            "waitCycles": 1,
-                            "cancelCycles": 1,
-                            "state": {"severity": "CRITICAL",
-                                      "condition": {
-                                          "type": "CONDITION_HT",
-                                          "key": "cpu|usage_average","operator": "GT","value": "0.0",
-                                          "valueType": "NUMERIC","instanced": False,
-                                          "thresholdType": "STATIC"}
-                                      }
-                           }
+            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",
+            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",
+            "adapterKindKey": "VMWARE",
+            "resourceKindKey": "VirtualMachine",
+            "waitCycles": 1,
+            "cancelCycles": 1,
+            "state": {"severity": "CRITICAL",
+                      "condition": {
+                          "type": "CONDITION_HT",
+                          "key": "cpu|usage_average", "operator": "GT", "value": "0.0",
+                          "valueType": "NUMERIC", "instanced": False,
+                          "thresholdType": "STATIC"}
+                      }
+        }
 
         m_put.return_value.status_code = 200
         m_put.return_value.content = '{\
@@ -1202,49 +1169,48 @@ class TestMonPlugin(unittest.TestCase):
                     "type":"CONDITION_HT","key":"cpu|usage_average","operator":"GT","value":"5.0",\
                     "valueType":"NUMERIC","instanced":False,"thresholdType":"STATIC"}}}'
 
-        #Call update_symptom_defination method under test
-        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,\
+        # Call update_symptom_defination method under test
+        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,
                                                                  new_alarm_config)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_symptom_defination_details.assert_called_with(symptom_defination_id)
-        #m_put.assert_called_with(symptom_defination_id,new_alarm_config)
+        # m_put.assert_called_with(symptom_defination_id,new_alarm_config)
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(symptom_defination_id, symptom_uuid)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     @mock.patch.object(monPlugin.MonPlugin, 'get_symptom_defination_details')
-    def test_update_symptom_defination_invalid_symptom_req_response(self,\
-                                                                  m_get_symptom_defination_details,\
-                                                                  m_put):
+    def test_update_symptom_defination_invalid_symptom_req_response(self,
+                                                                    m_get_symptom_defination_details,
+                                                                    m_put):
         """Test update_symptom_defination: If invalid REST response received, return None"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_defination_id = 'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'
-        new_alarm_config = {'severity':"CRITICAL",
+        new_alarm_config = {'severity': "CRITICAL",
                             'operation': 'GT',
-                            'threshold_value':5,
-                            'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'
+                            'threshold_value': 5,
+                            'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'
                             }
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_symptom_defination_details.return_value = {
-                            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",
-                            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",
-                            "adapterKindKey": "VMWARE",
-                            "resourceKindKey": "VirtualMachine",
-                            "waitCycles": 1,
-                            "cancelCycles": 1,
-                            "state": {"severity": "CRITICAL",
-                                      "condition": {
-                                          "type": "CONDITION_HT",
-                                          "key": "cpu|usage_average","operator": "GT","value": "0.0",
-                                          "valueType": "NUMERIC","instanced": False,
-                                          "thresholdType": "STATIC"}
-                                      }
-                           }
+            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",
+            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",
+            "adapterKindKey": "VMWARE",
+            "resourceKindKey": "VirtualMachine",
+            "waitCycles": 1,
+            "cancelCycles": 1,
+            "state": {"severity": "CRITICAL",
+                      "condition": {
+                          "type": "CONDITION_HT",
+                          "key": "cpu|usage_average", "operator": "GT", "value": "0.0",
+                          "valueType": "NUMERIC", "instanced": False,
+                          "thresholdType": "STATIC"}
+                      }
+        }
 
         m_put.return_value.status_code = 500
         m_put.return_value.content = '{\
@@ -1258,56 +1224,54 @@ class TestMonPlugin(unittest.TestCase):
                  6bf2-4e2f-a91c-8c19d240eda4 does not exist and hence cannot be updated.;"}],\
             "httpStatusCode": 500,"apiErrorCode": 500}'
 
-        #Call update_symptom_defination method under test
-        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,\
+        # Call update_symptom_defination method under test
+        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,
                                                                  new_alarm_config)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_symptom_defination_details.assert_called_with(symptom_defination_id)
         m_put.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(symptom_uuid, None)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     @mock.patch.object(monPlugin.MonPlugin, 'get_symptom_defination_details')
-    def test_update_symptom_defination_failed_to_get_symptom_defination(self,\
-                                                                  m_get_symptom_defination_details,\
-                                                                  m_put):
+    def test_update_symptom_defination_failed_to_get_symptom_defination(self,
+                                                                        m_get_symptom_defination_details,
+                                                                        m_put):
         """Test update_symptom_defination: if fails to get symptom_defination returns None"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_defination_id = 'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'
-        new_alarm_config = {'severity':"CRITICAL",
+        new_alarm_config = {'severity': "CRITICAL",
                             'operation': 'GT',
-                            'threshold_value':5,
-                            'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'
+                            'threshold_value': 5,
+                            'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'
                             }
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_symptom_defination_details.return_value = None
 
-        #Call update_symptom_defination method under test
-        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,\
+        # Call update_symptom_defination method under test
+        symptom_uuid = self.mon_plugin.update_symptom_defination(symptom_defination_id,
                                                                  new_alarm_config)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_symptom_defination_details.assert_called_with(symptom_defination_id)
         m_put.assert_not_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(symptom_uuid, None)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_symptom_defination_details_valid_req_response(self,m_get):
+    def test_get_symptom_defination_details_valid_req_response(self, m_get):
         """Test update_symptom_defination: With valid REST response symptom is created"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_uuid = 'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{\
             "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",\
@@ -1316,32 +1280,34 @@ class TestMonPlugin(unittest.TestCase):
             "cancelCycles": 1,"state": {"severity": "CRITICAL","condition": {"type": "CONDITION_HT",\
             "key": "cpu|usage_average","operator": "GT","value": "6.0","valueType": "NUMERIC",\
             "instanced": false,"thresholdType": "STATIC"}}}'
-        expected_symptom_details = {\
-            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",\
-            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
-            "adapterKindKey": "VMWARE","resourceKindKey": "VirtualMachine","waitCycles": 1,\
-            "cancelCycles": 1,"state": {"severity": "CRITICAL","condition": {"type": "CONDITION_HT",\
-            "key": "cpu|usage_average","operator": "GT","value": "6.0","valueType": "NUMERIC",\
-            "instanced": False,"thresholdType": "STATIC"}}}
-
-        #Call update_symptom_defination method under test
+        expected_symptom_details = { \
+            "id": "SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278",
+            "name": "CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",
+            "adapterKindKey": "VMWARE", "resourceKindKey": "VirtualMachine", "waitCycles": 1,
+            "cancelCycles": 1, "state": {"severity": "CRITICAL", "condition": {"type": "CONDITION_HT",
+                                                                               "key": "cpu|usage_average",
+                                                                               "operator": "GT", "value": "6.0",
+                                                                               "valueType": "NUMERIC",
+                                                                               "instanced": False,
+                                                                               "thresholdType": "STATIC"}}}
+
+        # Call update_symptom_defination method under test
         symptom_details = self.mon_plugin.get_symptom_defination_details(symptom_uuid)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_symptom_details, symptom_details)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_symptom_defination_details_invalid_req_response(self,m_get):
+    def test_get_symptom_defination_details_invalid_req_response(self, m_get):
         """Test update_symptom_defination: if invalid REST response received return None"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_uuid = 'SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 404
         m_get.return_value.content = '{"message": "No such SymptomDefinition\
         - SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278.",\
@@ -1349,55 +1315,53 @@ class TestMonPlugin(unittest.TestCase):
 
         expected_symptom_details = None
 
-        #Call update_symptom_defination method under test
+        # Call update_symptom_defination method under test
         symptom_details = self.mon_plugin.get_symptom_defination_details(symptom_uuid)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_symptom_details, symptom_details)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_symptom_defination_details_symptom_uuid_not_provided(self,m_get):
+    def test_get_symptom_defination_details_symptom_uuid_not_provided(self, m_get):
         """Test update_symptom_defination: if required symptom uuid is not provided"""
 
-        #Expected symptom to be updated
+        # Expected symptom to be updated
         symptom_uuid = None
         expected_symptom_details = None
 
-        #Call update_symptom_defination method under test
+        # Call update_symptom_defination method under test
         symptom_details = self.mon_plugin.get_symptom_defination_details(symptom_uuid)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_not_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_symptom_details, symptom_details)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     def test_reconfigure_alarm_valid_req_response(self, m_put):
         """Test reconfigure_alarm: for valid REST response"""
 
-        #Set input parameters to reconfigure_alarm
+        # Set input parameters to reconfigure_alarm
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
             'description': 'CPU_Utilization_Above_Threshold', 'adapterKindKey': 'VMWARE',
-            'states':[{'impact':{'impactType':'BADGE', 'detail':'risk'}, 'severity':'CRITICAL',
-                       'base-symptom-set':{
-                           'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
-                           'relation': 'SELF','type': 'SYMPTOM_SET', 'aggregation':'ALL',
-                           'symptomSetOperator': 'AND'}}],
-            'type': 16, 'cancelCycles': 1, 'resourceKindKey': 'VirtualMachine','subType': 19,
+            'states': [{'impact': {'impactType': 'BADGE', 'detail': 'risk'}, 'severity': 'CRITICAL',
+                        'base-symptom-set': {
+                            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
+                            'relation': 'SELF', 'type': 'SYMPTOM_SET', 'aggregation': 'ALL',
+                            'symptomSetOperator': 'AND'}}],
+            'type': 16, 'cancelCycles': 1, 'resourceKindKey': 'VirtualMachine', 'subType': 19,
             'waitCycles': 1}
 
-        new_alarm_config = {'severity':'WARNING',
+        new_alarm_config = {'severity': 'WARNING',
                             'description': 'CPU_Utilization_Above_Threshold_Warning'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_put.return_value.status_code = 200
         m_put.return_value.content = '{"id":"AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d",\
             "name":"CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4",\
@@ -1408,40 +1372,39 @@ class TestMonPlugin(unittest.TestCase):
             "symptomDefinitionIds":["SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278"]},\
             "impact":{"impactType":"BADGE","detail":"risk"}}]}'
 
-        #Expected alarm_def_uuid to be returned
+        # Expected alarm_def_uuid to be returned
         expected_alarm_def_uuid = '9a6d8a14-9f25-4d81-bf91-4d773497444d'
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_def_uuid = self.mon_plugin.reconfigure_alarm(alarm_details_json, new_alarm_config)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_put.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_alarm_def_uuid, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     def test_reconfigure_alarm_invalid_req_response(self, m_put):
         """Test reconfigure_alarm: for invalid REST response, return None"""
 
-        #Set input parameters to reconfigure_alarm
+        # Set input parameters to reconfigure_alarm
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-bf91-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
             'description': 'CPU_Utilization_Above_Threshold', 'adapterKindKey': 'VMWARE',
-            'states':[{'impact':{'impactType':'BADGE', 'detail':'risk'}, 'severity':'CRITICAL',
-                       'base-symptom-set':{
-                           'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
-                           'relation': 'SELF','type': 'SYMPTOM_SET', 'aggregation':'ALL',
-                           'symptomSetOperator': 'AND'}}],
-            'type': 16, 'cancelCycles': 1, 'resourceKindKey': 'VirtualMachine','subType': 19,
+            'states': [{'impact': {'impactType': 'BADGE', 'detail': 'risk'}, 'severity': 'CRITICAL',
+                        'base-symptom-set': {
+                            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-bcd3-9b5884973278'],
+                            'relation': 'SELF', 'type': 'SYMPTOM_SET', 'aggregation': 'ALL',
+                            'symptomSetOperator': 'AND'}}],
+            'type': 16, 'cancelCycles': 1, 'resourceKindKey': 'VirtualMachine', 'subType': 19,
             'waitCycles': 1}
 
-        new_alarm_config = {'severity':'WARNING',
+        new_alarm_config = {'severity': 'WARNING',
                             'description': 'CPU_Utilization_Above_Threshold_Warning'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_put.return_value.status_code = 500
         m_put.return_value.content = '{"message": "Internal Server error, cause unknown.",\
             "moreInformation": [{"name": "errorMessage",\
@@ -1452,301 +1415,292 @@ class TestMonPlugin(unittest.TestCase):
             e14b203c-6bf2-4e2f-a91c-8c19d240eda4 since it does not exist;"}],\
             "httpStatusCode": 500,"apiErrorCode": 500}'
 
-        #Expected alarm_def_uuid to be returned
+        # Expected alarm_def_uuid to be returned
         expected_alarm_def_uuid = None
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_def_uuid = self.mon_plugin.reconfigure_alarm(alarm_details_json, new_alarm_config)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_put.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_alarm_def_uuid, alarm_def_uuid)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'delete_symptom_definition')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_alarm_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_delete_alarm_configuration_successful_alarm_deletion(self,\
-                                                                  m_get_alarm_defination_details,\
-                                                                  m_delete_notification_rule,\
-                                                                  m_delete_alarm_defination,\
+    def test_delete_alarm_configuration_successful_alarm_deletion(self,
+                                                                  m_get_alarm_defination_details,
+                                                                  m_delete_notification_rule,
+                                                                  m_delete_alarm_defination,
                                                                   m_delete_symptom_definition):
         """Test delete_alarm_configuration: for successful alarm deletion, return alarm uuid"""
 
-        #Set input parameters to delete_alarm_configuration
-        delete_alarm_req_dict = {'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'}
+        # Set input parameters to delete_alarm_configuration
+        delete_alarm_req_dict = {'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
+            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
         alarm_details = {
-            'alarm_name':'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'alarm_id':'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
-            'symptom_definition_id':'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
+            'alarm_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
+            'alarm_id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
+            'symptom_definition_id': 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
 
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
         m_delete_notification_rule.return_value = '989e7293-d78d-4405-92e30ec4f247'
         m_delete_alarm_defination.return_value = alarm_details['alarm_id']
         m_delete_symptom_definition.return_value = alarm_details['symptom_definition_id']
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_uuid = self.mon_plugin.delete_alarm_configuration(delete_alarm_req_dict)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_alarm_defination_details.assert_called_with(delete_alarm_req_dict['alarm_uuid'])
         m_delete_notification_rule.assert_called_with(alarm_details['alarm_name'])
         m_delete_alarm_defination.assert_called_with(alarm_details['alarm_id'])
         m_delete_symptom_definition.assert_called_with(alarm_details['symptom_definition_id'])
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(alarm_uuid, delete_alarm_req_dict['alarm_uuid'])
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'delete_symptom_definition')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_alarm_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_delete_alarm_configuration_failed_to_get_alarm_defination(self,\
-                                                                  m_get_alarm_defination_details,\
-                                                                  m_delete_notification_rule,\
-                                                                  m_delete_alarm_defination,\
-                                                                  m_delete_symptom_definition):
+    def test_delete_alarm_configuration_failed_to_get_alarm_defination(self,
+                                                                       m_get_alarm_defination_details,
+                                                                       m_delete_notification_rule,
+                                                                       m_delete_alarm_defination,
+                                                                       m_delete_symptom_definition):
         """Test delete_alarm_configuration: if failed to get alarm definition, return None"""
 
-        #Set input parameters to delete_alarm_configuration
-        delete_alarm_req_dict = {'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'}
+        # Set input parameters to delete_alarm_configuration
+        delete_alarm_req_dict = {'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         alarm_details_json = None
         alarm_details = None
 
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_uuid = self.mon_plugin.delete_alarm_configuration(delete_alarm_req_dict)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_alarm_defination_details.assert_called_with(delete_alarm_req_dict['alarm_uuid'])
         m_delete_notification_rule.assert_not_called()
         m_delete_alarm_defination.assert_not_called()
         m_delete_symptom_definition.assert_not_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(alarm_uuid, None)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'delete_symptom_definition')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_alarm_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_delete_alarm_configuration_failed_to_delete_notification_rule(self,\
-                                                                  m_get_alarm_defination_details,\
-                                                                  m_delete_notification_rule,\
-                                                                  m_delete_alarm_defination,\
-                                                                  m_delete_symptom_definition):
+    def test_delete_alarm_configuration_failed_to_delete_notification_rule(self,
+                                                                           m_get_alarm_defination_details,
+                                                                           m_delete_notification_rule,
+                                                                           m_delete_alarm_defination,
+                                                                           m_delete_symptom_definition):
         """Test delete_alarm_configuration: if failed to delete notification rule, return None"""
 
-        #Set input parameters to delete_alarm_configuration
-        delete_alarm_req_dict = {'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'}
+        # Set input parameters to delete_alarm_configuration
+        delete_alarm_req_dict = {'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
+            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
         alarm_details = {
-            'alarm_name':'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'alarm_id':'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
-            'symptom_definition_id':'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
+            'alarm_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
+            'alarm_id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
+            'symptom_definition_id': 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
 
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
         m_delete_notification_rule.return_value = None
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_uuid = self.mon_plugin.delete_alarm_configuration(delete_alarm_req_dict)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_alarm_defination_details.assert_called_with(delete_alarm_req_dict['alarm_uuid'])
         m_delete_notification_rule.assert_called_with(alarm_details['alarm_name'])
         m_delete_alarm_defination.assert_not_called()
         m_delete_symptom_definition.assert_not_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(alarm_uuid, None)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'delete_symptom_definition')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_alarm_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_delete_alarm_configuration_failed_to_delete_alarm_defination(self,\
-                                                                  m_get_alarm_defination_details,\
-                                                                  m_delete_notification_rule,\
-                                                                  m_delete_alarm_defination,\
-                                                                  m_delete_symptom_definition):
+    def test_delete_alarm_configuration_failed_to_delete_alarm_defination(self,
+                                                                          m_get_alarm_defination_details,
+                                                                          m_delete_notification_rule,
+                                                                          m_delete_alarm_defination,
+                                                                          m_delete_symptom_definition):
         """Test delete_alarm_configuration: if failed to delete alarm definition, return None"""
 
-        #Set input parameters to delete_alarm_configuration
-        delete_alarm_req_dict = {'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'}
+        # Set input parameters to delete_alarm_configuration
+        delete_alarm_req_dict = {'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
+            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
         alarm_details = {
-            'alarm_name':'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'alarm_id':'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
-            'symptom_definition_id':'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
+            'alarm_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
+            'alarm_id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
+            'symptom_definition_id': 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
 
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
         m_delete_notification_rule.return_value = '989e7293-d78d-4405-92e30ec4f247'
         m_delete_alarm_defination.return_value = None
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_uuid = self.mon_plugin.delete_alarm_configuration(delete_alarm_req_dict)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_alarm_defination_details.assert_called_with(delete_alarm_req_dict['alarm_uuid'])
         m_delete_notification_rule.assert_called_with(alarm_details['alarm_name'])
         m_delete_alarm_defination.assert_called_with(alarm_details['alarm_id'])
         m_delete_symptom_definition.assert_not_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(alarm_uuid, None)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'delete_symptom_definition')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_alarm_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'delete_notification_rule')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_delete_alarm_configuration_failed_to_delete_symptom_definition(self,\
-                                                                  m_get_alarm_defination_details,\
-                                                                  m_delete_notification_rule,\
-                                                                  m_delete_alarm_defination,\
-                                                                  m_delete_symptom_definition):
+    def test_delete_alarm_configuration_failed_to_delete_symptom_definition(self,
+                                                                            m_get_alarm_defination_details,
+                                                                            m_delete_notification_rule,
+                                                                            m_delete_alarm_defination,
+                                                                            m_delete_symptom_definition):
         """Test delete_alarm_configuration: if failed to delete symptom definition, return None"""
 
-        #Set input parameters to delete_alarm_configuration
-        delete_alarm_req_dict = {'alarm_uuid':'9a6d8a14-9f25-4d81-bf91-4d773497444d'}
+        # Set input parameters to delete_alarm_configuration
+        delete_alarm_req_dict = {'alarm_uuid': '9a6d8a14-9f25-4d81-bf91-4d773497444d'}
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         alarm_details_json = {
             'id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
             'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'symptomDefinitionIds':['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
+            'symptomDefinitionIds': ['SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278']}
         alarm_details = {
-            'alarm_name':'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
-            'alarm_id':'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
-            'symptom_definition_id':'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
+            'alarm_name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4',
+            'alarm_id': 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d',
+            'symptom_definition_id': 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'}
 
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
         m_delete_notification_rule.return_value = '989e7293-d78d-4405-92e30ec4f247'
         m_delete_alarm_defination.return_value = alarm_details['alarm_id']
         m_delete_symptom_definition.return_value = None
 
-        #Call reconfigure_alarm method under test
+        # Call reconfigure_alarm method under test
         alarm_uuid = self.mon_plugin.delete_alarm_configuration(delete_alarm_req_dict)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_alarm_defination_details.assert_called_with(delete_alarm_req_dict['alarm_uuid'])
         m_delete_notification_rule.assert_called_with(alarm_details['alarm_name'])
         m_delete_alarm_defination.assert_called_with(alarm_details['alarm_id'])
         m_delete_symptom_definition.assert_called_with(alarm_details['symptom_definition_id'])
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(alarm_uuid, None)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
     @mock.patch.object(monPlugin.MonPlugin, 'get_notification_rule_id_by_alarm_name')
-    def test_delete_notification_rule_successful_deletion_req_response(self,\
-                                                            m_get_notification_rule_id_by_alarm_name,\
-                                                            m_delete):
+    def test_delete_notification_rule_successful_deletion_req_response(self,
+                                                                       m_get_notification_rule_id_by_alarm_name,
+                                                                       m_delete):
         """Test delete_notification_rule: Valid notification rule is deleted & returns rule_id"""
 
-        #Set input parameters to delete_notification_rule
+        # Set input parameters to delete_notification_rule
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_notification_rule_id_by_alarm_name.return_value = '8db86441-71d8-4830-9e1a-a90be3776d12'
         m_delete.return_value.status_code = 204
 
-        #Call delete_notification_rule method under test
+        # Call delete_notification_rule method under test
         rule_id = self.mon_plugin.delete_notification_rule(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_notification_rule_id_by_alarm_name.assert_called_with(alarm_name)
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(rule_id, '8db86441-71d8-4830-9e1a-a90be3776d12')
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
     @mock.patch.object(monPlugin.MonPlugin, 'get_notification_rule_id_by_alarm_name')
-    def test_delete_notification_rule_failed_to_get_notification_rule_id(self,\
-                                                            m_get_notification_rule_id_by_alarm_name,\
-                                                            m_delete):
+    def test_delete_notification_rule_failed_to_get_notification_rule_id(self,
+                                                                         m_get_notification_rule_id_by_alarm_name,
+                                                                         m_delete):
         """Test delete_notification_rule: if notification rule is not found, returns None"""
 
-        #Set input parameters to delete_notification_rule
+        # Set input parameters to delete_notification_rule
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_notification_rule_id_by_alarm_name.return_value = None
 
-        #Call delete_notification_rule method under test
+        # Call delete_notification_rule method under test
         rule_id = self.mon_plugin.delete_notification_rule(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_notification_rule_id_by_alarm_name.assert_called_with(alarm_name)
         m_delete.assert_not_called()
 
         # verify return value with expected value
         self.assertEqual(rule_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
     @mock.patch.object(monPlugin.MonPlugin, 'get_notification_rule_id_by_alarm_name')
-    def test_delete_notification_rule_invalid_deletion_req_response(self,\
-                                                            m_get_notification_rule_id_by_alarm_name,\
-                                                            m_delete):
+    def test_delete_notification_rule_invalid_deletion_req_response(self,
+                                                                    m_get_notification_rule_id_by_alarm_name,
+                                                                    m_delete):
         """Test delete_notification_rule: If an invalid response is received, returns None"""
 
-        #Set input parameters to delete_notification_rule
+        # Set input parameters to delete_notification_rule
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get_notification_rule_id_by_alarm_name.return_value = '8db86441-71d8-4830-9e1a-a90be3776d12'
         m_delete.return_value.status_code = 404
 
-        #Call delete_notification_rule method under test
+        # Call delete_notification_rule method under test
         rule_id = self.mon_plugin.delete_notification_rule(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get_notification_rule_id_by_alarm_name.assert_called_with(alarm_name)
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(rule_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_notification_rule_id_by_alarm_name_valid_req_response(self,m_get):
+    def test_get_notification_rule_id_by_alarm_name_valid_req_response(self, m_get):
         """Test get_notification_rule_id_by_alarm_name: A valid request response received,
             returns notification_id
         """
 
-        #Set input parameters to get_notification_rule_id_by_alarm_name
+        # Set input parameters to get_notification_rule_id_by_alarm_name
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{\
         "pageInfo": {"totalCount": 0,"page": 0,"pageSize": 1000},\
@@ -1764,48 +1718,46 @@ class TestMonPlugin(unittest.TestCase):
         "links": [{"href": "/suite-api/api/notifications/rules/2b86fa23-0c15-445c-a2b1-7bd725c46f59",\
         "rel": "SELF","name": "linkToSelf"}]}]}'
 
-        #Call get_notification_rule_id_by_alarm_name method under test
+        # Call get_notification_rule_id_by_alarm_name method under test
         notification_id = self.mon_plugin.get_notification_rule_id_by_alarm_name(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(notification_id, '2b86fa23-0c15-445c-a2b1-7bd725c46f59')
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_notification_rule_id_by_alarm_name_invalid_req_response(self,m_get):
+    def test_get_notification_rule_id_by_alarm_name_invalid_req_response(self, m_get):
         """Test get_notification_rule_id_by_alarm_name: If an invalid response received,\
             returns None
         """
 
-        #Set input parameters to delete_alarm_configuration
+        # Set input parameters to delete_alarm_configuration
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 404
 
-        #Call get_notification_rule_id_by_alarm_name method under test
+        # Call get_notification_rule_id_by_alarm_name method under test
         notification_id = self.mon_plugin.get_notification_rule_id_by_alarm_name(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(notification_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
-    def test_get_notification_rule_id_by_alarm_name_rule_not_found(self,m_get):
+    def test_get_notification_rule_id_by_alarm_name_rule_not_found(self, m_get):
         """Test get_notification_rule_id_by_alarm_name: If a notification rule is not found,
             returns None
         """
 
-        #Set input parameters to delete_alarm_configuration
+        # Set input parameters to delete_alarm_configuration
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_get.return_value.status_code = 200
         m_get.return_value.content = '{\
         "pageInfo": {"totalCount": 0,"page": 0,"pageSize": 1000},\
@@ -1823,104 +1775,99 @@ class TestMonPlugin(unittest.TestCase):
         "links": [{"href": "/suite-api/api/notifications/rules/2b86fa23-0c15-445c-a2b1-7bd725c46f59",\
         "rel": "SELF","name": "linkToSelf"}]}]}'
 
-        #Call get_notification_rule_id_by_alarm_name method under test
+        # Call get_notification_rule_id_by_alarm_name method under test
         notification_id = self.mon_plugin.get_notification_rule_id_by_alarm_name(alarm_name)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_get.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(notification_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
-    def test_delete_alarm_defination_valid_req_response(self,m_delete):
+    def test_delete_alarm_defination_valid_req_response(self, m_delete):
         """Test delete_alarm_defination: A valid request response received,
             returns symptom_id
         """
 
-        #Set input parameters to delete_alarm_definition
+        # Set input parameters to delete_alarm_definition
         alarm_definition_id = 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_delete.return_value.status_code = 204
 
-        #Call delete_alarm_defination method under test
+        # Call delete_alarm_defination method under test
         actual_alarm_id = self.mon_plugin.delete_alarm_defination(alarm_definition_id)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(actual_alarm_id, alarm_definition_id)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
-    def test_delete_alarm_defination_invalid_req_response(self,m_delete):
+    def test_delete_alarm_defination_invalid_req_response(self, m_delete):
         """Test delete_alarm_defination: If an invalid request response received,
             returns None
         """
 
-        #Set input parameters to delete_alarm_definition
+        # Set input parameters to delete_alarm_definition
         alarm_definition_id = 'AlertDefinition-9a6d8a14-9f25-4d81-4d773497444d'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_delete.return_value.status_code = 404
 
-        #Call delete_alarm_defination method under test
+        # Call delete_alarm_defination method under test
         actual_alarm_id = self.mon_plugin.delete_alarm_defination(alarm_definition_id)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(actual_alarm_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
-    def test_delete_symptom_definition_valid_req_response(self,m_delete):
+    def test_delete_symptom_definition_valid_req_response(self, m_delete):
         """Test delete_symptom_definition: A valid request response received,
             returns symptom_id
         """
 
-        #Set input parameters to delete_symptom_definition
+        # Set input parameters to delete_symptom_definition
         symptom_definition_id = 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_delete.return_value.status_code = 204
 
-        #Call delete_symptom_definition method under test
+        # Call delete_symptom_definition method under test
         actual_symptom_id = self.mon_plugin.delete_symptom_definition(symptom_definition_id)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(actual_symptom_id, symptom_definition_id)
 
-
     @mock.patch.object(monPlugin.requests, 'delete')
-    def test_delete_symptom_definition_invalid_req_response(self,m_delete):
+    def test_delete_symptom_definition_invalid_req_response(self, m_delete):
         """Test delete_symptom_definition: If an invalid request response received,
             returns None
         """
 
-        #Set input parameters to delete_symptom_definition
+        # Set input parameters to delete_symptom_definition
         symptom_definition_id = 'SymptomDefinition-bcc2cb36-a67b-4deb-9b5884973278'
 
-        #Set mocked function's return values
+        # Set mocked function's return values
         m_delete.return_value.status_code = 404
 
-        #Call delete_symptom_definition method under test
+        # Call delete_symptom_definition method under test
         actual_symptom_id = self.mon_plugin.delete_symptom_definition(symptom_definition_id)
 
-        #Verify that mocked method is called with required parameters
+        # Verify that mocked method is called with required parameters
         m_delete.assert_called()
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(actual_symptom_id, None)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
     def test_configure_rest_plugin_valid_plugin_id(self, m_check_if_plugin_configured, m_post):
@@ -1939,16 +1886,15 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
-    @mock.patch.object(monPlugin.MonPlugin,'enable_rest_plugin')
+    @mock.patch.object(monPlugin.MonPlugin, 'enable_rest_plugin')
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
     def est_configure_rest_plugin_invalid_plugin_id(self, m_check_if_plugin_configured, m_post, m_enable_rest_plugin):
         """Test configure rest plugin method-invalid plugin id"""
 
         # mock return values
-        m_check_if_plugin_configured.return_value = None # not configured
-        m_post.return_value.status_code = 201 #success
+        m_check_if_plugin_configured.return_value = None  # not configured
+        m_post.return_value.status_code = 201  # success
         m_post.return_value.content = '{"pluginTypeId":"RestPlugin","pluginId":"1ef15663-9739-49fe-8c41-022bcc9f690c",\
                                         "name":"MON_module_REST_Plugin","version":1518693747871,"enabled":false,\
                                         "configValues":[{"name":"Url","value":"https://MON.lxd:8080/notify/"},\
@@ -1956,7 +1902,7 @@ class TestMonPlugin(unittest.TestCase):
                                         "value":"AA:E7:3E:A5:34:E0:25:FB:28:84:3B:74:B2:18:74:C0:C3:E8:26:50"},\
                                         {"name":"ConnectionCount","value":"20"}]}'
 
-        m_enable_rest_plugin.return_value = True #success
+        m_enable_rest_plugin.return_value = True  # success
         expected_return = '1ef15663-9739-49fe-8c41-022bcc9f690c'
 
         # call configure rest plugin method under test
@@ -1965,21 +1911,21 @@ class TestMonPlugin(unittest.TestCase):
         # verify that mocked method is called
         m_check_if_plugin_configured.assert_called()
         m_post.assert_called()
-        m_enable_rest_plugin.assert_called_with('1ef15663-9739-49fe-8c41-022bcc9f690c','MON_module_REST_Plugin')
+        m_enable_rest_plugin.assert_called_with('1ef15663-9739-49fe-8c41-022bcc9f690c', 'MON_module_REST_Plugin')
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
-    @mock.patch.object(monPlugin.MonPlugin,'enable_rest_plugin')
+    @mock.patch.object(monPlugin.MonPlugin, 'enable_rest_plugin')
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
-    def est_configure_rest_plugin_failed_to_enable_plugin(self, m_check_if_plugin_configured, m_post, m_enable_rest_plugin):
+    def est_configure_rest_plugin_failed_to_enable_plugin(self, m_check_if_plugin_configured, m_post,
+                                                          m_enable_rest_plugin):
         """Test configure rest plugin method-failed to enable plugin case"""
 
         # mock return values
-        m_check_if_plugin_configured.return_value = None # not configured
-        m_post.return_value.status_code = 201 #success
+        m_check_if_plugin_configured.return_value = None  # not configured
+        m_post.return_value.status_code = 201  # success
         m_post.return_value.content = '{"pluginTypeId":"RestPlugin","pluginId":"1ef15663-9739-49fe-8c41-022bcc9f690c",\
                                         "name":"MON_module_REST_Plugin","version":1518693747871,"enabled":false,\
                                         "configValues":[{"name":"Url","value":"https://MON.lxd:8080/notify/"},\
@@ -1987,7 +1933,7 @@ class TestMonPlugin(unittest.TestCase):
                                         "value":"AA:E7:3E:A5:34:E0:25:FB:28:84:3B:74:B2:18:74:C0:C3:E8:26:50"},\
                                         {"name":"ConnectionCount","value":"20"}]}'
 
-        m_enable_rest_plugin.return_value = False #return failure
+        m_enable_rest_plugin.return_value = False  # return failure
         expected_return = None
 
         # call configure rest plugin method under test
@@ -1996,12 +1942,11 @@ class TestMonPlugin(unittest.TestCase):
         # verify that mocked method is called
         m_check_if_plugin_configured.assert_called()
         m_post.assert_called()
-        m_enable_rest_plugin.assert_called_with('1ef15663-9739-49fe-8c41-022bcc9f690c','MON_module_REST_Plugin')
+        m_enable_rest_plugin.assert_called_with('1ef15663-9739-49fe-8c41-022bcc9f690c', 'MON_module_REST_Plugin')
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_check_if_plugin_configured_valid_req_response(self, m_get):
         """Test check if plugin configured method-valid request response"""
@@ -2030,7 +1975,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_check_if_plugin_configured_invalid_req_response(self, m_get):
         """Test check if plugin configured method-invalid request response"""
@@ -2059,7 +2003,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     def test_enable_rest_plugin_valid_req_response(self, m_put):
         """Test enable rest plugin method-valid request response"""
@@ -2080,15 +2023,14 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'put')
     def test_enable_rest_plugin_invalid_req_response(self, m_put):
         """Test enable rest plugin method-invalid request response"""
 
         plugin_name = 'MON_module_REST_Plugin'
-        plugin_id = '08018c0f-8879-4ca1-9b92-00e22d2ff81b' #invalid plugin id
+        plugin_id = '08018c0f-8879-4ca1-9b92-00e22d2ff81b'  # invalid plugin id
         # mock return values
-        m_put.return_value.status_code = 404 # api Error code
+        m_put.return_value.status_code = 404  # api Error code
         expected_return = False
         m_put.return_value.content = '<?xml version="1.0" encoding="UTF-8" standalone="yes"?><ops:\
                                       error xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" \
@@ -2107,7 +2049,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
     def test_create_alarm_notification_rule_valid_req(self, m_check_if_plugin_configured, m_post):
@@ -2141,7 +2082,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
     def test_create_alarm_notification_rule_invalid_req(self, m_check_if_plugin_configured, m_post):
@@ -2150,7 +2090,7 @@ class TestMonPlugin(unittest.TestCase):
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_id = 'AlertDefinition-f1163767-6eac-438f-8e60-a7a867257e14'
         res_id = 'ac87622f-b761-40a0-b151-00872a2a456e'
-        expected_return = None # invalid req should retrun none
+        expected_return = None  # invalid req should retrun none
 
         # mock return values
         m_check_if_plugin_configured.return_value = '03053f51-f829-438d-993d-cc33a435d76a'
@@ -2172,17 +2112,16 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'post')
     @mock.patch.object(monPlugin.MonPlugin, 'check_if_plugin_configured')
-    def test_create_alarm_notification_rule_failed_to_get_plugin_id(self, \
-                                    m_check_if_plugin_configured, m_post):
+    def test_create_alarm_notification_rule_failed_to_get_plugin_id(self,
+                                                                    m_check_if_plugin_configured, m_post):
         """Test create alarm notification rule method invalid plugin id"""
 
         alarm_name = 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         alarm_id = 'AlertDefinition-f1163767-6eac-438f-8e60-a7a867257e14'
         res_id = 'ac87622f-b761-40a0-b151-00872a2a456e'
-        expected_return = None # invalid req should retrun none
+        expected_return = None  # invalid req should retrun none
 
         # mock return values
         m_check_if_plugin_configured.return_value = None
@@ -2197,22 +2136,21 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_get_metrics_data_valid_rest_req_response(self, m_get_default_Params, \
-                                                      m_get_vm_moref_id, \
-                                                      m_get_vm_resource_id, \
+    def test_get_metrics_data_valid_rest_req_response(self, m_get_default_Params,
+                                                      m_get_vm_moref_id,
+                                                      m_get_vm_resource_id,
                                                       m_get):
         """Test get metrics data of resource method valid request response"""
 
-        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None, \
-                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',\
-                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef', \
-                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware', \
-                   'collection_unit': 'HR', 'vim_uuid':'1'}
+        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None,
+                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
+                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware',
+                   'collection_unit': 'HR', 'vim_uuid': '1'}
 
         # mock return value
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
@@ -2241,24 +2179,23 @@ class TestMonPlugin(unittest.TestCase):
         m_get.assert_called()
 
         # verify return value with expected value
-        #self.assertEqual(expected_return, actual_return)
-
+        # self.assertEqual(expected_return, actual_return)
 
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_get_metrics_data_invalid_rest_req_response(self, m_get_default_Params, \
-                                                      m_get_vm_moref_id, \
-                                                      m_get_vm_resource_id, \
-                                                      m_get):
+    def test_get_metrics_data_invalid_rest_req_response(self, m_get_default_Params,
+                                                        m_get_vm_moref_id,
+                                                        m_get_vm_resource_id,
+                                                        m_get):
         """Test get metrics data of resource method invalid request response"""
 
-        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None, \
-                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',\
-                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef', \
-                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware', \
-                   'collection_unit': 'HR', 'vim_uuid':'1'}
+        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None,
+                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
+                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware',
+                   'collection_unit': 'HR', 'vim_uuid': '1'}
 
         # mock return value
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
@@ -2271,7 +2208,7 @@ class TestMonPlugin(unittest.TestCase):
                            'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                            'metrics_data': {'time_series': [], 'metrics_series': []},
                            'schema_type': 'read_metric_data_response',
-                            'unit': '%', 'vim_uuid':'1'}
+                           'unit': '%', 'vim_uuid': '1'}
 
         # call get metrics data method under test
         actual_return = self.mon_plugin.get_metrics_data(metrics)
@@ -2290,14 +2227,13 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_get_metrics_data_metric_not_supported(self, m_get_default_Params, \
-                                                   m_get_vm_moref_id, \
-                                                   m_get_vm_resource_id, \
+    def test_get_metrics_data_metric_not_supported(self, m_get_default_Params,
+                                                   m_get_vm_moref_id,
+                                                   m_get_vm_resource_id,
                                                    m_get):
         """Test get metrics data of resource method invalid metric name"""
 
@@ -2306,16 +2242,16 @@ class TestMonPlugin(unittest.TestCase):
                    'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                    'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                    'schema_type': 'read_metric_data_request', 'vim_type': 'VMware',
-                   'collection_unit': 'HR', 'vim_uuid':'1'}
+                   'collection_unit': 'HR', 'vim_uuid': '1'}
 
         # mock return value
-        m_get_default_Params.return_value = {} # returns empty dict
+        m_get_default_Params.return_value = {}  # returns empty dict
 
-        expected_return = {'metric_name': 'invalid_metric', 'metric_uuid': '0','vim_uuid': '1',
-                           'schema_version': '1.0','resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+        expected_return = {'metric_name': 'invalid_metric', 'metric_uuid': '0', 'vim_uuid': '1',
+                           'schema_version': '1.0', 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
                            'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                            'metrics_data': {'time_series': [], 'metrics_series': []},
-                           'schema_type': 'read_metric_data_response','unit': None}
+                           'schema_type': 'read_metric_data_response', 'unit': None}
 
         # call get matrics data method under test
         actual_return = self.mon_plugin.get_metrics_data(metrics)
@@ -2329,22 +2265,21 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_get_metrics_data_failed_to_get_vm_moref_id(self, m_get_default_Params, \
-                                                        m_get_vm_moref_id, \
-                                                        m_get_vm_resource_id, \
+    def test_get_metrics_data_failed_to_get_vm_moref_id(self, m_get_default_Params,
+                                                        m_get_vm_moref_id,
+                                                        m_get_vm_resource_id,
                                                         m_get):
         """Test get metrics data method negative scenario- invalid resource id"""
 
-        metrics = {'collection_period': 1, 'metric_name': 'cpu_utilization', 'metric_uuid': None, \
-                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',\
-                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef', \
-                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware', \
-                   'collection_unit': 'HR', 'vim_uuid':'1'}
+        metrics = {'collection_period': 1, 'metric_name': 'cpu_utilization', 'metric_uuid': None,
+                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
+                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware',
+                   'collection_unit': 'HR', 'vim_uuid': '1'}
 
         # mock return value
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
@@ -2355,7 +2290,7 @@ class TestMonPlugin(unittest.TestCase):
                            'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                            'metrics_data': {'time_series': [], 'metrics_series': []},
                            'schema_type': 'read_metric_data_response',
-                            'unit': '%', 'vim_uuid':'1'}
+                           'unit': '%', 'vim_uuid': '1'}
 
         # call get matrics data method under test
         actual_return = self.mon_plugin.get_metrics_data(metrics)
@@ -2369,22 +2304,21 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_get_metrics_data_failed_to_get_vm_resource_id(self, m_get_default_Params, \
-                                                           m_get_vm_moref_id, \
-                                                           m_get_vm_resource_id, \
+    def test_get_metrics_data_failed_to_get_vm_resource_id(self, m_get_default_Params,
+                                                           m_get_vm_moref_id,
+                                                           m_get_vm_resource_id,
                                                            m_get):
         """Test get metrics data method negative scenario- invalid moref id"""
 
-        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None, \
-                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',\
-                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef', \
-                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware', \
-                   'collection_unit': 'HR', 'vim_uuid':'1'}
+        metrics = {'collection_period': 1, 'metric_name': 'CPU_UTILIZATION', 'metric_uuid': None,
+                   'schema_version': 1.0, 'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4',
+                   'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
+                   'schema_type': 'read_metric_data_request', 'vim_type': 'VMware',
+                   'collection_unit': 'HR', 'vim_uuid': '1'}
 
         # mock return value
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
@@ -2396,7 +2330,7 @@ class TestMonPlugin(unittest.TestCase):
                            'correlation_id': 'e14b203c-6bf2-4e2f-a91c-8c19d2abcdef',
                            'metrics_data': {'time_series': [], 'metrics_series': []},
                            'schema_type': 'read_metric_data_response',
-                            'unit': '%', 'vim_uuid':'1'}
+                           'unit': '%', 'vim_uuid': '1'}
 
         # call get matrics data method under test
         actual_return = self.mon_plugin.get_metrics_data(metrics)
@@ -2411,13 +2345,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'reconfigure_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'update_symptom_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_update_alarm_configuration_successful_updation(self, m_get_alarm_defination_details, \
-                                                            m_update_symptom_defination, \
-                                                            m_reconfigure_alarm ):
+    def test_update_alarm_configuration_successful_updation(self, m_get_alarm_defination_details,
+                                                            m_update_symptom_defination,
+                                                            m_reconfigure_alarm):
         """Test update alarm configuration method"""
 
         alarm_config = {'alarm_uuid': 'f1163767-6eac-438f-8e60-a7a867257e14',
@@ -2426,9 +2359,12 @@ class TestMonPlugin(unittest.TestCase):
 
         # mock return value
         alarm_details_json = {'states': [{'impact': {'impactType': 'BADGE', 'detail': 'risk'},
-                              'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
-                              ['SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
-                              'relation': 'SELF', 'type': 'SYMPTOM_SET', 'aggregation': 'ALL'}}],
+                                          'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
+                [
+                    'SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
+                'relation': 'SELF',
+                'type': 'SYMPTOM_SET',
+                'aggregation': 'ALL'}}],
                               'description': 'CPU_Utilization_Above_Threshold', 'type': 16,
                               'id': 'AlertDefinition-f1163767-6eac-438f-8e60-a7a867257e14',
                               'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d2'}
@@ -2446,21 +2382,20 @@ class TestMonPlugin(unittest.TestCase):
 
         # verify that mocked method is called
         m_get_alarm_defination_details.assert_called_with(alarm_config['alarm_uuid'])
-        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],\
+        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],
                                                        alarm_config)
         m_reconfigure_alarm.assert_called_with(alarm_details_json, alarm_config)
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'reconfigure_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'update_symptom_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_update_alarm_configuration_failed_to_reconfigure_alarm(self, \
-                                                        m_get_alarm_defination_details, \
-                                                        m_update_symptom_defination, \
-                                                        m_reconfigure_alarm ):
+    def test_update_alarm_configuration_failed_to_reconfigure_alarm(self,
+                                                                    m_get_alarm_defination_details,
+                                                                    m_update_symptom_defination,
+                                                                    m_reconfigure_alarm):
         """Test update alarm configuration method- failed to reconfigure alarm"""
 
         alarm_config = {'alarm_uuid': 'f1163767-6eac-438f-8e60-a7a867257e14',
@@ -2469,9 +2404,12 @@ class TestMonPlugin(unittest.TestCase):
 
         # mock return value
         alarm_details_json = {'states': [{'impact': {'impactType': 'BADGE', 'detail': 'risk'},
-                              'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
-                              ['SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
-                              'relation': 'SELF', 'type': 'SYMPTOM_SET', 'aggregation': 'ALL'}}],
+                                          'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
+                [
+                    'SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
+                'relation': 'SELF',
+                'type': 'SYMPTOM_SET',
+                'aggregation': 'ALL'}}],
                               'description': 'CPU_Utilization_Above_Threshold', 'type': 16,
                               'id': 'AlertDefinition-f1163767-6eac-438f-8e60-a7a867257e14',
                               'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d2'}
@@ -2482,28 +2420,27 @@ class TestMonPlugin(unittest.TestCase):
         m_get_alarm_defination_details.return_value = (alarm_details_json, alarm_details)
         m_update_symptom_defination.return_value = 'SymptomDefinition-47c88675-bea8-436a-bb41-\
                                                     8d2231428f44'
-        expected_return = m_reconfigure_alarm.return_value = None # failed to reconfigure
+        expected_return = m_reconfigure_alarm.return_value = None  # failed to reconfigure
 
         # call update alarm configuration method under test
         actual_return = self.mon_plugin.update_alarm_configuration(alarm_config)
 
         # verify that mocked method is called
         m_get_alarm_defination_details.assert_called_with(alarm_config['alarm_uuid'])
-        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],\
+        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],
                                                        alarm_config)
         m_reconfigure_alarm.assert_called_with(alarm_details_json, alarm_config)
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'reconfigure_alarm')
     @mock.patch.object(monPlugin.MonPlugin, 'update_symptom_defination')
     @mock.patch.object(monPlugin.MonPlugin, 'get_alarm_defination_details')
-    def test_update_alarm_configuration_failed_to_update_symptom(self, \
-                                                        m_get_alarm_defination_details, \
-                                                        m_update_symptom_defination, \
-                                                        m_reconfigure_alarm ):
+    def test_update_alarm_configuration_failed_to_update_symptom(self,
+                                                                 m_get_alarm_defination_details,
+                                                                 m_update_symptom_defination,
+                                                                 m_reconfigure_alarm):
         """Test update alarm configuration method- failed to update alarm"""
 
         alarm_config = {'alarm_uuid': 'f1163767-6eac-438f-8e60-a7a867257e14',
@@ -2512,9 +2449,12 @@ class TestMonPlugin(unittest.TestCase):
 
         # mock return value
         alarm_details_json = {'states': [{'impact': {'impactType': 'BADGE', 'detail': 'risk'},
-                              'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
-                              ['SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
-                              'relation': 'SELF', 'type': 'SYMPTOM_SET', 'aggregation': 'ALL'}}],
+                                          'severity': 'CRITICAL', 'base-symptom-set': {'symptomDefinitionIds':
+                [
+                    'SymptomDefinition-47c88675-bea8-436a-bb41-8d2231428f44'],
+                'relation': 'SELF',
+                'type': 'SYMPTOM_SET',
+                'aggregation': 'ALL'}}],
                               'description': 'CPU_Utilization_Above_Threshold', 'type': 16,
                               'id': 'AlertDefinition-f1163767-6eac-438f-8e60-a7a867257e14',
                               'name': 'CPU_Utilization_Above_Thr-e14b203c-6bf2-4e2f-a91c-8c19d2'}
@@ -2530,44 +2470,42 @@ class TestMonPlugin(unittest.TestCase):
 
         # verify that mocked method is called
         m_get_alarm_defination_details.assert_called_with(alarm_config['alarm_uuid'])
-        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],\
+        m_update_symptom_defination.assert_called_with(alarm_details['symptom_definition_id'],
                                                        alarm_config)
         m_reconfigure_alarm.assert_not_called()
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_verify_metric_support_metric_supported_with_unit(self,m_get_default_Params):
+    def test_verify_metric_support_metric_supported_with_unit(self, m_get_default_Params):
         """Test verify metric support method for supported metric"""
 
         # mock return value
         metric_info = {'metric_unit': '%', 'metric_name': 'cpu_utilization',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'}
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
-        expected_return = True #supported metric returns True
+        expected_return = True  # supported metric returns True
 
         # call verify metric support method under test
         actual_return = self.mon_plugin.verify_metric_support(metric_info)
 
         # verify that mocked method is called
         m_get_default_Params.assert_called_with(metric_info['metric_name'])
-        #m_get_default_Params.assert_called_with(metric_info)
+        # m_get_default_Params.assert_called_with(metric_info)
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_verify_metric_support_metric_not_supported(self,m_get_default_Params):
+    def test_verify_metric_support_metric_not_supported(self, m_get_default_Params):
         """Test verify metric support method for un-supported metric"""
 
         # mock return value
         metric_info = {'metric_unit': '%', 'metric_name': 'invalid_metric',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'}
         m_get_default_Params.return_value = {}
-        expected_return = False #supported metric returns True
+        expected_return = False  # supported metric returns True
 
         # call verify metric support method under test
         actual_return = self.mon_plugin.verify_metric_support(metric_info)
@@ -2578,17 +2516,16 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_default_Params')
-    def test_verify_metric_support_metric_supported_with_mismatched_unit(self, \
-                                                               m_get_default_Params):
+    def test_verify_metric_support_metric_supported_with_mismatched_unit(self,
+                                                                         m_get_default_Params):
         """Test verify metric support method for supported metric with mismatched unit"""
 
         # mock return value
         metric_info = {'metric_unit': '', 'metric_name': 'invalid_metric',
                        'resource_uuid': 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'}
         m_get_default_Params.return_value = {'metric_key': 'cpu|usage_average', 'unit': '%'}
-        expected_return = True #supported metric returns True
+        expected_return = True  # supported metric returns True
 
         # call verify metric support method under test
         actual_return = self.mon_plugin.verify_metric_support(metric_info)
@@ -2599,12 +2536,11 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_triggered_alarms_on_resource')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vrops_resourceid_from_ro_uuid')
-    def test_get_triggered_alarms_list_returns_triggered_alarms(self, \
-                                                        m_get_vrops_resourceid, \
-                                                        m_triggered_alarms):
+    def test_get_triggered_alarms_list_returns_triggered_alarms(self,
+                                                                m_get_vrops_resourceid,
+                                                                m_triggered_alarms):
         """Test get triggered alarm list method valid input"""
 
         # Mock list alarm input
@@ -2615,44 +2551,44 @@ class TestMonPlugin(unittest.TestCase):
 
         resource_id = m_get_vrops_resourceid.return_value = 'ac87622f-b761-40a0-b151-00872a2a456e'
         expected_return = m_triggered_alarms.return_value = [{'status': 'ACTIVE',
-                                           'update_date': '2018-01-12T08:34:05',
-                                           'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
-                                           'cancel_date': '0000-00-00T00:00:00',
-                                           'alarm_instance_uuid': 'd9e3bc84',
-                                           'alarm_uuid': '5714977d', 'vim_type': 'VMware',
-                                           'start_date': '2018-01-12T08:34:05'},
-                                          {'status': 'CANCELED','update_date':'2017-12-20T09:37:57',
-                                           'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
-                                           'cancel_date': '2018-01-12T06:49:19',
-                                           'alarm_instance_uuid': 'd3bbeef6',
-                                           'alarm_uuid': '7ba1bf3e', 'vim_type': 'VMware',
-                                           'start_date': '2017-12-20T09:37:57'}]
+                                                              'update_date': '2018-01-12T08:34:05',
+                                                              'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
+                                                              'cancel_date': '0000-00-00T00:00:00',
+                                                              'alarm_instance_uuid': 'd9e3bc84',
+                                                              'alarm_uuid': '5714977d', 'vim_type': 'VMware',
+                                                              'start_date': '2018-01-12T08:34:05'},
+                                                             {'status': 'CANCELED',
+                                                              'update_date': '2017-12-20T09:37:57',
+                                                              'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
+                                                              'cancel_date': '2018-01-12T06:49:19',
+                                                              'alarm_instance_uuid': 'd3bbeef6',
+                                                              'alarm_uuid': '7ba1bf3e', 'vim_type': 'VMware',
+                                                              'start_date': '2017-12-20T09:37:57'}]
 
         # call get triggered alarms list method under test
         actual_return = self.mon_plugin.get_triggered_alarms_list(list_alarm_input)
 
         # verify that mocked method is called
         m_get_vrops_resourceid.assert_called_with(list_alarm_input['resource_uuid'])
-        m_triggered_alarms.assert_called_with(list_alarm_input['resource_uuid'] , resource_id)
+        m_triggered_alarms.assert_called_with(list_alarm_input['resource_uuid'], resource_id)
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_triggered_alarms_on_resource')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vrops_resourceid_from_ro_uuid')
-    def test_get_triggered_alarms_list_invalid_resource_uuid(self, \
-                                                        m_get_vrops_resourceid, \
-                                                        m_triggered_alarms):
+    def test_get_triggered_alarms_list_invalid_resource_uuid(self,
+                                                             m_get_vrops_resourceid,
+                                                             m_triggered_alarms):
         """Test get triggered alarm list method invalid resource uuid"""
 
         # Mock list alarm input
         list_alarm_input = {'severity': 'CRITICAL',
                             'correlation_id': 'e14b203c',
                             'alarm_name': 'CPU_Utilization_Above_Threshold',
-                            'resource_uuid': '12345'} #invalid resource uuid
+                            'resource_uuid': '12345'}  # invalid resource uuid
 
-        expected_return = m_get_vrops_resourceid.return_value = None #returns empty list
+        expected_return = m_get_vrops_resourceid.return_value = None  # returns empty list
 
         # call get triggered alarms list method under test
         actual_return = self.mon_plugin.get_triggered_alarms_list(list_alarm_input)
@@ -2664,12 +2600,11 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual([], actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_triggered_alarms_on_resource')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vrops_resourceid_from_ro_uuid')
-    def test_get_triggered_alarms_list_resource_uuid_not_present(self, \
-                                                        m_get_vrops_resourceid, \
-                                                        m_triggered_alarms):
+    def test_get_triggered_alarms_list_resource_uuid_not_present(self,
+                                                                 m_get_vrops_resourceid,
+                                                                 m_triggered_alarms):
         """Test get triggered alarm list method resource not present"""
 
         # Mock list alarm input
@@ -2687,7 +2622,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual([], actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
     def test_get_vrops_resourceid_from_ro_uuid(self, m_get_vm_moref_id, m_get_vm_resource_id):
@@ -2696,7 +2630,7 @@ class TestMonPlugin(unittest.TestCase):
         # Mock the inputs
         ro_resource_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         vm_moref_id = m_get_vm_moref_id.return_value = 'vm-6626'
-        expected_return = m_get_vm_resource_id.return_value ='ac87622f-b761-40a0-b151-00872a2a456e'
+        expected_return = m_get_vm_resource_id.return_value = 'ac87622f-b761-40a0-b151-00872a2a456e'
 
         # call get_vrops_resourceid_from_ro_uuid method under test
         actual_return = self.mon_plugin.get_vrops_resourceid_from_ro_uuid(ro_resource_uuid)
@@ -2708,12 +2642,11 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
-    def test_get_vrops_resourceid_from_ro_uuid_failed_to_get_vm_resource_id(self, \
-                                                                    m_get_vm_moref_id, \
-                                                                    m_get_vm_resource_id):
+    def test_get_vrops_resourceid_from_ro_uuid_failed_to_get_vm_resource_id(self,
+                                                                            m_get_vm_moref_id,
+                                                                            m_get_vm_resource_id):
         """Test get vrops resourceid from ro uuid method negative scenario"""
 
         # Mock the inputs
@@ -2731,12 +2664,11 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_resource_id')
     @mock.patch.object(monPlugin.MonPlugin, 'get_vm_moref_id')
-    def test_get_vrops_resourceid_from_ro_uuid_failed_to_get_vm_moref_id(self, \
-                                                                    m_get_vm_moref_id, \
-                                                                    m_get_vm_resource_id):
+    def test_get_vrops_resourceid_from_ro_uuid_failed_to_get_vm_moref_id(self,
+                                                                         m_get_vm_moref_id,
+                                                                         m_get_vm_resource_id):
         """Test get vrops resourceid from ro uuid method negative scenario"""
 
         # Mock the inputs
@@ -2753,7 +2685,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_triggered_alarms_on_resource_valid_req_response(self, m_get):
         """Test get triggered alarms on resource method for valid request"""
@@ -2794,8 +2725,8 @@ class TestMonPlugin(unittest.TestCase):
         }]}'
 
         # call get_vrops_resourceid_from_ro_uuid method under test
-        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid, \
-                                                                        vrops_resource_id)
+        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid,
+                                                                         vrops_resource_id)
 
         # verify that mocked method is called
         m_get.assert_called()
@@ -2803,7 +2734,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_triggered_alarms_on_resource_invalid_req_response(self, m_get):
         """Test get triggered alarms on resource method for invalid request"""
@@ -2815,8 +2745,8 @@ class TestMonPlugin(unittest.TestCase):
         expected_return = None
 
         # call get_vrops_resourceid_from_ro_uuid method under test
-        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid, \
-                                                                        vrops_resource_id)
+        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid,
+                                                                         vrops_resource_id)
 
         # verify that mocked method is called
         m_get.assert_called()
@@ -2824,7 +2754,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_triggered_alarms_on_resource_no_alarms_present(self, m_get):
         """Test get triggered alarms on resource method for no alarms present"""
@@ -2837,8 +2766,8 @@ class TestMonPlugin(unittest.TestCase):
         m_get.return_value.content = '{"alerts": []}'
 
         # call get_vrops_resourceid_from_ro_uuid method under test
-        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid, \
-                                                                        vrops_resource_id)
+        actual_return = self.mon_plugin.get_triggered_alarms_on_resource(ro_resource_uuid,
+                                                                         vrops_resource_id)
 
         # verify that mocked method is called
         m_get.assert_called()
@@ -2846,7 +2775,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     def test_convert_date_time_valid_date_time(self):
         """Test convert date time method valid input"""
 
@@ -2873,7 +2801,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_vm_resource_id_rest_valid_req_response(self, m_get):
         """Test get vms resource id valid request"""
@@ -2883,29 +2810,29 @@ class TestMonPlugin(unittest.TestCase):
         m_get.return_value.status_code = 200
         expected_return = "ac87622f-b761-40a0-b151-00872a2a456e"
         m_get.return_value.content = \
-        '{ \
-            "resourceList": [\
-               {\
-                   "creationTime": 1497770174130,\
-                   "resourceKey": {\
-                       "name": "OCInst2.ubuntu(4337d51f-1e65-4ab0-9c08-4897778d4fda)",\
-                       "adapterKindKey": "VMWARE",\
-                       "resourceKindKey": "VirtualMachine",\
-                       "resourceIdentifiers": [\
-                           {\
-                               "identifierType": {\
-                               "name": "VMEntityObjectID",\
-                               "dataType": "STRING",\
-                               "isPartOfUniqueness": true\
-                               },\
-                               "value": "vm-6626"\
-                           }\
-                       ]\
-                   },\
-                   "identifier": "ac87622f-b761-40a0-b151-00872a2a456e"\
-                }\
-            ]\
-        }'
+            '{ \
+                "resourceList": [\
+                   {\
+                       "creationTime": 1497770174130,\
+                       "resourceKey": {\
+                           "name": "OCInst2.ubuntu(4337d51f-1e65-4ab0-9c08-4897778d4fda)",\
+                           "adapterKindKey": "VMWARE",\
+                           "resourceKindKey": "VirtualMachine",\
+                           "resourceIdentifiers": [\
+                               {\
+                                   "identifierType": {\
+                                   "name": "VMEntityObjectID",\
+                                   "dataType": "STRING",\
+                                   "isPartOfUniqueness": true\
+                                   },\
+                                   "value": "vm-6626"\
+                               }\
+                           ]\
+                       },\
+                       "identifier": "ac87622f-b761-40a0-b151-00872a2a456e"\
+                    }\
+                ]\
+            }'
 
         # call get_vm_resource_id method under test
         actual_return = self.mon_plugin.get_vm_resource_id(vm_moref_id)
@@ -2916,7 +2843,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_vm_resource_id_rest_invalid_req_response(self, m_get):
         """Test get vms resource id invalid request"""
@@ -2936,7 +2862,6 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     def test_get_vm_resource_id_rest_invalid_response(self, m_get):
         """Test get vms resource id invalid response"""
@@ -2946,44 +2871,43 @@ class TestMonPlugin(unittest.TestCase):
         m_get.return_value.status_code = 200
         expected_return = None
         m_get.return_value.content = \
-        '{ \
-            "resourceList": \
-               {\
-                   "creationTime": 1497770174130,\
-                   "resourceKey": {\
-                       "name": "OCInst2.ubuntu(4337d51f-1e65-4ab0-9c08-4897778d4fda)",\
-                       "adapterKindKey": "VMWARE",\
-                       "resourceKindKey": "VirtualMachine",\
-                       "resourceIdentifiers": [\
-                           {\
-                               "identifierType": {\
-                               "name": "VMEntityObjectID",\
-                               "dataType": "STRING",\
-                               "isPartOfUniqueness": true\
-                               },\
-                               "value": "vm-6626"\
-                           }\
-                       ]\
-                   },\
-                   "identifier": "ac87622f-b761-40a0-b151-00872a2a456e"\
-                }\
-        }'
+            '{ \
+                "resourceList": \
+                   {\
+                       "creationTime": 1497770174130,\
+                       "resourceKey": {\
+                           "name": "OCInst2.ubuntu(4337d51f-1e65-4ab0-9c08-4897778d4fda)",\
+                           "adapterKindKey": "VMWARE",\
+                           "resourceKindKey": "VirtualMachine",\
+                           "resourceIdentifiers": [\
+                               {\
+                                   "identifierType": {\
+                                   "name": "VMEntityObjectID",\
+                                   "dataType": "STRING",\
+                                   "isPartOfUniqueness": true\
+                                   },\
+                                   "value": "vm-6626"\
+                               }\
+                           ]\
+                       },\
+                       "identifier": "ac87622f-b761-40a0-b151-00872a2a456e"\
+                    }\
+            }'
 
         # call get_vm_resource_id method under test
         actual_return = self.mon_plugin.get_vm_resource_id(vm_moref_id)
 
         # verify that mocked method is called
-        #m_get.assert_called
+        # m_get.assert_called
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_vapp_details_rest')
-    def test_get_vm_moref_id_valid_id_found (self, m_get_vapp_details_rest):
+    def test_get_vm_moref_id_valid_id_found(self, m_get_vapp_details_rest):
         """Test get vm moref id valid scenario"""
 
-        #mock the inputs
+        # mock the inputs
         vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         m_get_vapp_details_rest.return_value = {'vm_vcenter_info': {'vm_moref_id': 'vm-6626'}}
         expected_return = 'vm-6626'
@@ -2997,13 +2921,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.MonPlugin, 'get_vapp_details_rest')
     def test_get_vm_moref_id_valid_id_not_found(self, m_get_vapp_details_rest):
         """Test get vm moref id invalid scenario"""
 
-        #mock the inputs
-        vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda'#invalid uuid
+        # mock the inputs
+        vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda'  # invalid uuid
         m_get_vapp_details_rest.return_value = {}
         expected_return = None
 
@@ -3016,13 +2939,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'connect_as_admin')
     def test_get_vapp_details_rest_valid_req_response(self, m_connect_as_admin, m_get):
         """Test get vapp details rest method for valid request response"""
 
-        #mock the inputs
+        # mock the inputs
         vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         m_connect_as_admin.return_value = self.vca
         self.vca._session = self.session
@@ -3056,13 +2978,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'connect_as_admin')
     def test_get_vapp_details_rest_invalid_req_response(self, m_connect_as_admin, m_get):
         """Test get vapp details rest method for invalid request response"""
 
-        #mock the inputs
+        # mock the inputs
         vapp_uuid = 'Invalid-e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         m_connect_as_admin.return_value = self.vca
         self.vca._session = self.session
@@ -3080,13 +3001,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'connect_as_admin')
     def test_get_vapp_details_rest_failed_to_connect_vcd(self, m_connect_as_admin, m_get):
         """Test get vapp details rest method for failed to connect to vcd"""
 
-        #mock the inputs
+        # mock the inputs
         vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         m_connect_as_admin.return_value = None
         expected_return = {}
@@ -3101,13 +3021,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.requests, 'get')
     @mock.patch.object(monPlugin.MonPlugin, 'connect_as_admin')
     def test_get_vapp_details_rest_invalid_response(self, m_connect_as_admin, m_get):
         """Test get vapp details rest method for invalid response"""
 
-        #mock the inputs
+        # mock the inputs
         vapp_uuid = 'e14b203c-6bf2-4e2f-a91c-8c19d240eda4'
         m_connect_as_admin.return_value = self.vca
         self.vca._session = self.session
@@ -3140,13 +3059,12 @@ class TestMonPlugin(unittest.TestCase):
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
     @mock.patch.object(monPlugin.Client, 'set_credentials')
     @mock.patch.object(monPlugin, 'Client')
     def test_connect_as_admin(self, m_client, m_set_credentials):
         """Test connect as admin to vCD method"""
 
-        #mock the inputs and mocked returns
+        # mock the inputs and mocked returns
         expected_return = m_client.return_value = self.vca
         m_set_credentials.retrun_value = True
 
@@ -3155,14 +3073,11 @@ class TestMonPlugin(unittest.TestCase):
 
         # verify that mocked method is called
         m_client.assert_called_with(self.m_vim_access_config['vim_url'],
-                                                 verify_ssl_certs=False)
+                                    verify_ssl_certs=False)
 
         # verify return value with expected value
         self.assertEqual(expected_return, actual_return)
 
-
 # For testing purpose
-#if __name__ == '__main__':
+# if __name__ == '__main__':
 #   unittest.main()
-
-
index 1d63bd9..cc5dea9 100644 (file)
 
 """ Mock tests for VMware vROPs plugin recevier """
 
-import sys
-#sys.path.append("/root/MON/")
-
 import json
-
 import logging
-
+import os
+import sys
 import unittest
+from io import UnsupportedOperation
 
 import mock
 
-import requests
-
-import os
+# sys.path.append("/root/MON/")
 
 log = logging.getLogger(__name__)
 
-sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)),"..","..",".."))
+sys.path.append(os.path.join(os.path.dirname(os.path.abspath(__file__)), "..", "..", ".."))
 
 from osm_mon.plugins.vRealiseOps import plugin_receiver as monPluginRec
 from osm_mon.core.database import VimCredentials
@@ -68,321 +64,228 @@ class TestPluginReceiver(unittest.TestCase):
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_create_alarm_status')
     @mock.patch.object(monPluginRec.PluginReceiver, 'create_alarm')
-    def test_consume_create_alarm_request_key(self, m_create_alarm,\
-                                              m_publish_create_alarm_status):
+    def test_consume_create_alarm_request_key(self, m_create_alarm, m_publish_create_alarm_status):
         """Test functionality of consume for create_alarm_request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "alarm_request"
-        msg.key = "create_alarm_request"
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"alarm_create_request":"alarm_details"})
+        value = {"vim_uuid": vim_uuid, "alarm_create_request": "alarm_details"}
         m_create_alarm.return_value = "test_alarm_id"
 
-        config_alarm_info = json.loads(msg.value)
-
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        self.plugin_receiver.handle_alarm_requests('create_alarm_request', value, vim_uuid)
 
         # verify if create_alarm and publish methods called with correct params
-        m_create_alarm.assert_called_with(config_alarm_info)
-        m_publish_create_alarm_status.assert_called_with("test_alarm_id", config_alarm_info)
-
+        m_create_alarm.assert_called_with(value)
+        m_publish_create_alarm_status.assert_called_with("test_alarm_id", value)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_update_alarm_status')
     @mock.patch.object(monPluginRec.PluginReceiver, 'update_alarm')
-    def test_consume_update_alarm_request_key(self, m_update_alarm,\
+    def test_consume_update_alarm_request_key(self, m_update_alarm,
                                               m_publish_update_alarm_status):
         """Test functionality of consume for update_alarm_request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "alarm_request"
-        msg.key = "update_alarm_request"
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"alarm_update_request":"alarm_details"})
+        value = {"vim_uuid": vim_uuid, "alarm_update_request": "alarm_details"}
 
         # set return value to mocked method
         m_update_alarm.return_value = "test_alarm_id"
 
-        update_alarm_info = json.loads(msg.value)
-
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        self.plugin_receiver.handle_alarm_requests('update_alarm_request', value, vim_uuid)
 
         # verify update_alarm and publish method called with correct params
-        m_update_alarm.assert_called_with(update_alarm_info)
-        m_publish_update_alarm_status.assert_called_with("test_alarm_id", update_alarm_info)
-
+        m_update_alarm.assert_called_with(value)
+        m_publish_update_alarm_status.assert_called_with("test_alarm_id", value)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_delete_alarm_status')
     @mock.patch.object(monPluginRec.PluginReceiver, 'delete_alarm')
-    def test_consume_delete_alarm_request_key(self, m_delete_alarm,\
+    def test_consume_delete_alarm_request_key(self, m_delete_alarm,
                                               m_publish_delete_alarm_status):
         """Test functionality of consume for delete_alarm_request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "alarm_request"
-        msg.key = "delete_alarm_request"
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"alarm_delete_request":"alarm_details"})
+        value = {"vim_uuid": vim_uuid, "alarm_delete_request": "alarm_details"}
         m_delete_alarm.return_value = "test_alarm_id"
 
-        delete_alarm_info = json.loads(msg.value)
-
         # Call the consume method of plugin_receiver and check delete_alarm request
-        self.plugin_receiver.consume(msg,vim_uuid)
-        m_delete_alarm.assert_called_with(delete_alarm_info)
+        self.plugin_receiver.handle_alarm_requests('delete_alarm_request', value, vim_uuid)
+        m_delete_alarm.assert_called_with(value)
 
         # Check if publish method called with correct parameters
-        m_publish_delete_alarm_status.assert_called_with("test_alarm_id", delete_alarm_info)
-
+        m_publish_delete_alarm_status.assert_called_with("test_alarm_id", value)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_list_alarm_response')
     @mock.patch.object(monPluginRec.PluginReceiver, 'list_alarms')
-    def test_consume_list_alarm_request_key(self, m_list_alarms,\
+    def test_consume_list_alarm_request_key(self, m_list_alarms,
                                             m_publish_list_alarm_response):
         """ Test functionality of list alarm request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "alarm_request"
-        msg.key = "list_alarm_request"
-        test_alarm_list = [{"alarm_uuid":"alarm1_details"},{"alarm_uuid":"alarm2_details"}]
+        value = {"vim_uuid": vim_uuid, "alarm_list_request": "alarm_details"}
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"alarm_list_request":"alarm_details"})
-        m_list_alarms.return_value = test_alarm_list
+        test_alarm_list = [{"alarm_uuid": "alarm1_details"}, {"alarm_uuid": "alarm2_details"}]
 
-        list_alarms_info = json.loads(msg.value)
+        m_list_alarms.return_value = test_alarm_list
 
         # Call the consume method of plugin_receiver and check delete_alarm request
-        self.plugin_receiver.consume(msg,vim_uuid)
-        m_list_alarms.assert_called_with(list_alarms_info)
-
-        # Check if publish method called with correct parameters
-        m_publish_list_alarm_response.assert_called_with(test_alarm_list, list_alarms_info)
-
-
-    @mock.patch.object(monPluginRec.PluginReceiver, 'publish_access_update_response')
-    @mock.patch.object(monPluginRec.PluginReceiver, 'update_access_credentials')
-    def test_consume_vim_access_request_key(self, m_update_access_credentials,\
-                                            m_publish_access_update_response):
-        """Test functionality of consume for vim_access_credentials request key"""
-
-        vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "access_credentials"
-        msg.key = "vim_access_credentials"
-
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"access_config":"access_details"})
-        # set return value to mocked method
-        m_update_access_credentials.return_value = True
-
-        access_info = json.loads(msg.value)
-
-        # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
-
-        # check if mocked method called with required parameters
-        m_update_access_credentials.assert_called_with("access_details")
+        self.plugin_receiver.handle_alarm_requests('list_alarm_request', value, vim_uuid)
+        m_list_alarms.assert_called_with(value)
 
         # Check if publish method called with correct parameters
-        m_publish_access_update_response.assert_called_with(True, access_info)
-
+        m_publish_list_alarm_response.assert_called_with(test_alarm_list, value)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_create_alarm_status')
     @mock.patch.object(monPluginRec.PluginReceiver, 'create_alarm')
-    def test_consume_invalid_alarm_request_key(self, m_create_alarm,\
+    def test_consume_invalid_alarm_request_key(self, m_create_alarm,
                                                m_publish_create_alarm_status):
         """Test functionality of consume for vim_access_credentials invalid request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message with invalid alarm request key
-        msg = Message()
-        msg.topic = "alarm_request"
-        msg.key = "invalid_alarm_request" # invalid key
 
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        with self.assertRaises(UnsupportedOperation):
+            self.plugin_receiver.handle_alarm_requests('invalid_key', {}, vim_uuid)
 
         # verify that create_alarm and publish_create_alarm_status methods not called
         m_create_alarm.assert_not_called()
         m_publish_create_alarm_status.assert_not_called()
 
-
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_metrics_data_status')
     @mock.patch.object(monPluginRec.MonPlugin, 'get_metrics_data')
-    def test_consume_invalid_metric_request_key(self, m_get_metrics_data,\
+    def test_consume_invalid_metric_request_key(self, m_get_metrics_data,
                                                 m_publish_metric_data_status):
         """Test functionality of invalid metric key request"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message with invalid metric request key
-        msg = Message()
-        msg.topic = "metric_request"
-        msg.key = "invalid_metric_data_request" #invalid key
 
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        with self.assertRaises(UnsupportedOperation):
+            self.plugin_receiver.handle_metric_requests('invalid_key', {}, vim_uuid)
 
-        # verify that get martics data and publish methods not called
+        # verify that get metrics data and publish methods not called
         m_get_metrics_data.assert_not_called()
         m_publish_metric_data_status.assert_not_called()
 
-
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_metrics_data_status')
     @mock.patch.object(monPluginRec.MonPlugin, 'get_metrics_data')
-    @mock.patch.object(monPluginRec.PluginReceiver,'get_vim_access_config')
-    def test_consume_read_metric_data_request_key(self, m_get_vim_access_config,\
-                                                  m_get_metrics_data,\
+    @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
+    def test_consume_read_metric_data_request_key(self, m_get_vim_access_config,
+                                                  m_get_metrics_data,
                                                   m_publish_metric_data_status):
         """Test functionality of consume for read_metric_data_request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "metric_request"
-        msg.key = "read_metric_data_request"
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"metric_name":"metric_details"})
-        m_get_metrics_data.return_value = {"metrics_data":"metrics_details"}
+        value = {"vim_uuid": vim_uuid, "metric_name": "metric_details"}
+        m_get_metrics_data.return_value = {"metrics_data": "metrics_details"}
 
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-            'vrops_user':'user',
-            'vrops_password':'passwd',
-            'vim_url':'vcd_url',
-            'admin_username':'admin',
-            'admin_password':'admin_passwd',
-            'vim_uuid':'1',
-            'tenant_id':'org_vdc_1'}
-
-        metric_request_info = json.loads(msg.value)
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
-        m_get_metrics_data.assert_called_with(metric_request_info)
+        self.plugin_receiver.handle_metric_requests('read_metric_data_request', value, vim_uuid)
+        m_get_metrics_data.assert_called_with(value)
 
         # Check if publish method called with correct parameters
-        m_publish_metric_data_status.assert_called_with({"metrics_data":"metrics_details"})
-
+        m_publish_metric_data_status.assert_called_with({"metrics_data": "metrics_details"})
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_create_metric_response')
     @mock.patch.object(monPluginRec.PluginReceiver, 'verify_metric')
-    def test_consume_create_metric_request_key(self, m_verify_metric,\
+    def test_consume_create_metric_request_key(self, m_verify_metric,
                                                m_publish_create_metric_response):
         """Test functionality of consume for create_metric_request key"""
 
         vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
-        # Mock a message
-        msg = Message()
-        msg.topic = "metric_request"
-        msg.key = "create_metric_request"
-
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"metric_create":"metric_details"})
+        value = {"vim_uuid": vim_uuid, "metric_create": "metric_details"}
 
         # set the return value
         m_verify_metric.return_value = True
 
-        metric_info = json.loads(msg.value)
-
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
-        m_verify_metric.assert_called_with(metric_info)
+        self.plugin_receiver.handle_metric_requests('create_metric_request', value, vim_uuid)
+        m_verify_metric.assert_called_with(value)
 
         # Check if publish method called with correct parameters
-        m_publish_create_metric_response.assert_called_with(metric_info, True)
-
+        m_publish_create_metric_response.assert_called_with(value, True)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_update_metric_response')
     @mock.patch.object(monPluginRec.PluginReceiver, 'verify_metric')
-    def test_consume_update_metric_request_key(self, m_verify_metric,\
+    def test_consume_update_metric_request_key(self, m_verify_metric,
                                                m_publish_update_metric_response):
         """Test functionality of update metric request key"""
 
-        vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb" 
-        # Mock a message
-        msg = Message()
-        msg.topic = "metric_request"
-        msg.key = "update_metric_request"
+        vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
 
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"metric_create":"metric_details"})
+        value = {"vim_uuid": vim_uuid, "metric_create": "metric_details"}
 
         # set the return value
         m_verify_metric.return_value = True
 
-        metric_info = json.loads(msg.value)
-
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        self.plugin_receiver.handle_metric_requests('update_metric_request', value, vim_uuid)
 
         # verify mocked methods called with correct parameters
-        m_verify_metric.assert_called_with(metric_info)
-        m_publish_update_metric_response.assert_called_with(metric_info, True)
-
+        m_verify_metric.assert_called_with(value)
+        m_publish_update_metric_response.assert_called_with(value, True)
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'publish_delete_metric_response')
     def test_consume_delete_metric_request_key(self, m_publish_delete_metric_response):
         """Test functionality of consume for delete_metric_request key"""
 
         # Note: vROPS doesn't support deleting metric data
-        vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"  
-        # Mock a message
-        msg = Message()
-        msg.topic = "metric_request"
-        msg.key = "delete_metric_request"
-
-        msg.value = json.dumps({"vim_uuid":vim_uuid,"metric_name":"metric_details"})
+        vim_uuid = "f85fc39e-723d-4172-979b-de28b36465bb"
 
-        metric_info = json.loads(msg.value)
+        value = {"vim_uuid": vim_uuid, "metric_name": "metric_details"}
 
         # Call the consume method of plugin_receiver
-        self.plugin_receiver.consume(msg,vim_uuid)
+        self.plugin_receiver.handle_metric_requests('delete_metric_request', value, vim_uuid)
 
         # Check if publish method called with correct parameters
-        m_publish_delete_metric_response.assert_called_with(metric_info)
-
+        m_publish_delete_metric_response.assert_called_with(value)
 
     @mock.patch.object(monPluginRec.MonPlugin, 'configure_alarm')
     @mock.patch.object(monPluginRec.MonPlugin, 'configure_rest_plugin')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
-    def test_create_alarm_successful(self, m_get_vim_access_config,\
-                                     m_configure_rest_plugin,\
+    def test_create_alarm_successful(self, m_get_vim_access_config,
+                                     m_configure_rest_plugin,
                                      m_configure_alarm):
         """ Test functionality of create alarm method-positive case"""
 
         # Mock config_alarm_info
-        config_alarm_info = {"schema_version":1.0,
-                             "schema_type":"create_alarm_request",
-                             "vim_type":"VMware",
-                             "vim_uuid":"1",
-                             "alarm_create_request":{"correlation_id": 1,
-                             "alarm_name": "CPU_Utilize_Threshold",
-                             "metric_name": "CPU_UTILIZATION",
-                             "tenant_uuid": "tenant_uuid",
-                             "resource_uuid": "resource_uuid",
-                             "description": "test_create_alarm",
-                             "severity": "CRITICAL",
-                             "operation": "GT",
-                             "threshold_value": 10,
-                             "unit": "%",
-                             "statistic": "AVERAGE"}}
+        config_alarm_info = {"schema_version": 1.0,
+                             "schema_type": "create_alarm_request",
+                             "vim_type": "VMware",
+                             "vim_uuid": "1",
+                             "alarm_create_request": {"correlation_id": 1,
+                                                      "alarm_name": "CPU_Utilize_Threshold",
+                                                      "metric_name": "CPU_UTILIZATION",
+                                                      "tenant_uuid": "tenant_uuid",
+                                                      "resource_uuid": "resource_uuid",
+                                                      "description": "test_create_alarm",
+                                                      "severity": "CRITICAL",
+                                                      "operation": "GT",
+                                                      "threshold_value": 10,
+                                                      "unit": "%",
+                                                      "statistic": "AVERAGE"}}
 
         # set return value to plugin uuid
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-            'vrops_user':'user',
-            'vrops_password':'passwd',
-            'vim_url':'vcd_url',
-            'admin_username':'admin',
-            'admin_password':'admin_passwd',
-            'vim_uuid':'1',
-            'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         m_configure_rest_plugin.retrun_value = "plugin_uuid"
         m_configure_alarm.return_value = "alarm_uuid"
@@ -395,41 +298,40 @@ class TestPluginReceiver(unittest.TestCase):
         m_configure_rest_plugin.assert_called_with()
         m_configure_alarm.assert_called_with(config_alarm_info["alarm_create_request"])
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'configure_alarm')
     @mock.patch.object(monPluginRec.MonPlugin, 'configure_rest_plugin')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
-    def test_create_alarm_failed(self, m_get_vim_access_config,\
-                                m_configure_rest_plugin,\
-                                m_configure_alarm):
+    def test_create_alarm_failed(self, m_get_vim_access_config,
+                                 m_configure_rest_plugin,
+                                 m_configure_alarm):
         """ Test functionality of create alarm method negative case"""
 
         # Mock config_alarm_info
-        config_alarm_info = {"schema_version":1.0,
-                             "schema_type":"create_alarm_request",
-                             "vim_type":"VMware",
-                             "vim_uuid":"1",
-                             "alarm_create_request":{"correlation_id": 1,
-                             "alarm_name": "CPU_Utilize_Threshold",
-                             "metric_name": "CPU_UTILIZATION",
-                             "tenant_uuid": "tenant_uuid",
-                             "resource_uuid": "resource_uuid",
-                             "description": "test_create_alarm",
-                             "severity": "CRITICAL",
-                             "operation": "GT",
-                             "threshold_value": 10,
-                             "unit": "%",
-                             "statistic": "AVERAGE"}}
+        config_alarm_info = {"schema_version": 1.0,
+                             "schema_type": "create_alarm_request",
+                             "vim_type": "VMware",
+                             "vim_uuid": "1",
+                             "alarm_create_request": {"correlation_id": 1,
+                                                      "alarm_name": "CPU_Utilize_Threshold",
+                                                      "metric_name": "CPU_UTILIZATION",
+                                                      "tenant_uuid": "tenant_uuid",
+                                                      "resource_uuid": "resource_uuid",
+                                                      "description": "test_create_alarm",
+                                                      "severity": "CRITICAL",
+                                                      "operation": "GT",
+                                                      "threshold_value": 10,
+                                                      "unit": "%",
+                                                      "statistic": "AVERAGE"}}
 
         # set return value to plugin uuid and alarm_uuid to None
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-            'vrops_user':'user',
-            'vrops_password':'passwd',
-            'vim_url':'vcd_url',
-            'admin_username':'admin',
-            'admin_password':'admin_passwd',
-            'vim_uuid':'1',
-            'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
         m_configure_rest_plugin.retrun_value = "plugin_uuid"
         m_configure_alarm.return_value = None
 
@@ -444,27 +346,26 @@ class TestPluginReceiver(unittest.TestCase):
         # verify create alarm method returns None when failed
         self.assertEqual(alarm_uuid, None)
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'update_alarm_configuration')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
     def test_update_alarm_successful(self, m_get_vim_access_config, m_update_alarm_configuration):
         """ Test functionality of update alarm method-positive case"""
 
         # Mock update_alarm_info
-        update_alarm_info = {"schema_version":1.0,"schema_type":"update_alarm_request",
-                             "vim_type":"VMware","vim_uuid":"1",
-                             "alarm_update_request":{'alarm_uuid': 'abc', 'correlation_id': 14203}}
+        update_alarm_info = {"schema_version": 1.0, "schema_type": "update_alarm_request",
+                             "vim_type": "VMware", "vim_uuid": "1",
+                             "alarm_update_request": {'alarm_uuid': 'abc', 'correlation_id': 14203}}
 
         # set return value to mocked method
         m_update_alarm_configuration.return_value = "alarm_uuid"
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # check update alarm gets called and returned correct value
         ret_value = self.plugin_receiver.update_alarm(update_alarm_info)
@@ -476,27 +377,26 @@ class TestPluginReceiver(unittest.TestCase):
         # check return value and passed values are correct
         self.assertEqual(ret_value, "alarm_uuid")
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'update_alarm_configuration')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
     def test_update_alarm_failed(self, m_get_vim_access_config, m_update_alarm_configuration):
         """ Test functionality of update alarm method negative case"""
 
         # Mock update_alarm_info
-        update_alarm_info = {"schema_version":1.0,"schema_type":"update_alarm_request",
-                             "vim_type":"VMware","vim_uuid":"1",
-                             "alarm_update_request":{'alarm_uuid': 'abc', 'correlation_id': 14203}}
+        update_alarm_info = {"schema_version": 1.0, "schema_type": "update_alarm_request",
+                             "vim_type": "VMware", "vim_uuid": "1",
+                             "alarm_update_request": {'alarm_uuid': 'abc', 'correlation_id': 14203}}
 
         # set return value to mocked method
         m_update_alarm_configuration.return_value = None
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # check update alarm gets called and returned correct value
         ret_value = self.plugin_receiver.update_alarm(update_alarm_info)
@@ -508,27 +408,26 @@ class TestPluginReceiver(unittest.TestCase):
         # check return value and passed values are correct
         self.assertEqual(ret_value, None)
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'delete_alarm_configuration')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
     def test_delete_alarm_successful(self, m_get_vim_access_config, m_delete_alarm_configuration):
         """ Test functionality of delete alarm method-positive case"""
 
         # Mock delete_alarm_info
-        delete_alarm_info = {"schema_version":1.0,"schema_type":"delete_alarm_request",
-                             "vim_type":"VMware","vim_uuid":"1",
-                             "alarm_delete_request":{'alarm_uuid': 'abc', 'correlation_id': 14203}}
+        delete_alarm_info = {"schema_version": 1.0, "schema_type": "delete_alarm_request",
+                             "vim_type": "VMware", "vim_uuid": "1",
+                             "alarm_delete_request": {'alarm_uuid': 'abc', 'correlation_id': 14203}}
 
         # set return value to mocked method
         m_delete_alarm_configuration.return_value = "alarm_uuid"
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # check delete alarm gets called and returned correct value
         ret_value = self.plugin_receiver.delete_alarm(delete_alarm_info)
@@ -540,27 +439,26 @@ class TestPluginReceiver(unittest.TestCase):
         # check return value and passed values are correct
         self.assertEqual(ret_value, "alarm_uuid")
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'delete_alarm_configuration')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
     def test_delete_alarm_failed(self, m_get_vim_access_config, m_delete_alarm_configuration):
         """ Test functionality of delete alarm method-negative case"""
 
         # Mock update_alarm_info
-        delete_alarm_info = {"schema_version":1.0,"schema_type":"delete_alarm_request",
-                             "vim_type":"VMware","vim_uuid":"1",
-                             "alarm_delete_request":{'alarm_uuid': 'abc', 'correlation_id': 14203}}
+        delete_alarm_info = {"schema_version": 1.0, "schema_type": "delete_alarm_request",
+                             "vim_type": "VMware", "vim_uuid": "1",
+                             "alarm_delete_request": {'alarm_uuid': 'abc', 'correlation_id': 14203}}
 
         # set return value to mocked method
         m_delete_alarm_configuration.return_value = None
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # check delete alarm gets called and returned correct value
         ret_value = self.plugin_receiver.delete_alarm(delete_alarm_info)
@@ -572,13 +470,11 @@ class TestPluginReceiver(unittest.TestCase):
         # check return value to check failed status
         self.assertEqual(ret_value, None)
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_create_alarm_status(self, m_publish):
+    def test_publish_create_alarm_status(self):
         """ Test functionality of publish create alarm status method"""
 
         # Mock config_alarm_info
-        config_alarm_info = {'vim_type': 'VMware', "vim_uuid":"1",
+        config_alarm_info = {'vim_type': 'VMware', "vim_uuid": "1",
                              'alarm_create_request': {
                                  'threshold_value': 0,
                                  'severity': 'CRITICAL',
@@ -587,7 +483,7 @@ class TestPluginReceiver(unittest.TestCase):
                                  'correlation_id': 1234,
                                  'statistic': 'AVERAGE',
                                  'metric_name': 'CPU_UTILIZATION'}
-                            }
+                             }
 
         alarm_uuid = "xyz"
 
@@ -595,24 +491,20 @@ class TestPluginReceiver(unittest.TestCase):
         self.plugin_receiver.publish_create_alarm_status(alarm_uuid, config_alarm_info)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='create_alarm_response',\
-                                     value=mock.ANY,\
-                                     topic='alarm_response')
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_update_alarm_status(self, m_publish):
+    def test_publish_update_alarm_status(self):
         """ Test functionality of publish update alarm status method"""
 
         # Mock update_alarm_info
-        update_alarm_info = {'vim_type' : 'VMware',
-                             'vim_uuid':'1',
+        update_alarm_info = {'vim_type': 'VMware',
+                             'vim_uuid': '1',
                              'schema_type': 'update_alarm_request',
-                             'alarm_update_request':{'alarm_uuid': '6486e69',
-                                                     'correlation_id': 14203,
-                                                     'operation': 'GT'
-                                                     }
-                            }
+                             'alarm_update_request': {'alarm_uuid': '6486e69',
+                                                      'correlation_id': 14203,
+                                                      'operation': 'GT'
+                                                      }
+                             }
 
         alarm_uuid = "xyz"
 
@@ -620,24 +512,20 @@ class TestPluginReceiver(unittest.TestCase):
         self.plugin_receiver.publish_update_alarm_status(alarm_uuid, update_alarm_info)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='update_alarm_response',\
-                                     value=mock.ANY,\
-                                     topic='alarm_response')
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_delete_alarm_status(self, m_publish):
+    def test_publish_delete_alarm_status(self):
         """ Test functionality of publish delete alarm status method"""
 
         # Mock delete_alarm_info
-        delete_alarm_info = {'vim_type' : 'VMware',
-                             "vim_uuid":"1",
+        delete_alarm_info = {'vim_type': 'VMware',
+                             "vim_uuid": "1",
                              'schema_type': 'delete_alarm_request',
-                             'alarm_delete_request':{'alarm_uuid': '6486e69',
-                                                     'correlation_id': 14203,
-                                                     'operation': 'GT'
-                                                     }
-                            }
+                             'alarm_delete_request': {'alarm_uuid': '6486e69',
+                                                      'correlation_id': 14203,
+                                                      'operation': 'GT'
+                                                      }
+                             }
 
         alarm_uuid = "xyz"
 
@@ -645,56 +533,49 @@ class TestPluginReceiver(unittest.TestCase):
         self.plugin_receiver.publish_delete_alarm_status(alarm_uuid, delete_alarm_info)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='delete_alarm_response',\
-                                     value=mock.ANY,\
-                                     topic='alarm_response')
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_metrics_data_status(self, m_publish):
+    def test_publish_metrics_data_status(self):
         """ Test functionality of publish metrics data status method"""
 
         # Mock metrics data
         metrics_data = {
-                        'vim_uuid':'1',
-                        'metric_name': 'CPU_UTILIZATION', 'metric_uuid': '0',
-                        'resource_uuid': 'e14b20', 'correlation_id': 14203,
-                        'metrics_data': {'time_series': [15162011, 15162044],
-                        'metrics_series': [0.1166666671, 0.1266666650]},
-                        'tenant_uuid': 123, 'unit': '%'
-                       }
+            'vim_uuid': '1',
+            'metric_name': 'CPU_UTILIZATION', 'metric_uuid': '0',
+            'resource_uuid': 'e14b20', 'correlation_id': 14203,
+            'metrics_data': {'time_series': [15162011, 15162044],
+                             'metrics_series': [0.1166666671, 0.1266666650]},
+            'tenant_uuid': 123, 'unit': '%'
+        }
 
         # call publish metrics data status method under test
         self.plugin_receiver.publish_metrics_data_status(metrics_data)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='read_metric_data_response',\
-                                     value=mock.ANY,\
-                                     topic='metric_response')
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
     @mock.patch.object(monPluginRec.MonPlugin, 'verify_metric_support')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
-    def test_verify_metric_supported_metric(self, m_get_vim_access_config,\
+    def test_verify_metric_supported_metric(self, m_get_vim_access_config,
                                             m_verify_metric_support):
         """ Test functionality of verify metric method"""
 
         # mock metric_info
-        metric_info = {'vim_uuid':'1',\
-                       'metric_create_request':{'metric_unit': '%',\
-                                        'metric_name': 'CPU_UTILIZATION',\
-                                        'resource_uuid': 'e14b203'}}
+        metric_info = {'vim_uuid': '1',
+                       'metric_create_request': {'metric_unit': '%',
+                                                 'metric_name': 'CPU_UTILIZATION',
+                                                 'resource_uuid': 'e14b203'}}
 
         # set mocked function return value to true
         m_verify_metric_support.return_value = True
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # call verify_metric method under test
         ret_value = self.plugin_receiver.verify_metric(metric_info)
@@ -706,29 +587,28 @@ class TestPluginReceiver(unittest.TestCase):
         # verify the return value
         self.assertEqual(ret_value, True)
 
-
     @mock.patch.object(monPluginRec.MonPlugin, 'verify_metric_support')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
-    def test_verify_metric_unsupported_metric(self, m_get_vim_access_config,\
+    def test_verify_metric_unsupported_metric(self, m_get_vim_access_config,
                                               m_verify_metric_support):
         """ Test functionality of verify metric method-negative case"""
 
         # mock metric_info with unsupported metrics name
-        metric_info = {'vim_uuid':'1',\
-                       'metric_create_request':{'metric_unit': '%',\
-                                        'metric_name': 'Invalid',\
-                                        'resource_uuid': 'e14b203'}}
+        metric_info = {'vim_uuid': '1',
+                       'metric_create_request': {'metric_unit': '%',
+                                                 'metric_name': 'Invalid',
+                                                 'resource_uuid': 'e14b203'}}
 
         # set mocked function return value to true
         m_verify_metric_support.return_value = False
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # call verify_metric method under test
         ret_value = self.plugin_receiver.verify_metric(metric_info)
@@ -740,23 +620,21 @@ class TestPluginReceiver(unittest.TestCase):
         # verify the return value
         self.assertEqual(ret_value, False)
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_create_metric_response(self, m_publish):
+    def test_publish_create_metric_response(self):
         """ Test functionality of publish create metric response method"""
 
         # Mock metric_info
         metric_info = {
-                       'vim_uuid':'1',
-                       'vim_type' : 'VMware',
-                       'correlation_id': 14203,
-                       'schema_type': 'create_metric_request',
-                       'metric_create_request':{
-                                        'resource_uuid': '6486e69',
-                                        'metric_name': 'CPU_UTILIZATION',
-                                        'metric_unit': '%'
-                                        }
-                       }
+            'vim_uuid': '1',
+            'vim_type': 'VMware',
+            'correlation_id': 14203,
+            'schema_type': 'create_metric_request',
+            'metric_create_request': {
+                'resource_uuid': '6486e69',
+                'metric_name': 'CPU_UTILIZATION',
+                'metric_unit': '%'
+            }
+        }
 
         metric_status = True
 
@@ -764,27 +642,23 @@ class TestPluginReceiver(unittest.TestCase):
         self.plugin_receiver.publish_create_metric_response(metric_info, metric_status)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='create_metric_response',\
-                                     value=mock.ANY,\
-                                     topic='metric_response')
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_update_metric_response(self, m_publish):
+    def test_publish_update_metric_response(self):
         """ Test functionality of publish update metric response method"""
 
         # Mock metric_info
         metric_info = {
-                       'vim_uuid':'1',
-                       'vim_type' : 'VMware',
-                       'correlation_id': 14203,
-                       'schema_type': 'update_metric_request',
-                       'metric_create':{
-                                        'resource_uuid': '6486e69',
-                                        'metric_name': 'CPU_UTILIZATION',
-                                        'metric_unit': '%'
-                                        }
-                       }
+            'vim_uuid': '1',
+            'vim_type': 'VMware',
+            'correlation_id': 14203,
+            'schema_type': 'update_metric_request',
+            'metric_create': {
+                'resource_uuid': '6486e69',
+                'metric_name': 'CPU_UTILIZATION',
+                'metric_unit': '%'
+            }
+        }
 
         metric_status = True
 
@@ -792,32 +666,23 @@ class TestPluginReceiver(unittest.TestCase):
         self.plugin_receiver.publish_update_metric_response(metric_info, metric_status)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='update_metric_response',\
-                                     value=mock.ANY,\
-                                     topic='metric_response')
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_delete_metric_response(self, m_publish):
+    def test_publish_delete_metric_response(self):
         """ Test functionality of publish delete metric response method"""
 
         # Mock metric_info
-        metric_info = {'vim_uuid':'1', 'vim_type' : 'VMware','correlation_id': 14203,
+        metric_info = {'vim_uuid': '1', 'vim_type': 'VMware', 'correlation_id': 14203,
                        'metric_uuid': 'e14b203c', 'resource_uuid': '6486e69',
                        'metric_name': 'CPU_UTILIZATION',
                        'schema_type': 'delete_metric_request'}
 
-        metric_status = True
-
         # call publish delete metric method under test-vROPS doesn't support
         # delete metric,just returns response with success
         self.plugin_receiver.publish_delete_metric_response(metric_info)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key='delete_metric_response',\
-                                     value=mock.ANY,\
-                                     topic='metric_response')
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
     @mock.patch.object(monPluginRec.MonPlugin, 'get_triggered_alarms_list')
     @mock.patch.object(monPluginRec.PluginReceiver, 'get_vim_access_config')
@@ -826,18 +691,18 @@ class TestPluginReceiver(unittest.TestCase):
 
         # Mock list alarm input
         list_alarm_input = {
-                            'vim_uuid':'1',
-                            'vim_type' : 'VMware',
-                            'alarm_list_request':{
-                                'severity': 'CRITICAL',
-                                'correlation_id': 14203,
-                                'alarm_name': 'CPU_Utilization_Above_Threshold',
-                                'resource_uuid': 'd14b203c'}}
+            'vim_uuid': '1',
+            'vim_type': 'VMware',
+            'alarm_list_request': {
+                'severity': 'CRITICAL',
+                'correlation_id': 14203,
+                'alarm_name': 'CPU_Utilization_Above_Threshold',
+                'resource_uuid': 'd14b203c'}}
 
         # set return value to mocked method
         m_return = [{'status': 'ACTIVE', 'update_date': '2018-01-12T08:34:05',
                      'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
-                     'cancel_date': '0000-00-00T00:00:00','alarm_instance_uuid': 'd9e3bc84',
+                     'cancel_date': '0000-00-00T00:00:00', 'alarm_instance_uuid': 'd9e3bc84',
                      'alarm_uuid': '5714977d', 'vim_type': 'VMware',
                      'start_date': '2018-01-12T08:34:05'},
                     {'status': 'CANCELED', 'update_date': '2017-12-20T09:37:57',
@@ -847,15 +712,14 @@ class TestPluginReceiver(unittest.TestCase):
                      'start_date': '2017-12-20T09:37:57'}]
         m_get_triggered_alarms_list.return_value = m_return
 
-        m_get_vim_access_config.return_value = {'vrops_site':'abc',
-                                                'vrops_user':'user',
-                                                'vrops_password':'passwd',
-                                                'vim_url':'vcd_url',
-                                                'admin_username':'admin',
-                                                'admin_password':'admin_passwd',
-                                                'vim_uuid':'1',
-                                                'tenant_id':'org_vdc_1'}
-
+        m_get_vim_access_config.return_value = {'vrops_site': 'abc',
+                                                'vrops_user': 'user',
+                                                'vrops_password': 'passwd',
+                                                'vim_url': 'vcd_url',
+                                                'admin_username': 'admin',
+                                                'admin_password': 'admin_passwd',
+                                                'vim_uuid': '1',
+                                                'tenant_id': 'org_vdc_1'}
 
         # call list alarms method under test
         return_value = self.plugin_receiver.list_alarms(list_alarm_input)
@@ -867,24 +731,22 @@ class TestPluginReceiver(unittest.TestCase):
         # verify list alarm method returns correct list
         self.assertEqual(return_value, m_return)
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_list_alarm_response(self, m_publish):
+    def test_publish_list_alarm_response(self):
         """ Test functionality of publish list alarm response method"""
 
         # Mock list alarm input
         msg_key = 'list_alarm_response'
         topic = 'alarm_response'
-        list_alarm_input = {'vim_uuid':'1',
-                            'vim_type' : 'VMware',
-                            'alarm_list_request':{
+        list_alarm_input = {'vim_uuid': '1',
+                            'vim_type': 'VMware',
+                            'alarm_list_request': {
                                 'severity': 'CRITICAL',
                                 'correlation_id': 14203,
                                 'alarm_name': 'CPU_Utilization_Above_Threshold',
                                 'resource_uuid': 'd14b203c'}}
 
         triggered_alarm_list = [{'status': 'ACTIVE', 'update_date': '2018-01-12T08:34:05',
-                                 'severity': 'CRITICAL','resource_uuid': 'e14b203c',
+                                 'severity': 'CRITICAL', 'resource_uuid': 'e14b203c',
                                  'cancel_date': '0000-00-00T00:00:00',
                                  'start_date': '2018-01-12T08:34:05',
                                  'alarm_instance_uuid': 'd9e3bc84',
@@ -893,22 +755,18 @@ class TestPluginReceiver(unittest.TestCase):
                                  }]
 
         # call publish list alarm response method under test
-        self.plugin_receiver.publish_list_alarm_response(triggered_alarm_list, list_alarm_input)
+        response = self.plugin_receiver.publish_list_alarm_response(triggered_alarm_list, list_alarm_input)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key=msg_key,value=mock.ANY, topic=topic)
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
-
-    @mock.patch.object(monPluginRec.KafkaProducer, 'publish')
-    def test_publish_access_update_response(self, m_publish):
+    def test_publish_access_update_response(self):
         """ Test functionality of publish access update response method"""
 
         # Mock required inputs
         access_update_status = True
-        msg_key = 'vim_access_credentials_response'
-        topic = 'access_credentials'
         access_info_req = {'vim_type': 'VMware',
-                           'vim_uuid':'1',
+                           'vim_uuid': '1',
                            'access_config': {'vrops_password': 'vmware',
                                              'vcloud-site': 'https://192.169.241.105',
                                              'vrops_user': 'Admin', 'correlation_id': 14203,
@@ -916,23 +774,22 @@ class TestPluginReceiver(unittest.TestCase):
                            }
 
         # call publish access update response method under test
-        self.plugin_receiver.publish_access_update_response(access_update_status, access_info_req)
+        response = self.plugin_receiver.publish_access_update_response(access_update_status, access_info_req)
 
         # verify mocked method called with correct params
-        m_publish.assert_called_with(key=msg_key ,value=mock.ANY, topic=topic)
-
+        # TODO(diazb): Validate payload generation (self.assertEquals(response, expected_message))
 
     @mock.patch.object(monPluginRec.PluginReceiver, 'write_access_config')
     def test_update_access_credentials_successful(self, m_write_access_config):
         """ Test functionality of update access credentials-positive case"""
 
         # Mock access_info
-        access_info = {'vrops_site':'https://192.169.241.13','vrops_user':'admin',
-                       'vrops_password':'vmware','vcloud-site':'https://192.169.241.15',
-                       'admin_username':'admin','admin_password':'vmware',
-                       'vcenter_ip':'192.169.241.13','vcenter_port':'443',
-                       'vcenter_user':'admin','vcenter_password':'vmware',
-                       'vim_tenant_name':'Org2','orgname':'Org2','tenant_id':'Org2'}
+        access_info = {'vrops_site': 'https://192.169.241.13', 'vrops_user': 'admin',
+                       'vrops_password': 'vmware', 'vcloud-site': 'https://192.169.241.15',
+                       'admin_username': 'admin', 'admin_password': 'vmware',
+                       'vcenter_ip': '192.169.241.13', 'vcenter_port': '443',
+                       'vcenter_user': 'admin', 'vcenter_password': 'vmware',
+                       'vim_tenant_name': 'Org2', 'orgname': 'Org2', 'tenant_id': 'Org2'}
 
         # Mock return values
         expected_status = m_write_access_config.return_value = True
@@ -946,17 +803,16 @@ class TestPluginReceiver(unittest.TestCase):
         # verify update access credentials returns correct status
         self.assertEqual(expected_status, actual_status)
 
-
     @mock.patch.object(monPluginRec.PluginReceiver, 'write_access_config')
     def test_update_access_credentials_less_config_params(self, m_write_access_config):
         """ Test functionality of update access credentials-negative case"""
 
         # Mock access_info
-        access_info = {'vrops_site':'https://192.169.241.13','vrops_user':'admin',
-                       'vrops_password':'vmware','vcloud-site':'https://192.169.241.15',
-                       'admin_username':'admin','admin_password':'vmware',
-                       'vcenter_ip':'192.169.241.13','vcenter_port':'443','vcenter_user':'admin',
-                       'vim_tenant_name':'Org2','orgname':'Org2','tenant_id':'Org2'}
+        access_info = {'vrops_site': 'https://192.169.241.13', 'vrops_user': 'admin',
+                       'vrops_password': 'vmware', 'vcloud-site': 'https://192.169.241.15',
+                       'admin_username': 'admin', 'admin_password': 'vmware',
+                       'vcenter_ip': '192.169.241.13', 'vcenter_port': '443', 'vcenter_user': 'admin',
+                       'vim_tenant_name': 'Org2', 'orgname': 'Org2', 'tenant_id': 'Org2'}
 
         # Mock return values
         expected_status = m_write_access_config.return_value = False
@@ -970,18 +826,17 @@ class TestPluginReceiver(unittest.TestCase):
         # verify update access credentials returns correct status
         self.assertEqual(expected_status, actual_status)
 
-
     @mock.patch.object(monPluginRec.PluginReceiver, 'write_access_config')
     def test_update_access_credentials_failed(self, m_write_access_config):
         """ Test functionality of update access credentials-failed case """
 
         # Mock access_info
-        access_info = {'vrops_site':'https://192.169.241.13','vrops_user':'admin',
-                       'vrops_password':'vmware','vcloud-site':'https://192.169.241.15',
-                       'admin_username':'admin','admin_password':'vmware',
-                       'vcenter_ip':'192.169.241.13','vcenter_port':'443',
-                       'vcenter_user':'admin','vcenter_password':'vmware',
-                       'vim_tenant_name':'Org2','orgname':'Org2','tenant_id':'Org2'}
+        access_info = {'vrops_site': 'https://192.169.241.13', 'vrops_user': 'admin',
+                       'vrops_password': 'vmware', 'vcloud-site': 'https://192.169.241.15',
+                       'admin_username': 'admin', 'admin_password': 'vmware',
+                       'vcenter_ip': '192.169.241.13', 'vcenter_port': '443',
+                       'vcenter_user': 'admin', 'vcenter_password': 'vmware',
+                       'vim_tenant_name': 'Org2', 'orgname': 'Org2', 'tenant_id': 'Org2'}
 
         # Mock return values
         expected_status = m_write_access_config.return_value = False
@@ -995,17 +850,16 @@ class TestPluginReceiver(unittest.TestCase):
         # verify update access credentials returns correct status
         self.assertEqual(expected_status, actual_status)
 
-
     def test_write_access_config_successful(self):
         """ Test functionality of write access config method-positive case"""
 
         # Mock access_info
-        access_info = {'vrops_sit':'https://192.169.241.13','vrops_user':'admin',
-                       'vrops_password':'vmware','vcloud-site':'https://192.169.241.15',
-                       'admin_username':'admin','admin_password':'vmware',
-                       'vcenter_ip':'192.169.241.13','vcenter_port':'443',
-                       'vcenter_user':'admin','vcenter_password':'vmware',
-                       'vim_tenant_name':'Org2','orgname':'Org2','tenant_id':'Org2'}
+        access_info = {'vrops_sit': 'https://192.169.241.13', 'vrops_user': 'admin',
+                       'vrops_password': 'vmware', 'vcloud-site': 'https://192.169.241.15',
+                       'admin_username': 'admin', 'admin_password': 'vmware',
+                       'vcenter_ip': '192.169.241.13', 'vcenter_port': '443',
+                       'vcenter_user': 'admin', 'vcenter_password': 'vmware',
+                       'vim_tenant_name': 'Org2', 'orgname': 'Org2', 'tenant_id': 'Org2'}
 
         # call write access config method under test
         actual_status = self.plugin_receiver.write_access_config(access_info)
@@ -1013,12 +867,11 @@ class TestPluginReceiver(unittest.TestCase):
         # verify write access config returns correct status
         self.assertEqual(True, actual_status)
 
-
     def test_write_access_config_failed(self):
         """ Test functionality of write access config method-negative case"""
 
         # Mock access_info
-        access_info = [] # provided incorrect info to generate error
+        access_info = []  # provided incorrect info to generate error
 
         # call write access config method under test
         actual_status = self.plugin_receiver.write_access_config(access_info)
@@ -1026,7 +879,6 @@ class TestPluginReceiver(unittest.TestCase):
         # verify write access config returns correct status
         self.assertEqual(False, actual_status)
 
-
     @mock.patch.object(monPluginRec.AuthManager, 'get_credentials')
     def test_get_vim_access_config(self, m_get_credentials):
         """ Test functionality of get_vim_access_config method-positive case"""
@@ -1051,27 +903,25 @@ class TestPluginReceiver(unittest.TestCase):
                         "vcenter_password": "vcenter_pwd", "vcenter_port": "443"}'
         m_get_credentials.return_value = vim_details
         expected_config = {'vrops_password': 'vrops_pwd', 'vcenter_password': 'vcenter_pwd',
-                         'name': 'vrops_vcd82', 'org_user': 'admin',
-                         'org_password': 'passwd', 'nsx_user': 'admin', 'vim_tenant_name': 'MANO-VDC',
-                         'admin_username': 'administrator', 'vcenter_port': '443',
-                         'vim_url': 'https://10.10.1.1', 'orgname': 'MANO-Org',
-                         'admin_password':'vcd_pwd', 'vrops_user':'admin', 'vcenter_ip':'10.10.1.4',
-                         'vrops_site': 'https://10.10.1.2', 'nsx_manager': 'https://10.10.1.3',
-                         'nsx_password': 'nsx_pwd', 'vim_type': 'VMware', 'vim_uuid': '1',
-                         'vcenter_user': 'admin@vsphere.local'}
+                           'name': 'vrops_vcd82', 'org_user': 'admin',
+                           'org_password': 'passwd', 'nsx_user': 'admin', 'vim_tenant_name': 'MANO-VDC',
+                           'admin_username': 'administrator', 'vcenter_port': '443',
+                           'vim_url': 'https://10.10.1.1', 'orgname': 'MANO-Org',
+                           'admin_password': 'vcd_pwd', 'vrops_user': 'admin', 'vcenter_ip': '10.10.1.4',
+                           'vrops_site': 'https://10.10.1.2', 'nsx_manager': 'https://10.10.1.3',
+                           'nsx_password': 'nsx_pwd', 'vim_type': 'VMware', 'vim_uuid': '1',
+                           'vcenter_user': 'admin@vsphere.local'}
 
         # call get_vim_access_config method under test
         actual_config = self.plugin_receiver.get_vim_access_config('1')
 
-        #verify that mocked method is called
+        # verify that mocked method is called
         m_get_credentials.assert_called_with(vim_uuid)
 
-        #Verify return value with expected value
+        # Verify return value with expected value
         self.assertEqual(expected_config, actual_config)
 
-
 # For testing purpose
-#if __name__ == '__main__':
+# if __name__ == '__main__':
 
 #    unittest.main()
-
diff --git a/osm_mon/test/core/kafka_test.py b/osm_mon/test/core/kafka_test.py
deleted file mode 100644 (file)
index 7e4267c..0000000
+++ /dev/null
@@ -1,63 +0,0 @@
-#!/usr/bin/env python
-# -*- coding: utf-8 -*-
-
-# Copyright 2017 Intel Research and Development Ireland Limited
-# *************************************************************
-
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
-
-#__author__ = "Prithiv Mohan"
-#__date__   = "25/Sep/2017"
-
-import sys
-import threading
-import pytest
-from kafka import KafkaConsumer, KafkaProducer
-
-def test_end_to_end():
-    producer = KafkaProducer(bootstrap_servers='localhost:9092',
-                             retries=5,
-                             max_block_ms=10000,
-                             value_serializer=str.encode)
-    consumer = KafkaConsumer(bootstrap_servers='localhost:9092',
-                             group_id=None,
-                             consumer_timeout_ms=10000,
-                             auto_offset_reset='earliest',
-                             value_deserializer=bytes.decode)
-
-    topic = 'TutorialTopic'
-
-    messages = 100
-    futures = []
-    for i in range(messages):
-        futures.append(producer.send(topic, 'msg %d' % i))
-    ret = [f.get(timeout=30) for f in futures]
-    assert len(ret) == messages
-
-    producer.close()
-
-    consumer.subscribe([topic])
-    msgs = set()
-    for i in range(messages):
-        try:
-            msgs.add(next(consumer).value)
-        except StopIteration:
-            break
-
-    assert msgs == set(['msg %d' % i for i in range(messages)])
index 0306893..eaf06c1 100644 (file)
@@ -5,8 +5,8 @@ from kafka import KafkaProducer
 from kafka.errors import KafkaError
 from osm_common import dbmongo
 
-from osm_mon.core.database import VimCredentials
-from osm_mon.core.message_bus.common_consumer import *
+from osm_mon.core.database import VimCredentials, DatabaseManager
+from osm_mon.core.message_bus.common_consumer import CommonConsumer
 
 
 @mock.patch.object(dbmongo.DbMongo, "db_connect", mock.Mock())
diff --git a/osm_mon/test/core/test_producer.py b/osm_mon/test/core/test_producer.py
deleted file mode 100644 (file)
index 5dc3caf..0000000
+++ /dev/null
@@ -1,82 +0,0 @@
-# Copyright 2017 Intel Research and Development Ireland Limited
-# *************************************************************
-
-# This file is part of OSM Monitoring module
-# All Rights Reserved to Intel Corporation
-
-# Licensed under the Apache License, Version 2.0 (the "License"); you may
-# not use this file except in compliance with the License. You may obtain
-# a copy of the License at
-
-#         http://www.apache.org/licenses/LICENSE-2.0
-
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-# License for the specific language governing permissions and limitations
-# under the License.
-
-# For those usages not covered by the Apache License, Version 2.0 please
-# contact: prithiv.mohan@intel.com or adrian.hoban@intel.com
-##
-"""This is a KafkaProducer with a request function to test the plugins."""
-
-import json
-
-import logging as log
-
-import os
-
-import jsmin
-
-from kafka import KafkaProducer as kaf
-
-from kafka.errors import KafkaError
-
-
-class KafkaProducer(object):
-    """A KafkaProducer for testing purposes."""
-
-    def __init__(self, topic):
-        """Initialize a KafkaProducer and it's topic."""
-        self._topic = topic
-
-        if "ZOOKEEPER_URI" in os.environ:
-            broker = os.getenv("ZOOKEEPER_URI")
-        else:
-            broker = "localhost:9092"
-
-            '''
-            If the zookeeper broker URI is not set in the env, by default,
-            localhost container is taken as the host because an instance of
-            is already running.
-            '''
-
-        self.producer = kaf(
-            key_serializer=str.encode,
-            value_serializer=lambda v: json.dumps(v).encode('ascii'),
-            bootstrap_servers=broker, api_version=(0, 10))
-
-    def publish(self, key, value, topic):
-        """Send messages to the message bus with a defing key and topic."""
-        try:
-            future = self.producer.send(topic=topic, key=key, value=value)
-            self.producer.flush()
-        except Exception:
-            log.exception("Error publishing to {} topic." .format(topic))
-            raise
-        try:
-            record_metadata = future.get(timeout=10)
-            log.debug("TOPIC:", record_metadata.topic)
-            log.debug("PARTITION:", record_metadata.partition)
-            log.debug("OFFSET:", record_metadata.offset)
-        except KafkaError:
-            pass
-
-    def request(self, path, key, message, topic):
-        """Test json files are loaded and sent on the message bus."""
-        # External to MON
-        payload_create_alarm = jsmin(open(os.path.join(path)).read())
-        self.publish(key=key,
-                     value=json.loads(payload_create_alarm),
-                     topic=topic)