distcloud/distributedcloud/dcmanager/manager/peer_group_audit_manager.py
Li Zhu 7ad78ea2ae Allow rehome related data update when subcloud migration fails
If the subcloud rehome_data contains an incorrect bootstrap-address in
site A and the user migrates the corresponding peer group to site B,
the migration would fail. Subsequently, it will have the 'rehome-failed'
deploy-status in site B and 'rehome-pending' deploy-status in site A.
Then the user won't be able to update the bootstrap-address in either
site due to the following restrictions:
a) Primary site (site A) is not the current leader of the peer group;
b) Update in non-primary site (site B) is not allowed.

To fix this issue, the following changes are made:
1. In the non-primary site, if the subcloud deploy-status is
rehome-failed and the primary site is unavailable, updating
the bootstrap-values and bootstrap-address will be allowed, and the PGA
will be marked as out-of-sync.
2. Modify audit to automatically sync the rehome_data from non-primary
site to primary site if subcloud in the non-primary site is managed and
online and the PGA is out-of-sync.

Additional fix for the system_leader_id issue: When migrating SPG from
one site to another, if all of the subclouds rehome fail, the leader id
of the SPG in the target site has already been updated to the target
site's UUID. However, in the source site, the leader id is not updated
to the target UUID. The fix ensures that regardless of the migration's
success, only if the migration completes, the leader id in both sites
should be updated to the target UUID.

Test plan:
Pre-Steps: 1. Create the system peer from Site A to Site B
           2. Create System peer from Site B to Site A
           3. Create the subcloud peer group in the Site A
           4. Add a subcloud with an incorrect bootstrap-address
              to the peer group
           5. Create peer group association to associate system peer
              and subcloud peer group - Site A
           6. Check current sync status in sites A and B. Verify
              they are 'in-sync'.
           7. Run migration for the subcloud peer group from Site B.
           8. Verify 'rehome-failed' deploy-status in both sites.
PASS: Verify that the bootstrap-address can be updated in site B when
      site A is down, and the PGA sync status is set to out-of-sync
      in site B. Also, verify that the audit will sync the rehome_data
      to site A and change back the PGA to in-sync once the reattempt of
      migration is successful and site A is up.
PASS: Verify that the bootstrap-values and bootstrap-address are
      the only fields that can be updated in site B when site A is down.
PASS: Verify that the update of bootstrap-address was rejected in site B
      when site A is up.
PASS: Verify that even if all of the subclouds in an SPG experience
      rehome failures, the system_leader_id in both sites is updated to
      the target's UUID.
PASS: Verify that when site A is always online or recovered during
      the migration to site B, the subcloud deploy_status in both sites
      is "rehome-failed" after the migration completes. In this
      scenario, site A can migrate the subcloud back, even though it's
      still failed. However, after correcting the bootstrap-address in
      site A, the reattempt of migration in site A succeeds.

Closes-Bug: 2057981

Change-Id: I999dbf035e29950fd823e9cdb087160ce40fd4ca
Signed-off-by: lzhu1 <li.zhu@windriver.com>
2024-03-20 11:27:56 -04:00

412 lines
19 KiB
Python

#
# Copyright (c) 2023-2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import threading
from oslo_config import cfg
from oslo_log import log as logging
from fm_api import constants as fm_const
from fm_api import fm_api
from dccommon import consts as dccommon_consts
from dcmanager.common import consts
from dcmanager.common import context
from dcmanager.common.i18n import _
from dcmanager.common import manager
from dcmanager.common import utils
from dcmanager.db import api as db_api
from dcmanager.manager.system_peer_manager import SystemPeerManager
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
class PeerGroupAuditManager(manager.Manager):
"""Manages audit related tasks."""
def __init__(self, subcloud_manager, peer_group_id, *args, **kwargs):
LOG.debug(_('PeerGroupAuditManager initialization...'))
super().__init__(service_name="peer_group_audit_manager",
*args, **kwargs)
self.context = context.get_admin_context()
self.fm_api = fm_api.FaultAPIs()
self.subcloud_manager = subcloud_manager
self.peer_group_id = peer_group_id
self.require_audit_flag = True
self.thread = None
self.thread_lock = threading.Lock()
def _get_subclouds_by_peer_group_from_system_peer(self,
dc_client,
system_peer,
peer_group_name):
try:
subclouds = dc_client.get_subcloud_list_by_peer_group(
peer_group_name)
return subclouds
except Exception:
LOG.exception(f"Failed to get subclouds of peer group "
f"{peer_group_name} from DC: "
f"{system_peer.peer_name}")
@staticmethod
def _get_association_sync_status_from_peer_site(dc_client,
system_peer,
peer_group_id):
try:
# Get peer site system peer
dc_peer_system_peer = dc_client.get_system_peer(
utils.get_local_system().uuid)
association = dc_client. \
get_peer_group_association_with_peer_id_and_pg_id(
dc_peer_system_peer.get('id'), peer_group_id)
return association.get("sync-status")
except Exception:
LOG.exception(f"Failed to get subclouds of peer group "
f"{peer_group_id} from DC: {system_peer.peer_name}")
def _update_remote_peer_group_migration_status(self,
system_peer,
peer_group_name,
migration_status):
dc_client = SystemPeerManager.get_peer_dc_client(system_peer)
peer_group_kwargs = {
'migration_status': migration_status
}
dc_client.update_subcloud_peer_group(peer_group_name,
**peer_group_kwargs)
LOG.info(f"Updated Subcloud Peer Group {peer_group_name} on "
f"peer site {system_peer.peer_name}, set migration_status "
f"to: {migration_status}")
def _get_local_subclouds_to_update_and_delete(self,
local_peer_group,
remote_subclouds,
remote_sync_status):
local_subclouds_to_update = list()
local_subclouds_to_delete = list()
any_rehome_failed = False
remote_subclouds_dict = {remote_subcloud.get('region-name'):
remote_subcloud for remote_subcloud
in remote_subclouds}
local_subclouds = db_api.subcloud_get_for_peer_group(
self.context, local_peer_group.id)
for local_subcloud in local_subclouds:
remote_subcloud = remote_subclouds_dict.get(
local_subcloud.region_name)
if remote_subcloud:
# Check if the remote subcloud meets the conditions for update
# if it is 'managed' and the local subcloud is not
# in 'secondary' status
if (remote_subcloud.get('management-state') ==
dccommon_consts.MANAGEMENT_MANAGED and
not utils.subcloud_is_secondary_state(
local_subcloud.deploy_status)):
local_subclouds_to_update.append(local_subcloud)
# Sync rehome_data from remote to local subcloud if the remote
# PGA sync_status is out-of-sync once migration completes,
# indicating any bootstrap values/address updates to
# the subcloud on the remote site.
if remote_sync_status == \
consts.ASSOCIATION_SYNC_STATUS_OUT_OF_SYNC:
self._sync_rehome_data(
local_subcloud.id, remote_subcloud.get('rehome_data'))
elif remote_subcloud.get('deploy-status') in \
(consts.DEPLOY_STATE_REHOME_FAILED,
consts.DEPLOY_STATE_REHOME_PREP_FAILED):
# Set local subcloud to rehome-failed if the remote is
# rehome-failed or rehome-prep-failed, otherwise, the
# deploy_status will remain rehome-pending, which will
# block the correction of the bootstrap values/address.
db_api.subcloud_update(
self.context, local_subcloud.id,
deploy_status=consts.DEPLOY_STATE_REHOME_FAILED)
any_rehome_failed = True
else:
local_subclouds_to_delete.append(local_subcloud)
return local_subclouds_to_update, local_subclouds_to_delete, \
any_rehome_failed
def _set_local_subcloud_to_secondary(self, subcloud):
try:
LOG.info("Set local subcloud %s to secondary" % subcloud.name)
# There will be an exception when unmanage
# a subcloud in 'unamaged' state.
if subcloud.management_state != \
dccommon_consts.MANAGEMENT_UNMANAGED:
self.subcloud_manager.update_subcloud(
self.context,
subcloud.id,
management_state=dccommon_consts.
MANAGEMENT_UNMANAGED)
self.subcloud_manager.update_subcloud(
self.context,
subcloud.id,
deploy_status=consts.DEPLOY_STATE_SECONDARY)
except Exception as e:
LOG.exception(f"Failed to update local non-secondary "
f"and offline subcloud [{subcloud.name}], err: {e}")
raise e
def _sync_rehome_data(self, subcloud_id, rehome_data):
db_api.subcloud_update(self.context, subcloud_id, rehome_data=rehome_data)
def audit(self, system_peer, remote_peer_group, local_peer_group):
if local_peer_group.migration_status == consts.PEER_GROUP_MIGRATING:
LOG.info("Local peer group in migrating state, quit audit")
return
LOG.info("Auditing remote subcloud peer group:[%s] "
"migration_status:[%s] group_priority[%s], "
"local subcloud peer group:[%s] "
"migration_status:[%s] group_priority[%s]" %
(remote_peer_group.get("peer_group_name"),
remote_peer_group.get("migration_status"),
remote_peer_group.get("group_priority"),
local_peer_group.peer_group_name,
local_peer_group.migration_status,
local_peer_group.group_priority))
# if remote subcloud peer group's migration_status is 'migrating',
# 'unmanaged' all local subclouds in local peer group and change its
# deploy status to consts.DEPLOY_STATE_REHOME_PENDING to stop cert-mon
# audits.
if remote_peer_group.get("migration_status") == \
consts.PEER_GROUP_MIGRATING:
# Unmanaged all local subclouds of peer group
LOG.info(f"Unmanaged all local subclouds of peer group "
f"{local_peer_group.peer_group_name} "
f"since remote is in migrating state")
subclouds = db_api.subcloud_get_for_peer_group(self.context,
local_peer_group.id)
for subcloud in subclouds:
try:
# update_subcloud raises an exception when trying to umanage
# an already unmanaged subcloud, so the deploy status
# update must be done separately
if subcloud.management_state != \
dccommon_consts.MANAGEMENT_UNMANAGED:
# Unmanage and update the deploy-status
LOG.info("Unmanaging and setting the local subcloud "
f"{subcloud.name} deploy status to "
f"{consts.DEPLOY_STATE_REHOME_PENDING}")
self.subcloud_manager.update_subcloud(
self.context,
subcloud.id,
management_state=dccommon_consts.
MANAGEMENT_UNMANAGED,
deploy_status=consts.DEPLOY_STATE_REHOME_PENDING)
else:
# Already unmanaged, just update the deploy-status
LOG.info(f"Setting the local subcloud {subcloud.name} "
"deploy status to "
f"{consts.DEPLOY_STATE_REHOME_PENDING}")
self.subcloud_manager.update_subcloud(
self.context,
subcloud.id,
deploy_status=consts.DEPLOY_STATE_REHOME_PENDING)
except Exception as e:
LOG.exception(f"Fail to unmanage local subcloud "
f"{subcloud.name}, err: {e}")
raise e
SystemPeerManager.update_sync_status(
self.context, system_peer,
consts.ASSOCIATION_SYNC_STATUS_OUT_OF_SYNC,
local_peer_group, remote_peer_group)
self.require_audit_flag = False
# if remote subcloud peer group's migration_status is 'complete',
# get remote subclouds. For 'managed+online' subclouds,
# set 'unmanaged+secondary' to local on same subclouds
elif remote_peer_group.get("migration_status") == \
consts.PEER_GROUP_MIGRATION_COMPLETE:
dc_client = SystemPeerManager.get_peer_dc_client(system_peer)
remote_subclouds = \
self._get_subclouds_by_peer_group_from_system_peer(
dc_client,
system_peer,
remote_peer_group.get("peer_group_name"))
remote_sync_status = \
self._get_association_sync_status_from_peer_site(
dc_client,
system_peer,
remote_peer_group.get("id"))
local_subclouds_to_update, local_subclouds_to_delete, \
any_rehome_failed = \
self._get_local_subclouds_to_update_and_delete(
local_peer_group, remote_subclouds, remote_sync_status)
for subcloud in local_subclouds_to_update:
self._set_local_subcloud_to_secondary(subcloud)
# Change the local subcloud not exist on peer site's SPG to
# secondary status then delete it
for subcloud in local_subclouds_to_delete:
self._set_local_subcloud_to_secondary(subcloud)
try:
self.subcloud_manager.delete_subcloud(
self.context, subcloud.id)
LOG.info(f"Deleted local subcloud {subcloud.name}")
except Exception as e:
SystemPeerManager.update_sync_status(
self.context, system_peer,
consts.ASSOCIATION_SYNC_STATUS_OUT_OF_SYNC,
local_peer_group, remote_peer_group)
LOG.exception(f"Failed to delete local subcloud "
f"[{subcloud.name}] that does not exist "
f"under the same subcloud_peer_group on "
f"peer site, err: {e}")
raise e
if remote_peer_group.get("system_leader_id") == system_peer.peer_uuid:
self._clear_or_raise_alarm(system_peer,
local_peer_group,
remote_peer_group)
db_api.subcloud_peer_group_update(
self.context,
local_peer_group.id,
system_leader_id=system_peer.peer_uuid,
system_leader_name=system_peer.peer_name)
self._update_remote_peer_group_migration_status(
system_peer,
remote_peer_group.get("peer_group_name"),
None)
if not (remote_sync_status == consts.ASSOCIATION_SYNC_STATUS_OUT_OF_SYNC
and any_rehome_failed):
SystemPeerManager.update_sync_status(
self.context, system_peer,
consts.ASSOCIATION_SYNC_STATUS_IN_SYNC,
local_peer_group, remote_peer_group)
self.require_audit_flag = False
else:
# If remote peer group migration_status is 'None'
self.require_audit_flag = False
def _clear_or_raise_alarm(self,
system_peer,
local_peer_group,
remote_peer_group):
# If local subcloud peer group's group_priority is
# lower than remote subcloud peer group's group_priority,
# an alarm will be raised.
# lower number means higher priority
entity_instance_id = "peer_group=%s,peer=%s" % \
(local_peer_group.peer_group_name, system_peer.peer_uuid)
if local_peer_group.group_priority < remote_peer_group.get('group_priority'):
LOG.warning("Alarm: local subcloud peer group ["
f"{local_peer_group.peer_group_name}] "
f"is managed by remote system ["
f"{system_peer.peer_name}]")
try:
fault = fm_api.Fault(
alarm_id=fm_const.
FM_ALARM_ID_DC_SUBCLOUD_PEER_GROUP_NOT_MANAGED,
alarm_state=fm_const.FM_ALARM_STATE_SET,
entity_type_id=fm_const.
FM_ENTITY_TYPE_SUBCLOUD_PEER_GROUP,
entity_instance_id=entity_instance_id,
severity=fm_const.FM_ALARM_SEVERITY_MAJOR,
reason_text=("Subcloud peer group "
"(peer_group_name=%s) "
"is managed by remote "
"system (peer_uuid=%s) "
"with a lower priority." %
(local_peer_group.peer_group_name,
system_peer.peer_uuid)),
alarm_type=fm_const.FM_ALARM_TYPE_0,
probable_cause=fm_const.
ALARM_PROBABLE_CAUSE_UNKNOWN,
proposed_repair_action="Check the reported peer group "
"state. Migrate it back to the current system if the "
"state is 'rehomed' and the current system is stable. "
"Otherwise, wait until these conditions are met.",
service_affecting=False)
self.fm_api.set_fault(fault)
except Exception as e:
LOG.exception(e)
else:
try:
fault = self.fm_api.get_fault(
fm_const.FM_ALARM_ID_DC_SUBCLOUD_PEER_GROUP_NOT_MANAGED,
entity_instance_id)
if fault:
LOG.info(f"Clear alarm: {entity_instance_id}")
self.fm_api.clear_fault(
fm_const.FM_ALARM_ID_DC_SUBCLOUD_PEER_GROUP_NOT_MANAGED,
entity_instance_id)
except Exception:
LOG.exception(
f"Problem clearing fault [{entity_instance_id}], "
f"alarm_id="
f"{fm_const.FM_ALARM_ID_DC_SUBCLOUD_PEER_GROUP_NOT_MANAGED}")
def _do_audit(self, system_peer, remote_peer_group, local_peer_group):
with self.thread_lock:
try:
self.audit(system_peer, remote_peer_group, local_peer_group)
except Exception as e:
LOG.exception("audit error occured: %s" % e)
def stop(self):
if self.thread:
self.thread.join()
LOG.info(f"stopped peer group {self.peer_group_id} audit thread")
else:
LOG.info(f"No peer group {self.peer_group_id} audit thread to stop")
def start(self, system_peer, remote_peer_group, local_peer_group):
if self.thread_lock.locked():
LOG.warning(f"Audit thread for {local_peer_group.peer_group_name} "
f"has already started")
else:
self.thread = threading.Thread(
target=self._do_audit,
args=(system_peer, remote_peer_group, local_peer_group))
self.thread.start()
def audit_peer_group_from_system(self,
system_peer,
remote_peer_group,
local_peer_group):
LOG.info(f"Audit peer group [{local_peer_group.peer_group_name}] "
f"with remote system {system_peer.peer_name}")
self.start(system_peer, remote_peer_group, local_peer_group)
@staticmethod
def send_audit_peer_group(system_peers, peer_group):
if not system_peers:
return
local_system = utils.get_local_system()
for system in system_peers:
try:
dc_client = SystemPeerManager.get_peer_dc_client(system)
payload = db_api.subcloud_peer_group_db_model_to_dict(
peer_group)
if 'created-at' in payload:
del payload['created-at']
if 'updated-at' in payload:
del payload['updated-at']
payload['peer_uuid'] = local_system.uuid
LOG.info("Send audit payload [%s] of peer group %s" %
(payload, peer_group.peer_group_name))
response = dc_client.audit_subcloud_peer_group(
peer_group.peer_group_name,
**payload)
if response:
return response
except Exception:
LOG.exception("Failed to send audit request for peer group "
f"{peer_group.peer_group_name} to DC: "
f"{system.peer_name}")