Add subcloud audit parallelization

The subcloud audit for distributed cloud is reworked to audit subclouds
in parallel, using greenthreads.

Parallelization is introduced by organizing the subclouds under audit in
a PriorityQueue, where the queue 'priority' is a timestamp identifying
when the audit should run. To support this, items are inserted into the
queue by the tuple (next-audit-timestamp, SubcloudAuditData)

The existing periodic task infrastructure is maintained but repurposed
as follows: The audit_sc_cert_task now performs the role of scheduler:
pulling any subclouds due for audit off the queue and scheduling them
for execution via a GreenPool.

A new file, subcloud_audit_queue.py, is added to the module to assist
in organizing data associated with the subcloud audits:
- Subcloud audit data is captured in class SubcloudAuditData
- SubcloudAuditPriorityQueue subclasses PriorityQueue to
  provide a specific API for this use case

Story: 2008960
Task: 43242

Signed-off-by: Kyle MacLeod <kyle.macleod@windriver.com>
Change-Id: I42ffcde16de2a89856b2220eee034175707aa59a
This commit is contained in:
Kyle MacLeod 2021-09-08 15:57:17 -04:00
parent 75cb9b0ba3
commit ee8333af35
4 changed files with 472 additions and 131 deletions

View File

@ -17,16 +17,18 @@
# of this software may be licensed only pursuant to the terms
# of an applicable Wind River license agreement.
#
from eventlet import greenthread
import time
import eventlet
import greenlet
from oslo_config import cfg
from oslo_log import log
from oslo_serialization import base64
from oslo_service import periodic_task
import time
from sysinv.cert_mon import watcher
from sysinv.cert_mon import subcloud_audit_queue
from sysinv.cert_mon import utils
from sysinv.cert_mon import watcher
from sysinv.common import constants
from sysinv.common import utils as cutils
@ -39,12 +41,23 @@ cert_mon_opts = [
help='Interval to run certificate audit'),
cfg.IntOpt('retry_interval',
default=10 * 60, # retry every 10 minutes
help='interval to reattempt accessing external system '
help='Interval to reattempt accessing external system '
'if failure occurred'),
cfg.IntOpt('max_retry',
default=14, # retry 14 times to give at least 2 hours to recover
help='interval to reattempt accessing external system '
help='Max number of reattempts accessing external system '
'if failure occurred'),
cfg.BoolOpt('startup_audit_all',
default=False,
help='Audit all subclouds on startup'),
cfg.IntOpt('audit_batch_size',
default=10,
help='Batch size of subcloud audits per audit_interval'),
cfg.IntOpt('audit_greenpool_size',
default=4,
help='Size of subcloud audit greenpool.'
'Set to 0 to disable use of greenpool '
'(force serial audit).'),
]
CONF = cfg.CONF
@ -59,8 +72,10 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
self.dc_monitor = None
self.restapicert_monitor = None
self.registrycert_monitor = None
self.reattempt_tasks = []
self.subclouds_to_audit = []
self.reattempt_monitor_tasks = []
self.sc_audit_queue = subcloud_audit_queue.SubcloudAuditPriorityQueue()
self.sc_audit_pool = eventlet.greenpool.GreenPool(
size=CONF.certmon.audit_greenpool_size)
def periodic_tasks(self, context, raise_on_error=False):
"""Tasks to be run at a periodic interval."""
@ -69,7 +84,7 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
@periodic_task.periodic_task(spacing=CONF.certmon.audit_interval)
def audit_sc_cert_start(self, context):
"""Kicks an audit of all subclouds.
This task runs every very long period of time, such as 24 hours.
By default this task runs once every 24 hours.
"""
# auditing subcloud certificate
dc_role = utils.get_dc_role()
@ -77,8 +92,12 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
# Do nothing if it is not systemcontroller
return
self.subclouds_to_audit = utils.get_subclouds()[:]
LOG.info("Periodic: begin subcloud certificate audit: %d subclouds" % len(self.subclouds_to_audit))
all_subclouds = utils.get_subclouds()[:]
LOG.info("Periodic: begin subcloud certificate audit: %d subclouds"
% len(all_subclouds))
for subcloud_name in all_subclouds:
self.sc_audit_queue.enqueue(
subcloud_audit_queue.SubcloudAuditData(subcloud_name))
def on_start_audit(self):
"""
@ -90,123 +109,196 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
# Do nothing if it is not systemcontroller
return
LOG.info("Service start: begin subcloud certificate audit")
number_of_sc_to_audit = 0
token = utils.get_token()
subclouds = utils.get_subclouds_from_dcmanager(token)
for sc in subclouds:
if sc[utils.ENDPOINT_TYPE_DC_CERT] != utils.SYNC_STATUS_IN_SYNC:
self.subclouds_to_audit.append(sc['name'])
LOG.info('%s is out-of-sync, adding it to audit.' % sc['name'])
number_of_sc_to_audit = number_of_sc_to_audit + 1
if CONF.certmon.startup_audit_all:
LOG.info("Service start: audit all subclouds")
self.audit_sc_cert_start(None)
return
if number_of_sc_to_audit > 0:
LOG.info('%d subcloud(s) found out-of-sync to be audited' %
number_of_sc_to_audit)
LOG.info("Service start: begin subcloud certificate audit [batch: %s]"
% CONF.certmon.audit_batch_size)
token = utils.get_token()
all_subclouds = utils.get_subclouds_from_dcmanager(token)
for subcloud in all_subclouds:
if subcloud[utils.ENDPOINT_TYPE_DC_CERT] != utils.SYNC_STATUS_IN_SYNC:
subcloud_name = subcloud['name']
if self.sc_audit_queue.contains(subcloud_name):
LOG.info('%s is not in-sync but already under audit'
% subcloud_name)
else:
LOG.info('%s is not in-sync, adding it to audit'
% subcloud_name)
self.sc_audit_queue.enqueue(
subcloud_audit_queue.SubcloudAuditData(subcloud_name))
if self.sc_audit_queue.qsize() > 0:
LOG.info('Startup audit: %d subcloud(s) to be audited' %
self.sc_audit_queue.qsize())
else:
LOG.info('All subclouds are in-sync. No startup audit is required')
LOG.info('Startup audit: all subclouds are in-sync')
@periodic_task.periodic_task(spacing=5)
def audit_sc_cert_task(self, context):
if len(self.subclouds_to_audit) > 0:
subcloud_name = self.subclouds_to_audit[0]
if subcloud_name == TASK_NAME_PAUSE_AUDIT:
LOG.info('Pause audit for ongoing update to complete')
self.subclouds_to_audit.pop(0)
"""This task runs every N seconds, and is responsible for running
a single subcloud through its next step in the subcloud audit process.
Pull up to <batch_count> number of ready-to-audit subcloud audit
data items from the sc_audit_queue, and spawn each item to be
executed via the GreenPool (or directly invoke the audit if the
GreenPool is disabled).
"""
for batch_count in range(CONF.certmon.audit_batch_size):
if self.sc_audit_queue.qsize() < 1:
# Nothing to do
return
num_pause_tasks = self.subclouds_to_audit.count(TASK_NAME_PAUSE_AUDIT)
LOG.info('Auditing subcloud %s [#subclouds: %d #pause: %d]'
% (subcloud_name,
len(self.subclouds_to_audit) - num_pause_tasks,
num_pause_tasks))
if not utils.is_subcloud_online(subcloud_name):
LOG.info("Subcloud is not online, aborting audit: %s" % subcloud_name)
self.subclouds_to_audit.pop(0)
# Only continue if the next in queue is ready to be audited
# Peek into the timestamp of the next item in our priority queue
next_audit_timestamp = self.sc_audit_queue.queue[0][0]
if next_audit_timestamp >= int(time.time()):
LOG.debug("audit_sc_cert_task: no audits ready for "
"processing, qsize=%s"
% self.sc_audit_queue.qsize())
return
_, sc_audit_item = self.sc_audit_queue.get()
LOG.debug(
("audit_sc_cert_task: enqueue subcloud audit %s, "
"qsize:%s, batch:%s") %
(sc_audit_item, self.sc_audit_queue.qsize(), batch_count))
# This item is ready for audit
if CONF.certmon.audit_greenpool_size > 0:
self.sc_audit_pool.spawn_n(self.do_subcloud_audit,
sc_audit_item)
else:
self.do_subcloud_audit(sc_audit_item)
eventlet.sleep()
def do_subcloud_audit(self, sc_audit_item):
"""A wrapper function to ensure the subcloud audit task is marked done
within sc_audit_queue"""
try:
self._subcloud_audit(sc_audit_item)
finally:
self.sc_audit_queue.task_done()
def _subcloud_audit(self, sc_audit_item):
"""Invoke a subcloud audit"""
subcloud_name = sc_audit_item.name
LOG.info("Auditing subcloud %s, attempt #%s [qsize: %s]"
% (subcloud_name,
sc_audit_item.audit_count,
self.sc_audit_queue.qsize()))
dc_token = utils.get_dc_token(subcloud_name)
try:
subcloud_sysinv_url = utils.dc_get_subcloud_sysinv_url(
subcloud_name)
sc_ssl_cert = utils.get_endpoint_certificate(subcloud_sysinv_url)
except Exception as e:
LOG.error('Cannot audit ssl certificate on %s' % subcloud_name)
LOG.exception(e)
# certificate is not ready, no reaudit. Will be picked up
# by certificate MODIFIED event if it comes back
return
try:
secret = utils.get_sc_intermediate_ca_secret(subcloud_name)
check_list = ['ca.crt', 'tls.crt', 'tls.key']
for item in check_list:
if item not in secret.data:
raise Exception('%s certificate data missing: %s'
% (subcloud_name, item))
txt_ssl_cert = base64.decode_as_text(secret.data['tls.crt'])
txt_ssl_key = base64.decode_as_text(secret.data['tls.key'])
txt_ca_cert = base64.decode_as_text(secret.data['ca.crt'])
except Exception:
if not utils.is_subcloud_online(subcloud_name, dc_token):
LOG.exception("Error getting subcloud intermediate cert. "
"Subcloud is not online, aborting audit: %s"
% subcloud_name)
return
# Handle certificate-level issues
LOG.exception('Cannot audit ssl certificate on %s' % subcloud_name)
# certificate is not ready, no reaudit. Will be picked up
# by certificate MODIFIED event if it comes back
return
cert_chain = txt_ssl_cert + txt_ca_cert
if not cutils.verify_intermediate_ca_cert(cert_chain, sc_ssl_cert):
# The subcloud needs renewal.
LOG.info('Updating {} intermediate CA as it is out-of-sync'
.format(subcloud_name))
# reaudit this subcloud after delay
self.requeue_audit_subcloud(sc_audit_item)
try:
subcloud_sysinv_url = utils.dc_get_subcloud_sysinv_url(subcloud_name)
sc_ssl_cert = utils.get_endpoint_certificate(subcloud_sysinv_url)
secret = utils.get_sc_intermediate_ca_secret(subcloud_name)
check_list = ['ca.crt', 'tls.crt', 'tls.key']
for item in check_list:
if item not in secret.data:
raise Exception('%s certificate data missing: %s'
% (subcloud_name, item))
txt_ssl_cert = base64.decode_as_text(secret.data['tls.crt'])
txt_ssl_key = base64.decode_as_text(secret.data['tls.key'])
txt_ca_cert = base64.decode_as_text(secret.data['ca.crt'])
except Exception as e:
LOG.error('Cannot audit ssl certificate on %s' % subcloud_name)
LOG.exception(e)
# certificate is not ready, no reaudit. Will be picked up
# by certificate MODIFIED event if it comes back
self.subclouds_to_audit.pop(0)
return
cert_chain = txt_ssl_cert + txt_ca_cert
dc_token = utils.get_dc_token(subcloud_name)
if not cutils.verify_intermediate_ca_cert(cert_chain, sc_ssl_cert):
# The subcloud needs renewal.
LOG.info('Updating {} intermediate CA as it is out-of-sync'.format(subcloud_name))
# move the subcloud to the end of the queue for reauditing
self.requeue_audit(subcloud_name)
utils.update_subcloud_ca_cert(dc_token,
subcloud_name,
subcloud_sysinv_url,
txt_ca_cert,
txt_ssl_cert,
txt_ssl_key)
except Exception:
LOG.exception('Failed to update intermediate CA on %s'
% subcloud_name)
utils.update_subcloud_status(dc_token, subcloud_name,
utils.SYNC_STATUS_OUT_OF_SYNC)
try:
utils.update_subcloud_ca_cert(dc_token,
subcloud_name,
subcloud_sysinv_url,
txt_ca_cert,
txt_ssl_cert,
txt_ssl_key)
except Exception:
LOG.exception('Failed to update intermediate CA on %s' % subcloud_name)
else:
LOG.info('%s intermediate CA cert is in-sync' % subcloud_name)
utils.update_subcloud_status(dc_token, subcloud_name,
utils.SYNC_STATUS_IN_SYNC)
self.subclouds_to_audit.remove(subcloud_name)
else:
LOG.info('%s intermediate CA cert is in-sync' % subcloud_name)
utils.update_subcloud_status(dc_token, subcloud_name,
utils.SYNC_STATUS_IN_SYNC)
@periodic_task.periodic_task(spacing=CONF.certmon.retry_interval)
def retry_task(self, context):
def retry_monitor_task(self, context):
# Failed tasks that need to be reattempted will be taken care here
max_attempts = CONF.certmon.max_retry
tasks = self.reattempt_tasks[:]
tasks = self.reattempt_monitor_tasks[:]
num_tasks = len(tasks)
if num_tasks > 0:
LOG.info('Starting retry_task: #tasks in reattempt queue: %s' % num_tasks)
LOG.info('Start retry_monitor_task: #tasks in queue: %s' %
num_tasks)
# NOTE: this loop can potentially retry ALL subclouds, which
# may be a resource concern.
for task in tasks:
task_id = task.get_id()
LOG.info("retry_monitor_task: %s, attempt: %s"
% (task_id, task.number_of_reattempt))
if task.run():
self.reattempt_tasks.remove(task)
LOG.info('Reattempt has succeeded')
self.reattempt_monitor_tasks.remove(task)
LOG.info('retry_monitor_task: %s, reattempt has succeeded'
% task_id)
elif task.number_of_reattempt >= max_attempts:
LOG.error('Maximum attempts (%s) has been reached. Give up' %
max_attempts)
if task in self.reattempt_tasks:
self.reattempt_tasks.remove(task)
LOG.error(("retry_monitor_task: %s, maximum attempts (%s) "
"has been reached. Give up")
% (task_id, max_attempts))
if task in self.reattempt_monitor_tasks:
self.reattempt_monitor_tasks.remove(task)
# task has failed
task.failed()
# Pause and allow other eventlets to run
eventlet.sleep(0.1)
LOG.debug('End retry_monitor_task')
def start_audit(self):
LOG.info('Auditing interval %s' % CONF.certmon.audit_interval)
utils.init_keystone_auth_opts()
self.audit_thread = greenthread.spawn(self.audit_cert)
self.audit_thread = eventlet.greenthread.spawn(self.audit_cert_loop)
self.on_start_audit()
def init_dc_monitor(self):
self.dc_monitor = watcher.DC_CertWatcher()
self.dc_monitor.initialize(
audit_subcloud=lambda subcloud_name: self.requeue_audit(subcloud_name))
audit_subcloud=lambda subcloud_name:
self.audit_subcloud(subcloud_name, allow_requeue=True))
def init_restapicert_monitor(self):
self.restapicert_monitor = watcher.RestApiCert_CertWatcher()
@ -226,8 +318,8 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
self.init_registrycert_monitor()
# init dc monitor only if running in DC role
if (dc_role == constants.DISTRIBUTED_CLOUD_ROLE_SYSTEMCONTROLLER or
dc_role == constants.DISTRIBUTED_CLOUD_ROLE_SUBCLOUD):
if dc_role in (constants.DISTRIBUTED_CLOUD_ROLE_SYSTEMCONTROLLER,
constants.DISTRIBUTED_CLOUD_ROLE_SUBCLOUD):
self.init_dc_monitor()
except Exception as e:
LOG.exception(e)
@ -236,12 +328,17 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
break
# spawn threads (DC thread spawned only if running in DC role)
self.mon_threads.append(greenthread.spawn(self.monitor_cert, self.restapicert_monitor))
self.mon_threads.append(greenthread.spawn(self.monitor_cert, self.registrycert_monitor))
self.mon_threads.append(
eventlet.greenthread.spawn(self.monitor_cert,
self.restapicert_monitor))
self.mon_threads.append(
eventlet.greenthread.spawn(self.monitor_cert,
self.registrycert_monitor))
if (dc_role == constants.DISTRIBUTED_CLOUD_ROLE_SYSTEMCONTROLLER or
dc_role == constants.DISTRIBUTED_CLOUD_ROLE_SUBCLOUD):
self.mon_threads.append(greenthread.spawn(self.monitor_cert, self.dc_monitor))
if dc_role in (constants.DISTRIBUTED_CLOUD_ROLE_SYSTEMCONTROLLER,
constants.DISTRIBUTED_CLOUD_ROLE_SUBCLOUD):
self.mon_threads.append(
eventlet.greenthread.spawn(self.monitor_cert, self.dc_monitor))
def stop_monitor(self):
for mon_thread in self.mon_threads:
@ -255,7 +352,7 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
self.audit_thread.wait()
self.audit_thread = None
def audit_cert(self):
def audit_cert_loop(self):
while True:
try:
self.run_periodic_tasks(context=None)
@ -265,15 +362,40 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
except Exception as e:
LOG.exception(e)
def requeue_audit_subcloud(self, sc_audit_item, delay_secs=60):
if not self.sc_audit_queue.contains(sc_audit_item.name):
self.sc_audit_queue.enqueue(sc_audit_item, delay_secs)
def audit_subcloud(self, subcloud_name, allow_requeue=False):
"""Enqueue a subcloud audit
allow_requeue: This can come from a watch after a DC certificate renew.
i.e., outside of the periodic subcloud audit tasks.
We allow a re-enqueue here with a new delay.
"""
if self.sc_audit_queue.contains(subcloud_name):
if (allow_requeue
and self.sc_audit_queue.enqueued_subcloud_names.count(
subcloud_name) < 2):
LOG.info("audit_subcloud: requeing %s" % subcloud_name)
else:
LOG.debug("audit_subcloud: ignoring %s, already in queue"
% subcloud_name)
return
self.sc_audit_queue.enqueue(
subcloud_audit_queue.SubcloudAuditData(subcloud_name),
allow_requeue=allow_requeue)
def monitor_cert(self, monitor):
while True:
# never exit until exit signal received
try:
monitor.start_watch(
on_success=lambda task_id: self._purge_reattempt_task(
task_id, 'on success'),
on_error=lambda task: self._add_reattempt_task(task),
)
on_success=lambda task_id:
self._purge_reattempt_monitor_task(task_id,
'on success'),
on_error=lambda task:
self._add_reattempt_monitor_task(task))
except greenlet.GreenletExit:
break
except Exception:
@ -282,28 +404,15 @@ class CertificateMonManager(periodic_task.PeriodicTasks):
LOG.exception("Unexpected exception from start_watch")
time.sleep(1)
def _add_reattempt_task(self, task):
def _add_reattempt_monitor_task(self, task):
id = task.get_id()
self._purge_reattempt_task(id, 'for new reattempt')
self.reattempt_tasks.append(task)
self._purge_reattempt_monitor_task(id, 'for new reattempt')
self.reattempt_monitor_tasks.append(task)
def _purge_reattempt_task(self, id, reason_msg):
for t in self.reattempt_tasks:
def _purge_reattempt_monitor_task(self, id, reason_msg):
for t in self.reattempt_monitor_tasks:
if t.get_id() == id:
self.reattempt_tasks.remove(t)
LOG.info('Purging reattempt task %s: %s' % (reason_msg, id))
self.reattempt_monitor_tasks.remove(t)
LOG.info('Purging reattempt monitor task %s: %s'
% (reason_msg, id))
break
def requeue_audit(self, subcloud_name):
# move the subcloud to the end of the queue for auditing
# adding enough spaces so that the renewal would complete by
# next audit
if subcloud_name in self.subclouds_to_audit:
self.subclouds_to_audit.remove(subcloud_name)
for i in range(12, self.subclouds_to_audit.count(TASK_NAME_PAUSE_AUDIT), -1):
self.subclouds_to_audit.append(TASK_NAME_PAUSE_AUDIT)
self.subclouds_to_audit.append(subcloud_name)
def audit_subcloud(self, subcloud_name):
if subcloud_name not in self.subclouds_to_audit:
self.subclouds_to_audit.append(subcloud_name)

View File

@ -0,0 +1,99 @@
# 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.
#
# Copyright (c) 2021 Wind River Systems, Inc.
#
# The right to copy, distribute, modify, or otherwise make use
# of this software may be licensed only pursuant to the terms
# of an applicable Wind River license agreement.
#
import heapq
import time
from eventlet.queue import PriorityQueue
from oslo_log import log
LOG = log.getLogger(__name__)
class SubcloudAuditData(object):
"""Representation of a subcloud under audit.
The 'name' field is used for all comparisons.
"""
def __init__(self, name, audit_count=0):
self.name = name
self.audit_count = audit_count
def __eq__(self, other):
return self.name == other.name
def __hash__(self):
return hash(self.name)
def __str__(self):
return "SubcloudAuditData: %s, audit_count: %s" % (self.name,
self.audit_count)
class SubcloudAuditException(Exception):
"""Indicates subcloud audit issue"""
pass
class SubcloudAuditPriorityQueue(PriorityQueue):
"""A subclass of PriorityQueue which tracks enqueued subclouds"""
def _init(self, maxsize=None):
self.enqueued_subcloud_names = list()
PriorityQueue._init(self, maxsize)
@staticmethod
def __get_next_audit_timestamp(delay_secs):
next_audit_timestamp = int(time.time())
if delay_secs > 0:
next_audit_timestamp += delay_secs
return next_audit_timestamp
def contains(self, subcloud_name):
"""Check if subcloud is under audit"""
return subcloud_name in self.enqueued_subcloud_names
def enqueue(self, sc_audit_item, delay_secs=0,
timestamp=None, allow_requeue=False):
"""Custom top-level method to enqueue a subcloud in the audit
- convert delay to timestamp
- increment audit_count
"""
if (sc_audit_item.name in self.enqueued_subcloud_names
and not allow_requeue):
raise SubcloudAuditException("Subcloud already enqueued: %s"
% sc_audit_item.name)
if timestamp is None:
timestamp = self.__get_next_audit_timestamp(delay_secs)
# this PriorityQueue is ordered by the next timestamp:
sc_audit_item.audit_count += 1
self.put(
(timestamp, sc_audit_item)
)
def _get(self, heappop=heapq.heappop):
"""Modifies PriorityQueue.get() to track audited subcloud names"""
item = PriorityQueue._get(self, heappop)
self.enqueued_subcloud_names.remove(item[1].name)
return item
def _put(self, item, heappush=heapq.heappush):
"""Modifies PriorityQueue.put() to track audited subcloud names"""
subcloud_audit = item[1]
self.enqueued_subcloud_names.append(subcloud_audit.name)
LOG.info("Enqueued: %s" % str(subcloud_audit))
PriorityQueue._put(self, item, heappush)

View File

@ -227,8 +227,8 @@ def update_subcloud_ca_cert(
api_cmd_payload['sc_ca_key'] = tls_key
timeout = int(CONF.endpoint_cache.http_connect_timeout)
resp = rest_api_request(token, "POST", api_cmd, json.dumps(api_cmd_payload),
timeout=timeout)
resp = rest_api_request(token, "POST", api_cmd,
json.dumps(api_cmd_payload), timeout=timeout)
if 'result' in resp and resp['result'] == 'OK':
LOG.info('Update %s intermediate CA cert request succeed' % sc_name)
@ -298,13 +298,16 @@ def update_subcloud_status(token, subcloud_name, status):
api_cmd_payload = dict()
api_cmd_payload['endpoint'] = ENDPOINT_TYPE_DC_CERT
api_cmd_payload['status'] = status
resp = rest_api_request(token, "PATCH", api_cmd, json.dumps(api_cmd_payload))
resp = rest_api_request(token, "PATCH",
api_cmd, json.dumps(api_cmd_payload))
if 'result' in resp and resp['result'] == 'OK':
LOG.info('Successfully updated subcloud %s status: %s' % (subcloud_name, status))
LOG.info('Updated subcloud %s status: %s' % (subcloud_name, status))
else:
LOG.error('Failed to update subcloud %s status to %s, resp=%s' % (subcloud_name, status, resp))
raise Exception('Update subcloud status failed, subcloud=%s' % subcloud_name)
LOG.error("Failed to update subcloud %s status to '%s', resp=%s"
% (subcloud_name, status, resp))
raise Exception('Update subcloud status failed, subcloud=%s'
% subcloud_name)
def rest_api_request(token, method, api_cmd,

View File

@ -0,0 +1,130 @@
# Copyright (c) 2021 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
"""
Test class for Sysinv subcloud_audit
"""
import time
from sysinv.cert_mon.subcloud_audit_queue import SubcloudAuditData
from sysinv.cert_mon.subcloud_audit_queue import SubcloudAuditException
from sysinv.cert_mon.subcloud_audit_queue import SubcloudAuditPriorityQueue
from sysinv.tests.db import base
class SubcloudAuditTestCase(base.DbTestCase):
"""Test cases for subcloud_audit.py classes"""
def setUp(self):
super(SubcloudAuditTestCase, self).setUp()
# Set up objects for testing
self.sc_audit_queue = SubcloudAuditPriorityQueue()
def tearDown(self):
super(SubcloudAuditTestCase, self).tearDown()
def test_audit_item(self):
print("Running test_audit_item")
item1 = SubcloudAuditData("item1")
self.assertEqual(item1.name, "item1")
self.assertEqual(item1.audit_count, 0)
self.assertEqual(item1, SubcloudAuditData("item1", 0))
self.assertEqual(item1, SubcloudAuditData("item1", 1))
def test_subcloud_audit_queue_single(self):
sc_name = "subcloud1"
subcloud = SubcloudAuditData(sc_name)
self.sc_audit_queue.enqueue(subcloud)
assert self.sc_audit_queue.contains(sc_name)
assert self.sc_audit_queue.qsize() == 1
# peek using the underlying queue
_, sc_audit_item1 = self.sc_audit_queue.queue[0]
assert sc_audit_item1.name == sc_name
assert sc_audit_item1.audit_count == 1
def test_subcloud_audit_queue_multiple(self):
subclouds = [SubcloudAuditData("subcloud%s" % i) for i in range(20)]
delay = 0
for i in range(20):
self.sc_audit_queue.enqueue(subclouds[i], delay)
delay += 10
assert self.sc_audit_queue.qsize() == 20
_, first = self.sc_audit_queue.get()
assert first.name == subclouds[0].name
assert not self.sc_audit_queue.contains(subclouds[0].name)
assert self.sc_audit_queue.qsize() == 19
# re-enqueue with no delay; it should come out first again
self.sc_audit_queue.enqueue(first, 0)
_, first = self.sc_audit_queue.get()
assert first.name == subclouds[0].name
timestamp, second = self.sc_audit_queue.get()
assert second.name == subclouds[1].name
# The time now should be well under the timestamp for this item
assert int(time.time()) < timestamp
def test_subcloud_audit_queue_custom_timestamp(self):
subclouds = [SubcloudAuditData("subcloud%s" % i) for i in range(20)]
timestamp = 0
for i in range(20):
self.sc_audit_queue.enqueue(subclouds[i], timestamp=timestamp)
timestamp += 10
assert self.sc_audit_queue.qsize() == 20
_, first = self.sc_audit_queue.get()
assert first.name == subclouds[0].name
assert not self.sc_audit_queue.contains(subclouds[0].name)
assert self.sc_audit_queue.qsize() == 19
# re-enqueue with no delay; it should come out first again
self.sc_audit_queue.enqueue(first, timestamp=0)
_, first = self.sc_audit_queue.get()
assert first.name == subclouds[0].name
assert first == subclouds[0]
self.sc_audit_queue.enqueue(subclouds[0], timestamp=10000)
prev_timestamp = 0
for i in range(19):
next_timestamp, next_item = self.sc_audit_queue.get()
assert next_timestamp > prev_timestamp
assert next_item.name != subclouds[0].name
prev_timestamp = next_timestamp
next_timestamp, next_item = self.sc_audit_queue.get()
assert next_timestamp == 10000
assert next_item.name == subclouds[0].name
def test_subcloud_audit_requeue(self):
subclouds = [SubcloudAuditData("subcloud%s" % i, 0) for i in range(20)]
timestamp = 0
for i in range(20):
self.sc_audit_queue.enqueue(subclouds[i], timestamp=timestamp)
timestamp += 10
assert self.sc_audit_queue.qsize() == 20
assert self.sc_audit_queue.contains(subclouds[0].name)
got_exception = False
try:
self.sc_audit_queue.enqueue(subclouds[0], timestamp=timestamp)
except SubcloudAuditException:
got_exception = True
assert got_exception
got_exception = False
try:
self.sc_audit_queue.enqueue(
subclouds[0], timestamp=timestamp, allow_requeue=True
)
except SubcloudAuditException:
got_exception = True
assert not got_exception
count = 0
for name in self.sc_audit_queue.enqueued_subcloud_names:
if name == subclouds[0].name:
count += 1
assert count == 2