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
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()
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()
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)
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)
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()
-# 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)
-# 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)
+++ /dev/null
-
-/* 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"]
-}
--- /dev/null
+
+/* 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"]
+}
# 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']
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))
# 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
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))
''' Handling of alarms requests via BOTO 2.48 '''
__author__ = "Wajeeha Hamid"
-__date__ = "18-September-2017"
+__date__ = "18-September-2017"
import logging
"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
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()
"""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'])
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']:
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'
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:
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))
# 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
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.
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)
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']
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
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
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))
# 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
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))
# 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:
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))
--- /dev/null
+# 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
+++ /dev/null
-# 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
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()
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):
"""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()
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."""
--- /dev/null
+# 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
+++ /dev/null
-# 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
##
"""Generate valid responses to send back to the SO."""
-import json
import logging
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)
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",
"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",
"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",
"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",
"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",
"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",
"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",
"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",
"severity": kwargs['sev'],
"status": kwargs['state'],
"start_date": kwargs['date']}}
- return json.dumps(notify_alarm_resp)
+ return notify_alarm_resp
+++ /dev/null
-# -*- 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)
-
"""
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
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:
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
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):
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)
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
source.close()
return a_params
-
def create_symptom(self, symptom_params):
"""Create Symptom definition for an alarm
Params:
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)
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):
"""
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)
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):
"""
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:
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)
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):
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'
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
"""
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:
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):
"""
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):
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):
"""
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)
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.
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
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
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:
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':
"""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)
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 = {}
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:
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
"""
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 = {}
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:
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
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/'
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
"""
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):
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):
"""
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)
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:
"""
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):
"""
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
-
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
"""
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'])
"""
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'])
"""
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:
"""
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
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.
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)
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)
if __name__ == "__main__":
main()
"""
-
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
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):
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
"""
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']
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
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'):
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
server.ssl_adapter = pyOpenSSLAdapter(
certificate=CERTIFICATE,
private_key=KEY,
- # certificate_chain="intermediate_cert.crt"
+ # certificate_chain="intermediate_cert.crt"
)
try:
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')
# 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__)
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__)
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())
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):
@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
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(
@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
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(
@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
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(
@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
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(
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
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
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
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
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:
@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
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(
@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
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(
@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
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(
@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
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(
@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
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
##
"""Tests for all common OpenStack methods."""
-
from __future__ import unicode_literals
import json
import logging
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__)
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')
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():
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",
"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)
+# -*- 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__)
"""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__)
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')
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)
@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())
@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',
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__)
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):
@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(
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."""
"""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__)
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):
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(
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."""
"<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."""
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."""
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."""
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."""
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__)
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}'
# 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}'
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())
@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}'
# 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}'
# 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)
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",
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."""
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):
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):
"""Test that the correct responses are generated for each message."""
import logging
-
import unittest
import mock
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."""
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(
"""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__)
""" 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):
"""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': '%'}
# 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"""
# 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"""
# 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"
# 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"""
# 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
# 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"""
# 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
# 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"""
'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"
# 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"""
'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'
# 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"""
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
# 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')
@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'
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,
{'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'],
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')
@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',
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()
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')
@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',
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,
{}
]
- #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()
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')
@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',
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,
'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()
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')
@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',
'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,
'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)
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')
@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',
'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,
'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)
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')
@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',
'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,
'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)
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')
@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',
'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,
'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)
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')
@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',
'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'
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,
'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)
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",\
["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.",\
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": [\
"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": [\
"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 = '{\
"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 = '{\
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",\
"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.",\
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",\
"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",\
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},\
"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},\
"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):
# 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/"},\
"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
# 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/"},\
"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
# 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"""
# 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"""
# 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"""
# 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" \
# 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):
# 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):
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'
# 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
# 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': '%'}
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': '%'}
'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)
# 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"""
'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)
# 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': '%'}
'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)
# 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': '%'}
'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)
# 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',
# 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'}
# 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',
# 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'}
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',
# 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'}
# 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)
# 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)
# 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
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)
# 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
# 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):
# 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)
# 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
# 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
# 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"""
}]}'
# 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()
# 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"""
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()
# 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"""
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()
# 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"""
# 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"""
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)
# 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"""
# 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"""
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'
# 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
# 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
# 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
# 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 = {}
# 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
# 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
# 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()
-
-
""" 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
@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"
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
# 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)
# 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)
# 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)
# 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)
# 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',
'correlation_id': 1234,
'statistic': 'AVERAGE',
'metric_name': 'CPU_UTILIZATION'}
- }
+ }
alarm_uuid = "xyz"
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"
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"
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)
# 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)
# 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
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
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')
# 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',
'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)
# 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',
}]
# 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,
}
# 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
# 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
# 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
# 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)
# 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)
# 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"""
"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()
-
+++ /dev/null
-#!/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)])
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())
+++ /dev/null
-# 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)