Create USM dummy states and tests

This commit introduces dummy classes for all states
that will be utilized in the new USM API. The implementation
of these classes will be addressed in subsequent commits.

Test Plan:
1. Perform an upgrade-strategy with use_usm=True
- Check software orchestration will be called.
- All states will execute and only move to the next state
- Check the usm states tests pass.

Story: 2010676
Task: 48153

Change-Id: I3d2c236656d5caee6a4a827d03146e6c160103aa
Signed-off-by: Hugo Brito <hugo.brito@windriver.com>
This commit is contained in:
Hugo Brito 2023-09-15 17:28:38 -03:00 committed by Hugo Nicodemos
parent 83e5d159a8
commit 40c8359f41
39 changed files with 1194 additions and 21 deletions

View File

@ -78,6 +78,7 @@ SW_UPDATE_TYPE_KUBERNETES = "kubernetes"
SW_UPDATE_TYPE_PATCH = "patch"
SW_UPDATE_TYPE_PRESTAGE = "prestage"
SW_UPDATE_TYPE_UPGRADE = "upgrade"
SW_UPDATE_TYPE_SOFTWARE = "software"
# Software update states
SW_UPDATE_STATE_INITIAL = "initial"
@ -144,6 +145,22 @@ STRATEGY_STATE_CREATING_VIM_UPGRADE_STRATEGY = "creating VIM upgrade strategy"
STRATEGY_STATE_APPLYING_VIM_UPGRADE_STRATEGY = "applying VIM upgrade strategy"
STRATEGY_STATE_DELETING_LOAD = "deleting load"
# Software orchestration states
STRATEGY_STATE_SW_PRE_CHECK = "software pre check"
STRATEGY_STATE_SW_INSTALL_LICENSE = "software install license"
STRATEGY_STATE_SW_UPLOAD = "software upload"
STRATEGY_STATE_SW_DEPLOY_PRE_CHECK = "software deploy pre check"
STRATEGY_STATE_SW_DEPLOY_START = "software deploy start"
STRATEGY_STATE_SW_LOCK_CONTROLLER = "software lock controller"
STRATEGY_STATE_SW_UNLOCK_CONTROLLER = "software unlock controller"
STRATEGY_STATE_SW_SWACT_CONTROLLER = "software swact controller"
STRATEGY_STATE_SW_DEPLOY_HOST = "software deploy host"
STRATEGY_STATE_SW_DEPLOY_ACTIVATE = "software deploy activate"
STRATEGY_STATE_SW_DEPLOY_COMPLETE = "software deploy complete"
STRATEGY_STATE_SW_CREATE_VIM_STRATEGY = "create VIM software strategy"
STRATEGY_STATE_SW_APPLY_VIM_STRATEGY = "apply VIM software strategy"
STRATEGY_STATE_SW_FINISH_STRATEGY = "finish software strategy"
# Firmware update orchestration states
STRATEGY_STATE_IMPORTING_FIRMWARE = "importing firmware"
STRATEGY_STATE_CREATING_FW_UPDATE_STRATEGY = "creating fw update strategy"

View File

@ -8,8 +8,37 @@ from oslo_log import log as logging
from dccommon.drivers.openstack import vim
from dcmanager.common import consts
from dcmanager.db import api as db_api
from dcmanager.orchestrator.orch_thread import OrchThread
from dcmanager.orchestrator.states.software.apply_vim_software_strategy \
import ApplyVIMSoftwareStrategyState
from dcmanager.orchestrator.states.software.cache.shared_cache_repository import \
SharedCacheRepository
from dcmanager.orchestrator.states.software.create_vim_software_strategy \
import CreateVIMSoftwareStrategyState
from dcmanager.orchestrator.states.software.deploy_activate \
import DeployActivateState
from dcmanager.orchestrator.states.software.deploy_complete \
import DeployCompleteState
from dcmanager.orchestrator.states.software.deploy_host \
import DeployHostState
from dcmanager.orchestrator.states.software.deploy_pre_check \
import DeployPreCheckState
from dcmanager.orchestrator.states.software.deploy_start \
import DeployStartState
from dcmanager.orchestrator.states.software.finish_strategy \
import FinishStrategyState
from dcmanager.orchestrator.states.software.install_license \
import InstallLicenseState
from dcmanager.orchestrator.states.software.lock_controller \
import LockControllerState
from dcmanager.orchestrator.states.software.pre_check \
import PreCheckState
from dcmanager.orchestrator.states.software.swact_controller \
import SwactControllerState
from dcmanager.orchestrator.states.software.unlock_controller \
import UnlockControllerState
from dcmanager.orchestrator.states.software.upload \
import UploadState
LOG = logging.getLogger(__name__)
@ -31,33 +60,47 @@ class SoftwareOrchThread(OrchThread):
database as it goes, with state and progress information.
"""
# every state in sw upgrade orchestration should have an operator
STATE_OPERATORS = {
consts.STRATEGY_STATE_SW_PRE_CHECK: PreCheckState,
consts.STRATEGY_STATE_SW_INSTALL_LICENSE: InstallLicenseState,
consts.STRATEGY_STATE_SW_UPLOAD: UploadState,
consts.STRATEGY_STATE_SW_DEPLOY_PRE_CHECK: DeployPreCheckState,
consts.STRATEGY_STATE_SW_DEPLOY_START: DeployStartState,
consts.STRATEGY_STATE_SW_LOCK_CONTROLLER: LockControllerState,
consts.STRATEGY_STATE_SW_DEPLOY_HOST: DeployHostState,
consts.STRATEGY_STATE_SW_UNLOCK_CONTROLLER: UnlockControllerState,
consts.STRATEGY_STATE_SW_SWACT_CONTROLLER: SwactControllerState,
consts.STRATEGY_STATE_SW_CREATE_VIM_STRATEGY: CreateVIMSoftwareStrategyState,
consts.STRATEGY_STATE_SW_APPLY_VIM_STRATEGY: ApplyVIMSoftwareStrategyState,
consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE: DeployActivateState,
consts.STRATEGY_STATE_SW_DEPLOY_COMPLETE: DeployCompleteState,
consts.STRATEGY_STATE_SW_FINISH_STRATEGY: FinishStrategyState,
}
def __init__(self, strategy_lock, audit_rpc_client):
super(SoftwareOrchThread, self).__init__(
strategy_lock,
audit_rpc_client,
consts.SW_UPDATE_TYPE_UPGRADE, # software update strategy type
vim.STRATEGY_NAME_SW_UPGRADE, # strategy type used by vim
consts.STRATEGY_STATE_COMPLETE) # starting state
consts.SW_UPDATE_TYPE_UPGRADE, # software update strategy type
vim.STRATEGY_NAME_SW_UPGRADE, # strategy type used by vim
consts.STRATEGY_STATE_SW_PRE_CHECK) # starting state
# Initialize shared cache instances for the states that require them
self._shared_caches = SharedCacheRepository(consts.SW_UPDATE_TYPE_SOFTWARE)
self._shared_caches.initialize_caches()
def trigger_audit(self):
"""Trigger an audit for upgrade (which is combined with patch audit)"""
self.audit_rpc_client.trigger_patch_audit(self.context)
def delete(self, sw_update_strategy):
super(SoftwareOrchThread, self).delete(sw_update_strategy)
def pre_apply_setup(self):
# Restart caches for next strategy
self._shared_caches.initialize_caches()
super(SoftwareOrchThread, self).pre_apply_setup()
def apply(self, sw_update_strategy):
LOG.info("(%s) Applying update strategy" % self.update_type)
LOG.info("(%s) Strategy application is complete."
% self.update_type)
with self.strategy_lock:
db_api.sw_update_strategy_update(
self.context,
state=consts.SW_UPDATE_STATE_COMPLETE,
update_type=self.update_type)
self.subcloud_workers.clear()
# Trigger audit to update the sync status for each subcloud.
self.trigger_audit()
return
def determine_state_operator(self, strategy_step):
state = super(SoftwareOrchThread, self).determine_state_operator(
strategy_step)
state.add_shared_caches(self._shared_caches)
return state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class ApplyVIMSoftwareStrategyState(BaseState):
"""Apply VIM Software Strategy software orchestration state"""
def __init__(self, region_name):
super(ApplyVIMSoftwareStrategyState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE,
region_name=region_name
)
def perform_state_action(self, strategy_step):
"""Apply VIM Software Strategy region status"""
return self.next_state

View File

@ -0,0 +1,90 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.software.cache import clients
from dcmanager.orchestrator.states.software.cache.clients import \
CLIENT_READ_EXCEPTIONS
from dcmanager.orchestrator.states.software.cache.clients import \
CLIENT_READ_MAX_ATTEMPTS
class CacheSpecification(object):
def __init__(self, fetch_implementation,
post_filter_implementation=None, valid_filters=frozenset(),
retry_on_exception=CLIENT_READ_EXCEPTIONS,
max_attempts=CLIENT_READ_MAX_ATTEMPTS,
retry_sleep_msecs=consts.PLATFORM_RETRY_SLEEP_MILLIS):
"""Create cache specification.
:param fetch_implementation: implementation on how to retrieve data from
client
:type fetch_implementation: function
:param post_filter_implementation: implementation on how to post-filter
cached data, if any
:type post_filter_implementation: function
:param valid_filters: valid post-filter parameters
:type valid_filters: set
:param retry_on_exception: exceptions to be retried on client read
:type retry_on_exception: type|tuple
:param max_attempts: Maximum number of client read attempts if retryable
exceptions occur
:param retry_sleep_msecs: Fixed backoff interval
"""
self.fetch_implementation = fetch_implementation
self.post_filter_implementation = post_filter_implementation
self.valid_filters = valid_filters
# Retry configurations
self.retry_on_exception = retry_on_exception
self.max_attempts = max_attempts
self.retry_sleep_msecs = retry_sleep_msecs
"""Cache types"""
REGION_ONE_LICENSE_CACHE_TYPE = 'RegionOne system license'
REGION_ONE_SYSTEM_INFO_CACHE_TYPE = 'RegionOne system info'
REGION_ONE_RELEASE_USM_CACHE_TYPE = 'RegionOne release usm'
"""Cache specifications"""
REGION_ONE_LICENSE_CACHE_SPECIFICATION = CacheSpecification(
lambda: clients.get_sysinv_client().get_license())
REGION_ONE_SYSTEM_INFO_CACHE_SPECIFICATION = CacheSpecification(
lambda: clients.get_sysinv_client().get_system())
REGION_ONE_RELEASE_USM_CACHE_SPECIFICATION = CacheSpecification(
lambda: clients.get_software_client().query(),
# Filter results by patching state, if any is given
lambda patches, **filter_params: {
patch_id: patch for patch_id, patch in patches.items()
if filter_params.get('state') is None
or patch.get('state') == filter_params.get('state')
},
{'state'}
)
# Map each expected operation type to its required cache types
CACHE_TYPES_BY_OPERATION_TYPE = {
consts.SW_UPDATE_TYPE_SOFTWARE: {REGION_ONE_LICENSE_CACHE_TYPE,
REGION_ONE_SYSTEM_INFO_CACHE_TYPE,
REGION_ONE_RELEASE_USM_CACHE_TYPE}
}
# Map each cache type to its corresponding cache specification
SPECIFICATION_BY_CACHE_TYPE = {
REGION_ONE_LICENSE_CACHE_TYPE: REGION_ONE_LICENSE_CACHE_SPECIFICATION,
REGION_ONE_SYSTEM_INFO_CACHE_TYPE: REGION_ONE_SYSTEM_INFO_CACHE_SPECIFICATION,
REGION_ONE_RELEASE_USM_CACHE_TYPE: REGION_ONE_RELEASE_USM_CACHE_SPECIFICATION
}
def get_specifications_for_operation(operation_type):
# Retrieve all cache specifications required by a given operation type
# Return a mapping between each required type to its corresponding specification
return {cache_type: SPECIFICATION_BY_CACHE_TYPE.get(cache_type)
for cache_type in CACHE_TYPES_BY_OPERATION_TYPE.get(operation_type)}

View File

@ -0,0 +1,50 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import socket
from keystoneauth1 import exceptions as keystone_exceptions
from oslo_log import log as logging
from dccommon import consts as dccommon_consts
from dccommon.drivers.openstack.sdk_platform import OpenStackDriver
from dccommon.drivers.openstack.software_v1 import SoftwareClient
from dccommon.drivers.openstack.sysinv_v1 import SysinvClient
LOG = logging.getLogger(__name__)
""" Default timeout configurations for client reads """
CLIENT_READ_TIMEOUT_SECONDS = 60
CLIENT_READ_EXCEPTIONS = (socket.timeout, keystone_exceptions.ServiceUnavailable)
CLIENT_READ_MAX_ATTEMPTS = 2
""" Helper functions to retrieve clients for caching """
def get_sysinv_client():
ks_client = get_keystone_client()
return SysinvClient(dccommon_consts.DEFAULT_REGION_NAME, ks_client.session,
endpoint=ks_client.endpoint_cache.get_endpoint('sysinv'),
timeout=CLIENT_READ_TIMEOUT_SECONDS)
def get_software_client():
ks_client = get_keystone_client()
return SoftwareClient(dccommon_consts.DEFAULT_REGION_NAME, ks_client.session,
endpoint=ks_client.endpoint_cache.get_endpoint('usm'))
def get_keystone_client(region_name=dccommon_consts.DEFAULT_REGION_NAME):
"""Construct a (cached) keystone client (and token)"""
try:
os_client = OpenStackDriver(region_name=region_name,
region_clients=['sysinv'])
return os_client.keystone_client
except Exception:
LOG.warning('Failure initializing KeystoneClient for region: %s'
% region_name)
raise

View File

@ -0,0 +1,39 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from oslo_log import log
from dcmanager.common.exceptions import InvalidParameterValue
from dcmanager.orchestrator.states.software.cache import cache_specifications
from dcmanager.orchestrator.states.software.cache.shared_client_cache import \
SharedClientCache
LOG = log.getLogger(__name__)
class SharedCacheRepository(object):
def __init__(self, operation_type):
self._shared_caches = {}
self._operation_type = operation_type
def initialize_caches(self):
# Retrieve specifications for each cache type required by the operation
# Return mapping between each required type to a single cache instance of it
self._shared_caches = {
cache_type: SharedClientCache(cache_type, cache_specification)
for cache_type, cache_specification in
cache_specifications.get_specifications_for_operation(
self._operation_type).items()
}
def read(self, cache_type, **filter_params):
cache = self._shared_caches.get(cache_type)
if cache:
return cache.read(**filter_params)
else:
raise InvalidParameterValue(err="Specified cache type '%s' not "
"present" % cache_type)

View File

@ -0,0 +1,125 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import retrying
from oslo_concurrency import lockutils
from oslo_log import log
from dcmanager.common.exceptions import InvalidParameterValue
LOG = log.getLogger(__name__)
class SharedClientCache(object):
"""Data cache for sharing client or API data between concurrent threads
Used to avoid repeated requests and prevent client overload.
Cache is not self refreshing. User of the cache is responsible for triggering
the refresh.
"""
def __init__(self, cache_type, cache_specification):
"""Create cache instance.
:param cache_type: type of data being cached, for logging
:type cache_type: str
:param cache_specification: specifications on how the cache should
operate
:type cache_specification: dcmanager.orchestrator.states.upgrade.cache
.cache_specifications.CacheSpecification
"""
self._client_lock = lockutils.ReaderWriterLock()
self._cache = None
# Cache configurations
self._cache_type = cache_type
self._valid_filters = cache_specification.valid_filters
# Retry configurations
self._max_attempts = cache_specification.max_attempts
self._retry_sleep_msecs = cache_specification.retry_sleep_msecs
# Add retry to client read if any retryable exception is provided
self._load_data_from_client = cache_specification.fetch_implementation
retry_on_exception = cache_specification.retry_on_exception
if retry_on_exception:
retry = retrying.retry(retry_on_exception=lambda
ex: isinstance(ex, retry_on_exception),
stop_max_attempt_number=self._max_attempts,
wait_fixed=self._retry_sleep_msecs,
wait_func=self._retry_client_read)
self._load_data_from_client = \
retry(cache_specification.fetch_implementation)
# Use default implementation with no filtering if none is provided
self._post_filter_impl = cache_specification.post_filter_implementation\
or (lambda data, **filter_params: data)
def read(self, **filter_params):
"""Retrieve data from cache, if available.
Read from client and (re)populate cache, if not.
Only one thread may access the client at a time to prevent overload.
Concurrent reads are blocked until client read completes/fails. A recheck
for updates on the cache is performed afterwards.
Post-filtering can be applied to the results before returning. Data saved to
the cache will not include any filtering applied to returned data.
:param filter_params: parameters to be used for post-filtering
:type filter_params: string
:return: Cached data, filtered according to parameters given
:raises RuntimeError: If cache read fails due to concurrent client read error
:raises InvalidParameterError: If invalid filter parameters are given
"""
# Use data stored in the cache, if present. Otherwise, read and cache
# data from client
if self._cache is None:
self._cache_data_from_client()
# Filter cached data and return results
return self._post_filter(self._cache, **filter_params)
def _cache_data_from_client(self):
# Read from the client and update cache if no concurrent write is in progress
if self._client_lock.owner != lockutils.ReaderWriterLock.WRITER:
with self._client_lock.write_lock():
# Atomically fetch data from client and update the cache
LOG.info("Reading data from %s client for caching" %
self._cache_type)
self._cache = self._load_data_from_client()
else:
# If a concurrent write is in progress, wait for it and recheck cache
with self._client_lock.read_lock():
if self._cache is None:
raise RuntimeError("Failed to retrieve data from %s cache. "
"Possible failure on concurrent client "
"read." % self._cache_type)
def _retry_client_read(self, attempt, _):
# To be called when a client read operation fails with a retryable error
# After this, read operation should be retried
LOG.warn("Retryable error occurred while reading from %s client "
"(Attempt %s/%s)" % (self._cache_type, attempt, self._max_attempts))
return self._retry_sleep_msecs
def _post_filter(self, data, **filter_params):
# Validate the parameters and apply specified filter implementation
self._validate_filter_params(**filter_params)
return self._post_filter_impl(data, **filter_params)
def _validate_filter_params(self, **filter_params):
# Compare each passed parameter against the specified valid parameters
# Raise an exception if any unexpected parameter is found
if filter_params:
invalid_params = set(filter_params.keys()) - self._valid_filters
if invalid_params:
raise InvalidParameterValue(err="Invalid filter parameters: %s" %
invalid_params)

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class CreateVIMSoftwareStrategyState(BaseState):
"""Create VIM Software Strategy software orchestration state"""
def __init__(self, region_name):
super(CreateVIMSoftwareStrategyState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_APPLY_VIM_STRATEGY,
region_name=region_name
)
def perform_state_action(self, strategy_step):
"""Create VIM Software Strategy region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class DeployActivateState(BaseState):
"""Deploy activate software orchestration state"""
def __init__(self, region_name):
super(DeployActivateState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_COMPLETE,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Deploy Activate region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class DeployCompleteState(BaseState):
"""Deploy complete software orchestration state"""
def __init__(self, region_name):
super(DeployCompleteState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_FINISH_STRATEGY,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Deploy complete region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class DeployHostState(BaseState):
"""Deploy host software orchestration state"""
def __init__(self, region_name):
super(DeployHostState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_UNLOCK_CONTROLLER,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Deploy host region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class DeployPreCheckState(BaseState):
"""Deploy pre check software orchestration state"""
def __init__(self, region_name):
super(DeployPreCheckState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_START,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Deploy pre check region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class DeployStartState(BaseState):
"""Deploy start software orchestration state"""
def __init__(self, region_name):
super(DeployStartState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_LOCK_CONTROLLER,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Deploy Start region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class FinishStrategyState(BaseState):
"""Finish Strategy software orchestration state"""
def __init__(self, region_name):
super(FinishStrategyState, self).__init__(
next_state=consts.STRATEGY_STATE_COMPLETE,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Finish Strategy region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class InstallLicenseState(BaseState):
"""Install license software orchestration state"""
def __init__(self, region_name):
super(InstallLicenseState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_UPLOAD,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Install license region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class LockControllerState(BaseState):
"""Lock controller software orchestration state"""
def __init__(self, region_name):
super(LockControllerState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_HOST,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Lock controller region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class PreCheckState(BaseState):
"""Pre check software orchestration state"""
def __init__(self, region_name):
super(PreCheckState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_INSTALL_LICENSE,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Pre check region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class SwactControllerState(BaseState):
"""Swact controller software orchestration state"""
def __init__(self, region_name):
super(SwactControllerState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_CREATE_VIM_STRATEGY,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Swact controller region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class UnlockControllerState(BaseState):
"""Unlock controller software orchestration state"""
def __init__(self, region_name):
super(UnlockControllerState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Unlock controller region status"""
return self.next_state

View File

@ -0,0 +1,22 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.orchestrator.states.base import BaseState
class UploadState(BaseState):
"""Upload software orchestration state"""
def __init__(self, region_name):
super(UploadState, self).__init__(
next_state=consts.STRATEGY_STATE_SW_DEPLOY_PRE_CHECK,
region_name=region_name,
)
def perform_state_action(self, strategy_step):
"""Upload region status"""
return self.next_state

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestApplyVIMSoftwareStrategyState(TestSoftwareOrchestrator):
def setUp(self):
super(TestApplyVIMSoftwareStrategyState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_APPLY_VIM_STRATEGY)
def test_apply_vim_software_strategy_success(self):
"""Test apply vim software strategy when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,37 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import mock
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.test_base import TestSwUpdate
CACHE_CLIENT_PATH = "dcmanager.orchestrator.states.software.cache.clients"
class TestSoftwareOrchestrator(TestSwUpdate):
# Setting DEFAULT_STRATEGY_TYPE to software will setup the software
# orchestration worker, and will mock away the other orch threads
DEFAULT_STRATEGY_TYPE = consts.SW_UPDATE_TYPE_SOFTWARE
def setUp(self):
super(TestSoftwareOrchestrator, self).setUp()
# Modify cache helpers to return client mocks
self.software_cache_client_mock = mock.patch(
"%s.get_software_client" % CACHE_CLIENT_PATH,
return_value=self.software_client,
)
self.sysinv_cache_client_mock = mock.patch(
"%s.get_sysinv_client" % CACHE_CLIENT_PATH, return_value=self.sysinv_client
)
self.software_cache_client_mock.start()
self.sysinv_cache_client_mock.start()
def tearDown(self):
self.software_cache_client_mock.stop()
self.sysinv_cache_client_mock.stop()
super(TestSoftwareOrchestrator, self).tearDown()

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestCreateVIMSoftwareStrategyState(TestSoftwareOrchestrator):
def setUp(self):
super(TestCreateVIMSoftwareStrategyState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_APPLY_VIM_STRATEGY
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_CREATE_VIM_STRATEGY)
def test_create_vim_software_strategy_success(self):
"""Test create vim software strategy when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestDeployActivateState(TestSoftwareOrchestrator):
def setUp(self):
super(TestDeployActivateState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_COMPLETE
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE)
def test_deploy_activate_success(self):
"""Test deploy activate when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestDeployCompleteState(TestSoftwareOrchestrator):
def setUp(self):
super(TestDeployCompleteState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_FINISH_STRATEGY
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_DEPLOY_COMPLETE)
def test_deploy_complete_success(self):
"""Test deploy complete when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestDeployHostState(TestSoftwareOrchestrator):
def setUp(self):
super(TestDeployHostState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_UNLOCK_CONTROLLER
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_DEPLOY_HOST)
def test_deploy_host_success(self):
"""Test deploy host when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestDeployPreCheckState(TestSoftwareOrchestrator):
def setUp(self):
super(TestDeployPreCheckState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_START
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_DEPLOY_PRE_CHECK)
def test_deploy_pre_check_success(self):
"""Test deploy pre check when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestDeployStartState(TestSoftwareOrchestrator):
def setUp(self):
super(TestDeployStartState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_LOCK_CONTROLLER
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_DEPLOY_START)
def test_deploy_start_success(self):
"""Test deploy start when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestFinishStrategyState(TestSoftwareOrchestrator):
def setUp(self):
super(TestFinishStrategyState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_COMPLETE
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_FINISH_STRATEGY)
def test_finish_strategy_success(self):
"""Test finish strategy when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,35 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestInstallLicenseState(TestSoftwareOrchestrator):
def setUp(self):
super(TestInstallLicenseState, self).setUp()
# next state after install a license is 'upload'
self.on_success_state = consts.STRATEGY_STATE_SW_UPLOAD
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_INSTALL_LICENSE)
def test_upgrade_subcloud_license_install_success(self):
"""Test the install license step succeeds."""
# invoke the strategy state operation on the orch thread
self.worker.perform_state_action(self.strategy_step)
# On success, the next state after installing license is importing load
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestLockControllerState(TestSoftwareOrchestrator):
def setUp(self):
super(TestLockControllerState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_HOST
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_LOCK_CONTROLLER)
def test_lock_controller_success(self):
"""Test lock controller when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestPreCheckState(TestSoftwareOrchestrator):
def setUp(self):
super(TestPreCheckState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_INSTALL_LICENSE
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_PRE_CHECK)
def test_pre_check_success(self):
"""Test pre check when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestSwactControllerState(TestSoftwareOrchestrator):
def setUp(self):
super(TestSwactControllerState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_CREATE_VIM_STRATEGY
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_SWACT_CONTROLLER)
def test_swact_controller_success(self):
"""Test swact controller when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestUnlockControllerState(TestSoftwareOrchestrator):
def setUp(self):
super(TestUnlockControllerState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_ACTIVATE
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_UNLOCK_CONTROLLER)
def test_unlock_controller_success(self):
"""Test unlock controller when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -0,0 +1,32 @@
#
# Copyright (c) 2023 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
from dcmanager.common import consts
from dcmanager.tests.unit.orchestrator.states.software.test_base import \
TestSoftwareOrchestrator
class TestUploadState(TestSoftwareOrchestrator):
def setUp(self):
super(TestUploadState, self).setUp()
self.on_success_state = consts.STRATEGY_STATE_SW_DEPLOY_PRE_CHECK
# Add the subcloud being processed by this unit test
self.subcloud = self.setup_subcloud()
# Add the strategy_step state being processed by this unit test
self.strategy_step = self.setup_strategy_step(
self.subcloud.id, consts.STRATEGY_STATE_SW_UPLOAD)
def test_upload_success(self):
"""Test upload when the API call succeeds."""
self.worker.perform_state_action(self.strategy_step)
# On success, the state should transition to the next state
self.assert_step_updated(self.strategy_step.subcloud_id,
self.on_success_state)

View File

@ -115,6 +115,19 @@ class TestSwUpdate(base.DCManagerTestCase):
self.fake_sw_upgrade_orch_thread
self.addCleanup(p.stop)
if strategy_type == consts.SW_UPDATE_TYPE_SOFTWARE:
sw_update_manager.SoftwareOrchThread.stopped = lambda x: False
worker = sw_update_manager.SoftwareOrchThread(
mock_strategy_lock, mock_dcmanager_audit_api)
else:
# mock the software orch thread
self.fake_software_orch_thread = FakeOrchThread()
p = mock.patch.object(sw_update_manager, 'SoftwareOrchThread')
self.mock_software_orch_thread = p.start()
self.mock_software_orch_thread.return_value = \
self.fake_software_orch_thread
self.addCleanup(p.stop)
if strategy_type == consts.SW_UPDATE_TYPE_PATCH:
sw_update_manager.PatchOrchThread.stopped = lambda x: False
worker = \