DCorch Engine Update for Scalability

1. Refactor dcorch's generic_sync_manager.py and initial_sync_manager
   into a main process manager and a worker manager. The main manager
   will handle the allocation of eligible subclouds to each worker.
2. Rename the current EngineService to EngineWorkerService and introduce
   a new EngineService for the main process, similar to
   DCManagerAuditService and DCManagerAuditWorkerService.
3. Rename the current RPC EngineClient to EngineWorkerClient and
   introduce a new EngineClient. Adapt the RPC methods to accommodate
   the modifications in these main process managers and worker managers.
4. Move master resources data retrieval from each sync_thread to engine
   workers.
5. Implement 2 new db APIs for subcloud batch sync and state updates.
6. Remove code related to sync_lock and its associated db table schema.
7. Add ocf script for managing the start and stop of the dcorch
   engine-worker service, and make changes in packaging accordingly.
8. Bug fixes for the issues related to the usage of
   base64.urlsafe_b64encode and base64.urlsafe_b64decode in python3.
9. Update unit tests for the main process and worker managers.

Test Plan:
PASS: Verify that the dcorch audit runs properly every 5 minutes.
PASS: Verify that the initial sync runs properly every 10 seconds.
PASS: Verify that the sync subclouds operation runs properly every 5
      seconds.
PASS: Successfully start and stop the dcorch-engine and
      dcorch-engine-worker services using the sm commands.
PASS: Change the admin password on the system controller using
      the command "openstack --os-region-name SystemController user
      password set". Verify that the admin password is synchronized
      to the subcloud and the dcorch receives the corresponding sync
      request, followed by successful execution of sync resources for
      the subcloud.
PASS: Unmanage and then manage a subcloud, and verify that the initial
      sync is executed successfully for that subcloud.
PASS: Verify the removal of the sync_lock table from the dcorch db.

Story: 201106
Task: 50013

Change-Id: I329847bd1107ec43e67ec59bdd1e3111b7b37cd3
Signed-off-by: lzhu1 <li.zhu@windriver.com>
This commit is contained in:
Li Zhu 2024-04-30 17:44:44 -04:00
parent c98abd9af9
commit a1a9951c70
41 changed files with 2292 additions and 1582 deletions

View File

@ -138,7 +138,7 @@ class UsersController(object):
try:
# user specific event id is in the format of
# <user_id>_<issued_before> and encoded in base64
event_ref = base64.urlsafe_b64decode(str(event_id))
event_ref = base64.urlsafe_b64decode(event_id).decode('utf-8')
event_tags = event_ref.split('_')
user_id = event_tags[0]
issued_before = event_tags[1]
@ -172,7 +172,7 @@ class UsersController(object):
try:
# user specific event id is in the format of
# <user_id>_<issued_before> and encoded in base64
event_ref = base64.urlsafe_b64decode(str(event_id))
event_ref = base64.urlsafe_b64decode(event_id).decode('utf-8')
event_tags = event_ref.split('_')
user_id = event_tags[0]
issued_before = event_tags[1]

View File

@ -214,7 +214,7 @@ class SubcloudManager(manager.Manager):
super(SubcloudManager, self).__init__(service_name="subcloud_manager",
*args, **kwargs)
self.context = dcmanager_context.get_admin_context()
self.dcorch_rpc_client = dcorch_rpc_client.EngineClient()
self.dcorch_rpc_client = dcorch_rpc_client.EngineWorkerClient()
self.fm_api = fm_api.FaultAPIs()
self.audit_rpc_client = dcmanager_audit_rpc_client.ManagerAuditClient()
self.state_rpc_client = dcmanager_rpc_client.SubcloudStateClient()

View File

@ -62,7 +62,7 @@ class SubcloudStateManager(manager.Manager):
super(SubcloudStateManager,
self).__init__(service_name="subcloud_manager", *args, **kwargs)
self.context = context.get_admin_context()
self.dcorch_rpc_client = dcorch_rpc_client.EngineClient()
self.dcorch_rpc_client = dcorch_rpc_client.EngineWorkerClient()
self.fm_api = fm_api.FaultAPIs()
self.audit_rpc_client = dcmanager_audit_rpc_client.ManagerAuditClient()

View File

@ -373,7 +373,7 @@ class BaseTestSubcloudManager(base.DCManagerTestCase):
def _mock_dcorch_api(self):
"""Mock the DCOrch API"""
p = mock.patch('dcorch.rpc.client.EngineClient')
p = mock.patch('dcorch.rpc.client.EngineWorkerClient')
self.mock_dcorch_api = p.start()
self.addCleanup(p.stop)

View File

@ -1,4 +1,5 @@
# Copyright (c) 2017 Ericsson AB.
# Copyright (c) 2018-2019, 2024 Wind River Systems, Inc.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
@ -36,7 +37,7 @@ class SubcloudController(object):
def __init__(self, *args, **kwargs):
super(SubcloudController, self).__init__(*args, **kwargs)
self.rpc_client = rpc_client.EngineClient()
self.rpc_client = rpc_client.EngineWorkerClient()
# to do the version compatibility for future purpose
def _determine_version_cap(self, target):

View File

@ -225,7 +225,8 @@ def retrieve_token_audit_id(fernet_token):
unpacked_token = _unpack_token(fernet_token, fernet_keys)
if unpacked_token:
audit_id = unpacked_token[-1][0]
audit_id = base64.urlsafe_b64encode(audit_id).rstrip('=')
audit_id = base64.urlsafe_b64encode(
audit_id.encode('utf-8')).rstrip(b'=').decode('utf-8')
return audit_id

View File

@ -27,9 +27,7 @@ from oslo_i18n import _lazy # noqa: E402
from oslo_log import log as logging # noqa: E402
from oslo_service import service # noqa: E402
from dcmanager.common import messaging as dmanager_messaging # noqa: E402
from dcorch.common import config # noqa: E402
from dcorch.common import consts # noqa: E402
from dcorch.common import messaging # noqa: E402
from dcorch.engine import service as engine # noqa: E402
# pylint: enable=wrong-import-position
@ -45,15 +43,11 @@ def main():
logging.setup(cfg.CONF, 'dcorch-engine')
logging.set_defaults()
messaging.setup()
dmanager_messaging.setup()
LOG.info("Launching dcorch-engine, host=%s, workers=%s ...",
cfg.CONF.host, cfg.CONF.workers)
LOG.info("Launching dcorch-engine, host=%s ...", cfg.CONF.host)
srv = engine.EngineService(cfg.CONF.host,
consts.TOPIC_ORCH_ENGINE)
launcher = service.launch(cfg.CONF,
srv, workers=cfg.CONF.workers)
srv = engine.EngineService()
launcher = service.launch(cfg.CONF, srv)
# the following periodic tasks are intended serve as HA checking
# srv.create_periodic_tasks()
launcher.wait()

View File

@ -0,0 +1,52 @@
#!/usr/bin/env python
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
"""
DC Orchestrators Engine Server.
"""
import eventlet
eventlet.monkey_patch()
# pylint: disable=wrong-import-position
from oslo_config import cfg # noqa: E402
from oslo_i18n import _lazy # noqa: E402
from oslo_log import log as logging # noqa: E402
from oslo_service import service # noqa: E402
from dcmanager.common import messaging as dmanager_messaging # noqa: E402
from dcorch.common import config # noqa: E402
from dcorch.common import messaging # noqa: E402
from dcorch.engine import service as engine # noqa: E402
# pylint: enable=wrong-import-position
_lazy.enable_lazy()
config.register_options()
LOG = logging.getLogger('dcorch.engine-worker')
def main():
logging.register_options(cfg.CONF)
cfg.CONF(project='dcorch', prog='dcorch-engine-worker')
logging.setup(cfg.CONF, 'dcorch-engine-worker')
logging.set_defaults()
messaging.setup()
dmanager_messaging.setup()
LOG.info("Launching dcorch-engine-worker, host=%s, workers=%s ...",
cfg.CONF.host, cfg.CONF.worker_workers)
srv = engine.EngineWorkerService()
launcher = service.launch(cfg.CONF,
srv, workers=cfg.CONF.worker_workers)
# the following periodic tasks are intended serve as HA checking
# srv.create_periodic_tasks()
launcher.wait()
if __name__ == '__main__':
main()

View File

@ -198,8 +198,8 @@ scheduler_opts = [
]
common_opts = [
cfg.IntOpt('workers', default=5,
help='number of workers'),
cfg.IntOpt('worker_workers', default=5,
help='number of engine-worker workers'),
cfg.StrOpt('host',
default='localhost',
help='hostname of the machine'),

View File

@ -1,5 +1,5 @@
# Copyright (c) 2016 Ericsson AB.
# Copyright (c) 2017-2022 Wind River Systems, Inc.
# Copyright (c) 2017-2022, 2024 Wind River Systems, Inc.
# 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
@ -19,6 +19,7 @@ JOB_PROGRESS = "IN_PROGRESS"
RPC_API_VERSION = "1.0"
TOPIC_ORCH_ENGINE = "dcorch-engine"
TOPIC_ORCH_ENGINE_WORKER = "dcorch-engine-worker"
# SyncRequest States
ORCH_REQUEST_NONE = None
@ -138,3 +139,11 @@ AUDIT_STATUS_NONE = "none"
AUDIT_STATUS_IN_PROGRESS = "in-progress"
AUDIT_STATUS_COMPLETED = "completed"
AUDIT_STATUS_FAILED = "failed"
AUDIT_QUALIFIED_STATUS = [
AUDIT_STATUS_NONE,
AUDIT_STATUS_FAILED
]
AUDIT_CONDITIONAL_STATUS = [
AUDIT_STATUS_COMPLETED,
AUDIT_STATUS_IN_PROGRESS
]

View File

@ -13,7 +13,9 @@
# License for the specific language governing permissions and limitations
# under the License.
#
# Copyright (c) 2020 Wind River Systems, Inc.
# Copyright (c) 2018-2020, 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
"""
Interface for database access.
@ -150,6 +152,43 @@ def subcloud_get_all(context, region_name=None,
initial_sync_state=initial_sync_state)
def subcloud_capabilities_get_all(context, region_name=None,
management_state=None,
availability_status=None,
initial_sync_state=None):
return IMPL.subcloud_capabilities_get_all(
context, region_name=region_name,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=initial_sync_state)
def subcloud_sync_update_all_to_in_progress(context,
management_state,
availability_status,
initial_sync_state,
sync_requests):
return IMPL.subcloud_sync_update_all_to_in_progress(
context,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=initial_sync_state,
sync_requests=sync_requests)
def subcloud_audit_update_all_to_in_progress(context,
management_state,
availability_status,
initial_sync_state,
audit_interval):
return IMPL.subcloud_audit_update_all_to_in_progress(
context,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=initial_sync_state,
audit_interval=audit_interval)
def subcloud_create(context, region_name, values):
return IMPL.subcloud_create(context, region_name, values)
@ -162,6 +201,11 @@ def subcloud_delete(context, region_name):
return IMPL.subcloud_delete(context, region_name)
def subcloud_update_state_all(context, pre_initial_sync_state, initial_sync_state):
return IMPL.subcloud_update_state_all(
context, pre_initial_sync_state, initial_sync_state)
def resource_get_by_type_and_master_id(context, resource_type, master_id):
return IMPL.resource_get_by_type_and_master_id(
context, resource_type, master_id)
@ -310,28 +354,6 @@ def purge_deleted_records(context, age_in_days=1):
return IMPL.purge_deleted_records(context, age_in_days)
def sync_lock_acquire(context, engine_id, subcloud_name, endpoint_type, action):
return IMPL.sync_lock_acquire(context, engine_id, subcloud_name,
endpoint_type, action)
def sync_lock_release(context, subcloud_name, endpoint_type, action):
return IMPL.sync_lock_release(context, subcloud_name, endpoint_type, action)
def sync_lock_steal(context, engine_id, subcloud_name, endpoint_type, action):
return IMPL.sync_lock_steal(context, engine_id, subcloud_name,
endpoint_type, action)
def sync_lock_delete_by_engine_id(context, engine_id):
return IMPL.sync_lock_delete_by_engine_id(context, engine_id)
def purge_stale_sync_lock(context):
return IMPL.purge_stale_sync_lock(context)
def subcloud_sync_get(context, subcloud_name, endpoint_type):
return IMPL.subcloud_sync_get(context, subcloud_name, endpoint_type)
@ -341,6 +363,11 @@ def subcloud_sync_update(context, subcloud_name, endpoint_type, values):
values)
def subcloud_sync_update_all(context, management_state, endpoint_type, values):
return IMPL.subcloud_sync_update_all(context, management_state, endpoint_type,
values)
def subcloud_sync_create(context, subcloud_name, endpoint_type, values):
return IMPL.subcloud_sync_create(context, subcloud_name, endpoint_type,
values)

View File

@ -1,5 +1,5 @@
# Copyright (c) 2015 Ericsson AB.
# Copyright (c) 2017-2021 Wind River Systems, Inc.
# Copyright (c) 2017-2021, 2023-2024 Wind River Systems, Inc.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
@ -24,7 +24,6 @@ import datetime
import sys
import threading
from oslo_db import api as oslo_db_api
from oslo_db import exception as db_exc
from oslo_db.sqlalchemy import enginefacade
@ -33,12 +32,15 @@ from oslo_utils import strutils
from oslo_utils import timeutils
from oslo_utils import uuidutils
from sqlalchemy import and_
from sqlalchemy import asc
from sqlalchemy import desc
from sqlalchemy.exc import IntegrityError
from sqlalchemy import or_
from sqlalchemy.orm.exc import MultipleResultsFound
from sqlalchemy.orm.exc import NoResultFound
from sqlalchemy.orm import joinedload_all
from sqlalchemy import select
from sqlalchemy import update
from dcorch.common import consts
from dcorch.common import exceptions as exception
@ -451,6 +453,87 @@ def subcloud_get_all(context, region_name=None,
return query.all()
@require_context
def subcloud_capabilities_get_all(context, region_name=None,
management_state=None,
availability_status=None,
initial_sync_state=None):
results = subcloud_get_all(context, region_name, management_state,
availability_status, initial_sync_state)
return {result['region_name']: result['capabilities'] for result in results}
@require_context
def subcloud_sync_update_all_to_in_progress(context,
management_state,
availability_status,
initial_sync_state,
sync_requests):
with write_session() as session:
# Fetch the records of subcloud_sync that meet the update criteria
subcloud_sync_rows = session.query(models.SubcloudSync).join(
models.Subcloud,
models.Subcloud.region_name == models.SubcloudSync.subcloud_name
).filter(
models.Subcloud.management_state == management_state,
models.Subcloud.availability_status == availability_status,
models.Subcloud.initial_sync_state == initial_sync_state,
models.SubcloudSync.sync_request.in_(sync_requests)
).all()
# Update the sync status to in-progress for the selected subcloud_sync
# records
updated_rows = []
for subcloud_sync in subcloud_sync_rows:
subcloud_sync.sync_request = consts.SYNC_STATUS_IN_PROGRESS
updated_rows.append((subcloud_sync.subcloud_name,
subcloud_sync.endpoint_type))
return updated_rows
@require_context
def subcloud_audit_update_all_to_in_progress(context,
management_state,
availability_status,
initial_sync_state,
audit_interval):
threshold_time = timeutils.utcnow() - datetime.timedelta(seconds=audit_interval)
with write_session() as session:
# Fetch the records of subcloud_sync that meet the update criteria
subcloud_sync_rows = session.query(models.SubcloudSync).join(
models.Subcloud,
models.Subcloud.region_name == models.SubcloudSync.subcloud_name
).filter(
models.Subcloud.management_state == management_state,
models.Subcloud.availability_status == availability_status,
models.Subcloud.initial_sync_state == initial_sync_state,
or_(
# Search those with conditional audit status
# (completed/in-progress) and the last audit time is equal
# or greater than the audit interval
and_(
models.SubcloudSync.audit_status.in_(
consts.AUDIT_CONDITIONAL_STATUS),
models.SubcloudSync.last_audit_time <= threshold_time
),
models.SubcloudSync.audit_status.in_(consts.AUDIT_QUALIFIED_STATUS)
)
).all()
# Update the audit status to in-progress for the selected subcloud_sync
# records
updated_rows = []
for subcloud_sync in subcloud_sync_rows:
subcloud_sync.audit_status = consts.AUDIT_STATUS_IN_PROGRESS
subcloud_sync.last_audit_time = timeutils.utcnow()
updated_rows.append((subcloud_sync.subcloud_name,
subcloud_sync.endpoint_type))
return updated_rows
@require_admin_context
def subcloud_create(context, region_name, values):
with write_session() as session:
@ -489,6 +572,34 @@ def subcloud_delete(context, region_name):
raise exception.SubcloudNotFound(region_name=region_name)
@require_admin_context
def subcloud_update_state_all(context, pre_initial_sync_state, initial_sync_state):
updated_count = 0
with write_session() as session:
while True:
# Fetch a batch of records to update
result = session.query(models.Subcloud). \
filter_by(deleted=0). \
filter_by(initial_sync_state=pre_initial_sync_state). \
limit(1000)
# Fetch the records from the result
records = result.all()
# Check if there are no more records to update
if len(records) == 0:
break
# Update the initial_sync_state for the batch
for record in records:
record.initial_sync_state = initial_sync_state
# Increment the count of updated records
updated_count += len(records)
return updated_count
@require_context
def _resource_get(context, resource_type, master_id, session):
query = model_query(context, models.Resource, session=session). \
@ -962,83 +1073,6 @@ def purge_deleted_records(context, age_in_days):
LOG.info('%d records were purged from resource table.', count)
def sync_lock_acquire(
context, engine_id, subcloud_name, endpoint_type, action):
LOG.debug("sync_lock_acquire: %s/%s/%s/%s" % (engine_id, subcloud_name,
endpoint_type, action))
with write_session() as session:
lock = session.query(models.SyncLock). \
filter_by(deleted=0). \
filter_by(subcloud_name=subcloud_name). \
filter_by(endpoint_type=endpoint_type). \
filter_by(action=action).all()
if not lock:
lock_ref = models.SyncLock()
lock_ref.engine_id = engine_id
lock_ref.subcloud_name = subcloud_name
lock_ref.endpoint_type = endpoint_type
lock_ref.action = action
try:
session.add(lock_ref)
return True
except IntegrityError:
LOG.info("IntegrityError Engine id:%s, subcloud:%s, "
"endpoint_type:%s" %
(engine_id, subcloud_name, endpoint_type))
except db_exc.DBDuplicateEntry:
LOG.info("DBDuplicateEntry Engine id:%s, subcloud:%s, "
"endpoint_type:%s" %
(engine_id, subcloud_name, endpoint_type))
except Exception:
LOG.exception("Got session add exception")
return False
# For robustness, this will attempt max_retries with inc_retry_interval
# backoff to release the sync_lock.
@oslo_db_api.wrap_db_retry(max_retries=3, retry_on_deadlock=True,
retry_interval=0.5, inc_retry_interval=True)
def sync_lock_release(context, subcloud_name, endpoint_type, action):
with write_session() as session:
session.query(models.SyncLock).filter_by(
subcloud_name=subcloud_name). \
filter_by(endpoint_type=endpoint_type). \
filter_by(action=action). \
delete(synchronize_session='fetch')
def sync_lock_steal(context, engine_id, subcloud_name, endpoint_type, action):
sync_lock_release(context, subcloud_name, endpoint_type, action)
return sync_lock_acquire(context, engine_id, subcloud_name, endpoint_type,
action)
def sync_lock_delete_by_engine_id(context, engine_id):
"""Delete all sync_lock entries for a given engine."""
with write_session() as session:
results = session.query(models.SyncLock). \
filter_by(engine_id=engine_id).all()
for result in results:
LOG.info("Deleted sync lock id=%s engine_id=%s" %
(result.id, result.engine_id))
session.delete(result)
def purge_stale_sync_lock(context):
"""Delete all sync lock entries where service ID no longer exists."""
LOG.info('Purging stale sync_locks')
with write_session() as session:
# Purging sync_lock table
subquery = model_query(context, models.Service.id). \
group_by(models.Service.id)
count = session.query(models.SyncLock). \
filter(~models.SyncLock.engine_id.in_(subquery)). \
delete(synchronize_session='fetch')
LOG.info('%d records were purged from sync_lock table.', count)
def _subcloud_sync_get(context, subcloud_name, endpoint_type, session=None):
query = model_query(context, models.SubcloudSync, session=session). \
filter_by(subcloud_name=subcloud_name). \
@ -1082,6 +1116,25 @@ def subcloud_sync_update(context, subcloud_name, endpoint_type, values):
return result
def subcloud_sync_update_all(context, management_state, endpoint_type, values):
with write_session() as session:
subquery = select([models.SubcloudSync.id]). \
where(models.SubcloudSync.subcloud_name ==
models.Subcloud.region_name). \
where(models.Subcloud.management_state == management_state). \
where(models.SubcloudSync.endpoint_type == endpoint_type). \
where(models.SubcloudSync.deleted == 0). \
correlate(models.SubcloudSync)
stmt = update(models.SubcloudSync). \
where(models.SubcloudSync.id.in_(subquery)). \
values(values)
result = session.execute(stmt)
return result.rowcount
def subcloud_sync_delete(context, subcloud_name, endpoint_type):
with write_session() as session:
results = session.query(models.SubcloudSync). \

View File

@ -0,0 +1,19 @@
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import sqlalchemy
def upgrade(migrate_engine):
meta = sqlalchemy.MetaData()
meta.bind = migrate_engine
sync_lock = sqlalchemy.Table('sync_lock', meta, autoload=True)
sync_lock.drop()
def downgrade(migrate_engine):
raise NotImplementedError('Database downgrade not supported.')

View File

@ -1,5 +1,5 @@
# Copyright (c) 2015 Ericsson AB
# Copyright (c) 2017-2022 Wind River Systems, Inc.
# Copyright (c) 2017-2022, 2024 Wind River Systems, Inc.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
@ -295,22 +295,6 @@ class OrchRequest(BASE, OrchestratorBase):
ForeignKey('orch_job.id'), primary_key=True)
class SyncLock(BASE, OrchestratorBase):
"""Store locks to avoid overlapping of audit
syncing during automatic periodic sync jobs with
multiple-engines.
"""
__tablename__ = 'sync_lock'
id = Column(Integer, primary_key=True)
engine_id = Column(String(36), nullable=False)
subcloud_name = Column(String(255), nullable=False)
endpoint_type = Column(String(255), default="none")
action = Column(String(64), default="none")
class SubcloudSync(BASE, OrchestratorBase):
"""Store subcloud sync information to allow coordination of dcorch workload

View File

@ -1,94 +0,0 @@
# Copyright 2016 Ericsson AB
# 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.
from oslo_config import cfg
from oslo_log import log as logging
from dcorch.common.i18n import _
from dcorch.engine import scheduler
LOG = logging.getLogger(__name__)
lock_opts = [
cfg.IntOpt('lock_retry_times',
default=3,
help=_('Number of times trying to grab a lock.')),
cfg.IntOpt('lock_retry_interval',
default=10,
help=_('Number of seconds between lock retries.'))
]
lock_opts_group = cfg.OptGroup('locks')
cfg.CONF.register_group(lock_opts_group)
cfg.CONF.register_opts(lock_opts, group=lock_opts_group)
def sync_lock_acquire(engine_id, task_type, lock):
"""Try to lock with specified engine_id.
:param engine: ID of the engine which wants to lock the projects.
:param lock: the lock object owned by the caller
:returns: True if lock is acquired, or False otherwise.
"""
# Step 1: try lock the projects- if it returns True then success
LOG.info('Trying to acquire lock with %(engId)s for Task: %(task)s',
{'engId': engine_id,
'task': task_type
}
)
lock_status = lock.acquire(False)
if lock_status:
return True
# Step 2: retry using global configuration options
retries = cfg.CONF.locks.lock_retry_times
retry_interval = cfg.CONF.locks.lock_retry_interval
while retries > 0:
scheduler.sleep(retry_interval)
LOG.info('Retry acquire lock with %(engId)s for Task: %(task)s',
{'engId': engine_id,
'task': task_type
}
)
lock_status = lock.acquire(False)
if lock_status:
return True
retries = retries - 1
# Will reach here only when not able to acquire locks with retry
LOG.error('Not able to acquire lock for %(task)s with retry'
' with engineId %(engId)s',
{'engId': engine_id,
'task': task_type
}
)
return False
def sync_lock_release(engine_id, task_type, lock):
"""Release the lock for the projects"""
LOG.info('Releasing acquired lock with %(engId)s for Task: %(task)s',
{'engId': engine_id,
'task': task_type
}
)
return lock.release()
def list_opts():
yield lock_opts_group.name, lock_opts

View File

@ -124,15 +124,16 @@ class FernetKeyManager(manager.Manager):
self._schedule_work(consts.OPERATION_TYPE_PUT)
def distribute_keys(self, ctxt, subcloud_name):
keys = self._get_master_keys()
@staticmethod
def distribute_keys(subcloud_name):
keys = FernetKeyManager._get_master_keys()
if not keys:
LOG.info(_("No fernet keys returned from %s") %
dccommon_consts.CLOUD_0)
return
resource_info = FernetKeyManager.to_resource_info(keys)
key_list = FernetKeyManager.from_resource_info(resource_info)
self.update_fernet_repo(subcloud_name, key_list)
FernetKeyManager.update_fernet_repo(subcloud_name, key_list)
def reset_keys(self, subcloud_name):
self.update_fernet_repo(subcloud_name)

View File

@ -15,104 +15,51 @@
# See the License for the specific language governing permissions and
# limitations under the License.
import collections
import random
import eventlet
from keystoneauth1 import exceptions as keystone_exceptions
from oslo_config import cfg
from oslo_log import log as logging
from oslo_utils import timeutils
from dccommon import consts as dccommon_consts
from dcorch.common import consts as dco_consts
from dcorch.common import context
from dcorch.common import exceptions
from dcorch.db import api as db_api
from dcorch.drivers.openstack import sdk
from dcorch.engine import scheduler
from dcorch.engine import subcloud_lock
from dcorch.engine.sync_services.identity import IdentitySyncThread
from dcorch.engine.sync_services.sysinv import SysinvSyncThread
from dcorch.objects import subcloud
from dcorch.rpc import client
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
CHECK_AUDIT_INTERVAL = 300 # frequency to check for audit work
SYNC_TIMEOUT = 600 # Timeout for subcloud sync
CHECK_SYNC_INTERVAL = 5 # frequency to check for sync work
AUDIT_INTERVAL = 1200 # Default audit interval
# sync object endpoint type and subclass mappings
sync_object_class_map = {
dccommon_consts.ENDPOINT_TYPE_PLATFORM: SysinvSyncThread,
dccommon_consts.ENDPOINT_TYPE_IDENTITY: IdentitySyncThread,
dccommon_consts.ENDPOINT_TYPE_IDENTITY_OS: IdentitySyncThread
}
class GenericSyncManager(object):
"""Manages tasks related to resource management."""
def __init__(self, engine_id, *args, **kwargs):
def __init__(self, *args, **kwargs):
super(GenericSyncManager, self).__init__()
self.context = context.get_admin_context()
self.engine_id = engine_id
# Keeps track of greenthreads we create to do the sync work.
self.thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=100)
# Keeps track of greenthreads we create to do the audit work.
self.audit_thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=100)
# this needs to map a name to a dictionary
# stores the sync object per region per endpoint type
self.sync_objs = collections.defaultdict(dict)
# Track greenthreads created for each subcloud.
self.subcloud_threads = list()
self.subcloud_audit_threads = list()
self.engine_worker_rpc_client = client.EngineWorkerClient()
def init_from_db(self, context):
subclouds = subcloud.SubcloudList.get_all(context)
for sc in subclouds:
self.create_sync_objects(sc.region_name, sc.capabilities)
LOG.info('Engine id:(%s) create_sync_objects for'
'subcloud:%s.' % (self.engine_id, sc.region_name))
eventlet.sleep(0) # cooperative yield
def create_sync_objects(self, subcloud_name, capabilities):
"""Create sync object objects for the subcloud
The objects handle the syncing of the subcloud's endpoint_types
"""
endpoint_type_list = capabilities.get('endpoint_types', None)
if endpoint_type_list:
self.sync_objs[subcloud_name] = {}
for endpoint_type in endpoint_type_list:
LOG.info("Engine id:(%s) create %s/%s sync obj" %
(self.engine_id, subcloud_name, endpoint_type))
sync_obj = sync_object_class_map[endpoint_type](subcloud_name,
endpoint_type)
self.sync_objs[subcloud_name].update({
endpoint_type: sync_obj})
def sync_job_thread(self, engine_id):
def sync_job_thread(self):
"""Perform sync request for subclouds as required."""
while True:
try:
self.sync_subclouds(engine_id)
eventlet.greenthread.sleep(5)
self.sync_subclouds()
eventlet.greenthread.sleep(CHECK_SYNC_INTERVAL)
except eventlet.greenlet.GreenletExit:
# We have been told to exit
return
except Exception as e:
LOG.exception(e)
def sync_audit_thread(self, engine_id):
def sync_audit_thread(self):
"""Perform sync request for subclouds as required."""
while True:
try:
self.run_sync_audit(engine_id)
self.run_sync_audit()
eventlet.greenthread.sleep(CHECK_AUDIT_INTERVAL)
except eventlet.greenlet.GreenletExit:
# We have been told to exit
@ -120,489 +67,86 @@ class GenericSyncManager(object):
except Exception as e:
LOG.exception(e)
def sync_subclouds(self, engine_id):
# get a list of subclouds that is online, managed and initial_sync is
# completed, than check if subcloud_name in self.sync_objs
# When the subcloud is managed, it will be returned in the list in the
# next cycle. When the subcloud is unmanaged, it will not be included
# in the list in the next cycle
# get the subcloud/endpoint list has sync_request set to requested
#
subclouds = db_api.subcloud_get_all(
def _process_subclouds(self, rpc_method, subcloud_sync_list):
# We want a chunksize of at least 1 so add the number of workers.
chunksize = \
(len(subcloud_sync_list) + CONF.worker_workers) // (CONF.worker_workers)
subcloud_sync_chunk = []
for subcloud_sync in subcloud_sync_list:
subcloud_sync_chunk.append(subcloud_sync)
if len(subcloud_sync_chunk) == chunksize:
# We've gathered a batch of subclouds, send it to engine worker
# to process.
try:
rpc_method(self.context, subcloud_sync_chunk)
LOG.debug(
"Sent %s request message for %d subclouds."
% (rpc_method.__name__, len(subcloud_sync_chunk))
)
except Exception as e:
LOG.error("Exception occurred in %s for subclouds %s: %s" %
(rpc_method.__name__,
subcloud_sync_chunk,
e))
subcloud_sync_chunk = []
if len(subcloud_sync_chunk) > 0:
# We've got a partial batch...send it off for processing.
try:
rpc_method(self.context, subcloud_sync_chunk)
LOG.debug(f"Sent {rpc_method.__name__} request message for "
f"{len(subcloud_sync_chunk)} (subcloud, endpoint_type) "
f"pairs.")
except Exception as e:
LOG.error(f"Exception occurred in {rpc_method.__name__} for "
f"subclouds {subcloud_sync_chunk}: {e}")
else:
LOG.debug(f"Done sending {rpc_method.__name__} request messages.")
def sync_subclouds(self):
LOG.info("Start sync_subclouds")
# get a list of eligible subclouds (region_name, endpoint_type),
# and mark them as in-progress.
subcloud_sync_list = db_api.subcloud_sync_update_all_to_in_progress(
self.context,
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=dco_consts.INITIAL_SYNC_STATE_COMPLETED)
# randomize to reduce likelihood of sync_lock contention
random.shuffle(subclouds)
sc_names = []
for sc in subclouds:
if sc.region_name in self.sync_objs:
sc_names.append(sc.region_name)
for ept in self.sync_objs[sc.region_name].keys():
try:
self.sync_subcloud(self.context, engine_id, sc.region_name,
ept, 'sync')
except exceptions.SubcloudSyncNotFound:
# The endpoint in subcloud_sync has been removed
LOG.info("Engine id:(%s/%s) SubcloudSyncNotFound "
"remove from sync_obj endpoint_type %s" %
(engine_id, sc.region_name, ept))
self.sync_objs[sc.region_name].pop(ept, None)
initial_sync_state=dco_consts.INITIAL_SYNC_STATE_COMPLETED,
sync_requests=[dco_consts.SYNC_STATUS_REQUESTED,
dco_consts.SYNC_STATUS_FAILED])
LOG.debug('Engine id:(%s) Waiting for sync_subclouds %s to complete.'
% (engine_id, sc_names))
for thread in self.subcloud_threads:
thread.wait()
# Clear the list of threads before next interval
self.subcloud_threads = list()
LOG.debug('Engine id:(%s): All subcloud syncs have completed.'
% engine_id)
@subcloud_lock.sync_subcloud
def mutex_start_thread(self, context, engine_id, subcloud_name,
endpoint_type, action):
# Double check whether still need while locked this time
subcloud_sync = db_api.subcloud_sync_get(context, subcloud_name,
endpoint_type)
if subcloud_sync.sync_request in [dco_consts.SYNC_STATUS_REQUESTED,
dco_consts.SYNC_STATUS_FAILED]:
thread = self.thread_group_manager.start(
self._sync_subcloud, context, engine_id, subcloud_name,
endpoint_type)
self.subcloud_threads.append(thread)
if subcloud_sync_list:
self._process_subclouds(
self.engine_worker_rpc_client.sync_subclouds, subcloud_sync_list)
else:
LOG.debug("mutex_start_thread Engine id: %s/%s sync not required" %
(engine_id, subcloud_name))
LOG.info("No eligible subclouds for sync.")
def sync_subcloud(self, context, engine_id, subcloud_name, endpoint_type,
action):
# precheck if the sync_state is still started
subcloud_sync = db_api.subcloud_sync_get(context, subcloud_name,
endpoint_type)
def run_sync_audit(self):
LOG.info("Start run_sync_audit")
if subcloud_sync.sync_request in [dco_consts.SYNC_STATUS_REQUESTED,
dco_consts.SYNC_STATUS_FAILED]:
self.mutex_start_thread(
context, engine_id, subcloud_name, endpoint_type, action)
# get a list of eligible subclouds (region_name, endpoint_type),
# and mark them as in-progress.
# check if the last audit time is equal or greater than the audit
# interval only if the status is completed or in progress (in case
# the process is dead while audit is in progress), or go ahead with
# audit if the status is failed or none.
subcloud_sync_list = db_api.subcloud_audit_update_all_to_in_progress(
self.context,
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=dco_consts.INITIAL_SYNC_STATE_COMPLETED,
audit_interval=AUDIT_INTERVAL)
if subcloud_sync_list:
self._process_subclouds(
self.engine_worker_rpc_client.run_sync_audit, subcloud_sync_list)
else:
LOG.debug("Engine id: %s/%s sync not required" %
(engine_id, subcloud_name))
def _sync_subcloud(self, context, engine_id, subcloud_name, endpoint_type):
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'sync_request': dco_consts.SYNC_STATUS_IN_PROGRESS})
obj = self.sync_objs[subcloud_name][endpoint_type]
new_state = dco_consts.SYNC_STATUS_COMPLETED
timeout = eventlet.timeout.Timeout(SYNC_TIMEOUT)
try:
obj.sync(engine_id)
except eventlet.timeout.Timeout as t:
if t is not timeout:
raise # not my timeout
new_state = dco_consts.SYNC_STATUS_FAILED
except Exception as e:
LOG.exception('Sync failed for %s/%s: %s',
subcloud_name, endpoint_type, e)
new_state = dco_consts.SYNC_STATUS_FAILED
finally:
timeout.cancel()
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'sync_request': new_state})
def add_subcloud(self, context, name, version):
# create subcloud in DB and create the sync objects
LOG.info('adding subcloud %(sc)s' % {'sc': name})
capabilities = {}
endpoint_type_list = dco_consts.SYNC_ENDPOINT_TYPES_LIST[:]
capabilities.update({'endpoint_types': endpoint_type_list})
sc = subcloud.Subcloud(
context, region_name=name, software_version=version,
capabilities=capabilities)
sc = sc.create()
for endpoint_type in endpoint_type_list:
db_api.subcloud_sync_create(context, name, endpoint_type,
# pylint: disable-next=no-member
values={'subcloud_id': sc.id})
# Create the sync object for this engine
self.create_sync_objects(name, capabilities)
def del_subcloud(self, context, subcloud_name):
# first update the state of the subcloud
self.update_subcloud_state(
subcloud_name,
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE)
# shutdown, optionally deleting queued work
if subcloud_name not in self.sync_objs:
LOG.error("Subcloud %s sync_objs do not exist" % subcloud_name)
else:
del self.sync_objs[subcloud_name]
try:
# delete this subcloud
subcloud.Subcloud.delete_subcloud_by_name(context, subcloud_name)
except Exception:
raise exceptions.SubcloudNotFound(region_name=subcloud_name)
LOG.info("No eligible subclouds for audit.")
def sync_request(self, ctxt, endpoint_type):
# Someone has enqueued a sync job. set the endpoint sync_request to
# requested
subclouds = db_api.subcloud_get_all(
ctxt, management_state=dccommon_consts.MANAGEMENT_MANAGED)
for sc in subclouds:
GenericSyncManager.set_sync_request(ctxt, sc.region_name,
endpoint_type)
@classmethod
def set_sync_request(cls, ctxt, subcloud_name, endpoint_type):
db_api.subcloud_sync_update(
ctxt, subcloud_name, endpoint_type,
db_api.subcloud_sync_update_all(
ctxt, dccommon_consts.MANAGEMENT_MANAGED, endpoint_type,
values={'sync_request': dco_consts.SYNC_STATUS_REQUESTED})
def subcloud_state_matches(self, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
# compare subcloud states
match = True
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
if management_state is not None:
if sc.management_state != management_state:
match = False
if match and availability_status is not None:
if sc.availability_status != availability_status:
match = False
if match and initial_sync_state is not None:
if sc.initial_sync_state != initial_sync_state:
match = False
return match
def update_subcloud_state(self, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
LOG.info('updating state for subcloud %(sc)s - '
'management_state: %(mgmt)s '
'availability_status: %(avail)s '
'initial_sync_state: %(iss)s ' %
{'sc': subcloud_name, 'mgmt': management_state,
'avail': availability_status, 'iss': initial_sync_state})
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
if management_state is not None:
sc.management_state = management_state
if availability_status is not None:
sc.availability_status = availability_status
if initial_sync_state is not None:
sc.initial_sync_state = initial_sync_state
sc.save()
def init_subcloud_sync_audit(self, subcloud_name):
LOG.info('Initialize subcloud sync audit for '
'subcloud %(sc)s' %
{'sc': subcloud_name})
endpoint_type_list = dco_consts.SYNC_ENDPOINT_TYPES_LIST[:]
for endpoint_type in endpoint_type_list:
db_api.subcloud_sync_update(
self.context, subcloud_name, endpoint_type,
values={'audit_status': dco_consts.AUDIT_STATUS_NONE,
'sync_status_reported': dco_consts.SYNC_STATUS_NONE,
'sync_status_report_time': None,
'last_audit_time': None})
def enable_subcloud(self, context, subcloud_name):
LOG.info('enabling subcloud %(sc)s' % {'sc': subcloud_name})
if subcloud_name in self.sync_objs:
for sync_obj in self.sync_objs[subcloud_name].values():
LOG.info('Engine id: %(id)s enabling sync '
'thread subcloud %(sc)s' %
{'sc': subcloud_name, 'id': self.engine_id})
sync_obj.enable()
else:
LOG.error("enable_subcloud No sync objects for subcloud:%s" %
subcloud_name)
def disable_subcloud(self, context, subcloud_name):
LOG.info('disabling subcloud %(sc)s' % {'sc': subcloud_name})
# nothing to do here at the moment
pass
def is_subcloud_managed(self, subcloud_name):
# is this subcloud managed
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
return sc.management_state == dccommon_consts.MANAGEMENT_MANAGED
def is_subcloud_enabled(self, subcloud_name):
# is this subcloud enabled
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
# We only enable syncing if the subcloud is online and the initial
# sync has completed.
if (sc.availability_status == dccommon_consts.AVAILABILITY_ONLINE and
sc.initial_sync_state == dco_consts.INITIAL_SYNC_STATE_COMPLETED):
return True
else:
return False
def is_subcloud_ready(self, subcloud_name):
# is this subcloud ready for synchronization
return self.is_subcloud_managed(subcloud_name) and \
self.is_subcloud_enabled(subcloud_name)
def add_subcloud_sync_endpoint_type(self, context, subcloud_name,
endpoint_type_list=None):
# TODO(jkung): This method is currently only required by
# stx-openstack and is to be integrated with stx-openstack when
# that feature is enabled.
LOG.info("add_subcloud_sync_endpoint_type subcloud_name=%s "
"endpoint_type_list=%s" %
(subcloud_name, endpoint_type_list))
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
capabilities = sc.capabilities
c_endpoint_type_list = capabilities.get('endpoint_types', [])
# Update the DB first
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
if endpoint_type not in c_endpoint_type_list:
c_endpoint_type_list.append(endpoint_type)
if capabilities.get('endpoint_types') is None:
# assign back if 'endpoint_types' is not in capabilities
capabilities['endpoint_types'] = c_endpoint_type_list
sc.capabilities = capabilities
sc.save()
# Create objects for the endpoint types
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
# Check whether sync endpoint already exists
try:
subcloud_sync = db_api.subcloud_sync_get(
context, subcloud_name,
endpoint_type)
if subcloud_sync:
LOG.info("subcloud_sync subcloud=%s "
"endpoint_type=%s already exists" %
(subcloud_name, endpoint_type))
continue
except exceptions.SubcloudSyncNotFound:
pass
# skip creation if a sync_obj of this endpoint type already
# exists
sync_obj = self.sync_objs[subcloud_name].get(
endpoint_type == endpoint_type)
if not sync_obj:
LOG.info("add_subcloud_sync_endpoint_type "
"subcloud_name=%s, sync_obj add=%s" %
(subcloud_name, endpoint_type))
sync_obj = sync_object_class_map[endpoint_type](
subcloud_name, endpoint_type=endpoint_type)
self.sync_objs[subcloud_name].update(
{endpoint_type: sync_obj})
# create the subcloud_sync !!!
db_api.subcloud_sync_create(
context, subcloud_name, endpoint_type,
values={'subcloud_id': sc.id}) # pylint: disable=E1101
if self.is_subcloud_ready(subcloud_name):
sync_obj.enable()
sync_obj.initial_sync()
def remove_subcloud_sync_endpoint_type(self, context, subcloud_name,
endpoint_type_list=None):
# TODO(jkung): This method is currently only required by
# stx-openstack and is to be integrated with stx-openstack when
# that feature is enabled and remove action performed.
# The subcloud_sync delete can be more graceful by ensuring the
# sync object is updated for each engine on delete.
LOG.info("remove_subcloud_sync_endpoint_type subcloud_name=%s "
"endpoint_type_list=%s" %
(subcloud_name, endpoint_type_list))
# Remove sync_objs and subcloud_sync for endpoint types to be removed
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
self.sync_objs[subcloud_name].pop(endpoint_type, None)
try:
db_api.subcloud_sync_delete(
context, subcloud_name, endpoint_type)
except exceptions.SubcloudSyncNotFound:
pass
# remove the endpoint types from subcloud capabilities
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
capabilities = sc.capabilities
c_endpoint_type_list = capabilities.get('endpoint_types', [])
if endpoint_type_list and c_endpoint_type_list:
for endpoint_type in endpoint_type_list:
if endpoint_type in c_endpoint_type_list:
c_endpoint_type_list.remove(endpoint_type)
sc.capabilities = capabilities
sc.save()
def update_subcloud_version(self, context, subcloud_name, sw_version):
try:
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
sc.software_version = sw_version
sc.save()
except KeyError:
raise exceptions.SubcloudNotFound(region_name=subcloud_name)
def update_subcloud_endpoints(self, context, subcloud_name, endpoints):
try:
LOG.info("Updating service endpoints for subcloud %s in "
"endpoint cache" % subcloud_name)
endpoint_cache = sdk.OpenStackDriver(
region_name=dccommon_consts.CLOUD_0).keystone_client.endpoint_cache
endpoint_cache.update_master_service_endpoint_region(
subcloud_name, endpoints)
except (keystone_exceptions.EndpointNotFound,
keystone_exceptions.ConnectFailure,
IndexError):
LOG.error("Failed to update services endpoints for "
"subcloud: %s in dcorch." % subcloud_name)
def initial_sync(self, context, subcloud_name):
LOG.info('Initial sync subcloud %(sc)s %(id)s' %
{'sc': subcloud_name, 'id': self.engine_id})
# initial synchronization of the subcloud
if subcloud_name in self.sync_objs:
# self.sync_objs stores the sync object per endpoint
for sync_obj in self.sync_objs[subcloud_name].values():
sync_obj.initial_sync()
else:
LOG.info('Initial sync subcloud %(sc)s '
'sync_objs not found...creating' %
{'sc': subcloud_name})
capabilities = {}
endpoint_type_list = dco_consts.SYNC_ENDPOINT_TYPES_LIST[:]
capabilities.update({'endpoint_types': endpoint_type_list})
self.create_sync_objects(subcloud_name, capabilities)
if subcloud_name in self.sync_objs:
# self.sync_objs stores the sync object per endpoint
for sync_obj in self.sync_objs[subcloud_name].values():
sync_obj.initial_sync()
else:
LOG.error('Initial sync subcloud %(sc)s '
'sync_objs not found' %
{'sc': subcloud_name})
@subcloud_lock.sync_subcloud
def audit_subcloud(self, context, engine_id, subcloud_name, endpoint_type,
action):
subcloud_sync = db_api.subcloud_sync_get(context, subcloud_name,
endpoint_type)
# check if the last audit time is equal or greater than the audit
# interval ( only if the status is completed
# if status is failed, go ahead with audit
# restart audit if process death while audit is in progress
audit = False
if subcloud_sync.audit_status in [dco_consts.AUDIT_STATUS_COMPLETED,
dco_consts.AUDIT_STATUS_IN_PROGRESS]:
if subcloud_sync.last_audit_time:
delta = timeutils.delta_seconds(
subcloud_sync.last_audit_time, timeutils.utcnow())
# Audit interval
if delta >= AUDIT_INTERVAL:
audit = True
else:
audit = True
elif subcloud_sync.audit_status in [dco_consts.AUDIT_STATUS_NONE,
dco_consts.AUDIT_STATUS_FAILED]:
audit = True
if audit:
thread = self.thread_group_manager.start(
self._audit_subcloud, engine_id, subcloud_name, endpoint_type)
self.subcloud_audit_threads.append(thread)
def _audit_subcloud(self, engine_id, subcloud_name, endpoint_type):
# The last_audit_time is set up front in order to ensure synchronous
# audit_subcloud() check for in progress and last_audit_time
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'audit_status': dco_consts.AUDIT_STATUS_IN_PROGRESS,
'last_audit_time': timeutils.utcnow()})
obj = self.sync_objs[subcloud_name][endpoint_type]
new_state = dco_consts.AUDIT_STATUS_COMPLETED
timeout = eventlet.timeout.Timeout(SYNC_TIMEOUT)
try:
obj.run_sync_audit(engine_id)
except eventlet.timeout.Timeout as t:
if t is not timeout:
raise # not my timeout
new_state = dco_consts.AUDIT_STATUS_FAILED
except Exception as e:
LOG.exception('Audit failed for %s/%s: %s',
subcloud_name, endpoint_type, e)
new_state = dco_consts.AUDIT_STATUS_FAILED
finally:
timeout.cancel()
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'audit_status': new_state})
def run_sync_audit(self, engine_id):
LOG.info('run_sync_audit %(id)s' % {'id': engine_id})
# get a list of subclouds that are enabled
subclouds = db_api.subcloud_get_all(
self.context,
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=dco_consts.INITIAL_SYNC_STATE_COMPLETED)
# randomize to reduce likelihood of sync_lock contention
random.shuffle(subclouds)
for sc in subclouds:
if sc.region_name in list(self.sync_objs.keys()):
for e in self.sync_objs[sc.region_name].keys():
LOG.debug("Attempt audit_subcloud: %s/%s/%s",
engine_id, sc.region_name, e)
self.audit_subcloud(self.context, engine_id,
sc.region_name, e, 'audit')
else:
# In this case, distribution of sync objects are
# to each worker. If needed in future implementation,
# it is possible to distribute sync_objs to certain workers.
LOG.info('Run sync audit sync subcloud %(sc)s '
'sync_objs not found...creating' %
{'sc': sc.region_name})
capabilities = {}
endpoint_type_list = dco_consts.SYNC_ENDPOINT_TYPES_LIST[:]
capabilities.update({'endpoint_types': endpoint_type_list})
self.create_sync_objects(sc.region_name, capabilities)
# self.sync_objs stores the sync object per endpoint
if sc.region_name in list(self.sync_objs.keys()):
for e in self.sync_objs[sc.region_name].keys():
LOG.debug("Attempt audit_subcloud: %s/%s/%s",
engine_id, sc.region_name, e)
self.audit_subcloud(self.context, engine_id,
sc.region_name, e, 'audit')
else:
LOG.error('Run sync audit subcloud %(sc)s '
'sync_objs not found' %
{'sc': sc.region_name})
LOG.debug('Engine id:(%s) Waiting for audit_subclouds to complete.'
% engine_id)
for thread in self.subcloud_audit_threads:
thread.wait()
# Clear the list of threads before next interval
self.subcloud_audit_threads = list()
LOG.info('Engine id:(%s): All subcloud audit have completed.'
% engine_id)

View File

@ -0,0 +1,354 @@
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import eventlet
from keystoneauth1 import exceptions as keystone_exceptions
from oslo_log import log as logging
from dccommon import consts as dccommon_consts
from dcorch.common import consts as dco_consts
from dcorch.common import context
from dcorch.common import exceptions
from dcorch.db import api as db_api
from dcorch.drivers.openstack import sdk
from dcorch.engine import scheduler
from dcorch.engine.sync_services.identity import IdentitySyncThread
from dcorch.engine.sync_services.sysinv import SysinvSyncThread
from dcorch.engine.sync_thread import SyncThread
from dcorch.objects import subcloud
LOG = logging.getLogger(__name__)
SYNC_TIMEOUT = 600 # Timeout for subcloud sync
# sync object endpoint type and subclass mappings
sync_object_class_map = {
dccommon_consts.ENDPOINT_TYPE_PLATFORM: SysinvSyncThread,
dccommon_consts.ENDPOINT_TYPE_IDENTITY: IdentitySyncThread,
dccommon_consts.ENDPOINT_TYPE_IDENTITY_OS: IdentitySyncThread
}
class GenericSyncWorkerManager(object):
"""Manages tasks related to resource management."""
def __init__(self, engine_id, *args, **kwargs):
super(GenericSyncWorkerManager, self).__init__()
self.context = context.get_admin_context()
self.engine_id = engine_id
# Keeps track of greenthreads we create to do the sync work.
self.thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=100)
# Keeps track of greenthreads we create to do the audit work.
self.audit_thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=100)
def create_sync_objects(self, subcloud_name, capabilities):
"""Create sync object objects for the subcloud
The objects handle the syncing of the subcloud's endpoint_types
"""
sync_objs = {}
endpoint_type_list = capabilities.get('endpoint_types', None)
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
LOG.debug(f"Engine id:({self.engine_id}) create "
f"{subcloud_name}/{endpoint_type} sync obj")
sync_obj = sync_object_class_map[endpoint_type](subcloud_name,
endpoint_type)
sync_objs.update({endpoint_type: sync_obj})
return sync_objs
def sync_subclouds(self, context, subcloud_sync_list):
LOG.info('Engine id:(%s) Start to sync %d (subcloud, endpoint_type) pairs.'
% (self.engine_id, len(subcloud_sync_list)))
LOG.debug('Engine id:(%s) Start to sync %s.'
% (self.engine_id, subcloud_sync_list))
for sc_region_name, ept in subcloud_sync_list:
try:
self.thread_group_manager.start(self._sync_subcloud,
self.context,
sc_region_name,
ept)
except exceptions.SubcloudSyncNotFound:
# The endpoint in subcloud_sync has been removed
LOG.debug("Engine id:(%s/%s/%s) SubcloudSyncNotFound: "
"The endpoint in subcloud_sync has been removed" %
(self.engine_id, sc_region_name, ept))
except Exception as e:
LOG.error("Exception occurred when running sync %s for "
"subcloud %s: %s" % (ept, sc_region_name, e))
db_api.subcloud_sync_update(
self.context, sc_region_name, ept,
values={'sync_request': dco_consts.SYNC_STATUS_FAILED})
def _sync_subcloud(self, context, subcloud_name, endpoint_type):
sync_obj = sync_object_class_map[endpoint_type](subcloud_name,
endpoint_type)
new_state = dco_consts.SYNC_STATUS_COMPLETED
timeout = eventlet.timeout.Timeout(SYNC_TIMEOUT)
try:
sync_obj.sync(self.engine_id)
except eventlet.timeout.Timeout as t:
if t is not timeout:
raise # not my timeout
new_state = dco_consts.SYNC_STATUS_FAILED
except Exception as e:
LOG.exception('Sync failed for %s/%s: %s',
subcloud_name, endpoint_type, e)
new_state = dco_consts.SYNC_STATUS_FAILED
finally:
timeout.cancel()
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'sync_request': new_state})
def add_subcloud(self, context, name, version):
# create subcloud in DB and create the sync objects
LOG.info('adding subcloud %(sc)s' % {'sc': name})
capabilities = {}
endpoint_type_list = dco_consts.SYNC_ENDPOINT_TYPES_LIST[:]
capabilities.update({'endpoint_types': endpoint_type_list})
sc = subcloud.Subcloud(
context, region_name=name, software_version=version,
capabilities=capabilities)
sc = sc.create()
for endpoint_type in endpoint_type_list:
db_api.subcloud_sync_create(context, name, endpoint_type,
# pylint: disable-next=no-member
values={'subcloud_id': sc.id})
# Create the sync object for this engine
self.create_sync_objects(name, capabilities)
def del_subcloud(self, context, subcloud_name):
# first update the state of the subcloud
self.update_subcloud_state(
context,
subcloud_name,
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE)
try:
# delete this subcloud
subcloud.Subcloud.delete_subcloud_by_name(context, subcloud_name)
except Exception:
raise exceptions.SubcloudNotFound(region_name=subcloud_name)
def subcloud_state_matches(self, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
# compare subcloud states
match = True
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
if management_state is not None and \
sc.management_state != management_state:
match = False
if match and availability_status is not None and \
sc.availability_status != availability_status:
match = False
if match and initial_sync_state is not None and \
sc.initial_sync_state != initial_sync_state:
match = False
return match
def update_subcloud_state(self, context, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
LOG.info('updating state for subcloud %(sc)s - '
'management_state: %(mgmt)s '
'availability_status: %(avail)s '
'initial_sync_state: %(iss)s ' %
{'sc': subcloud_name, 'mgmt': management_state,
'avail': availability_status, 'iss': initial_sync_state})
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
if management_state is not None:
sc.management_state = management_state
if availability_status is not None:
sc.availability_status = availability_status
if initial_sync_state is not None:
sc.initial_sync_state = initial_sync_state
sc.save()
def is_subcloud_managed(self, subcloud_name):
# is this subcloud managed
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
return sc.management_state == dccommon_consts.MANAGEMENT_MANAGED
def is_subcloud_enabled(self, subcloud_name):
# is this subcloud enabled
sc = subcloud.Subcloud.get_by_name(self.context, subcloud_name)
# We only enable syncing if the subcloud is online and the initial
# sync has completed.
return (sc.availability_status == dccommon_consts.AVAILABILITY_ONLINE and
sc.initial_sync_state == dco_consts.INITIAL_SYNC_STATE_COMPLETED)
def is_subcloud_ready(self, subcloud_name):
# is this subcloud ready for synchronization
return self.is_subcloud_managed(subcloud_name) and \
self.is_subcloud_enabled(subcloud_name)
def add_subcloud_sync_endpoint_type(self, context, subcloud_name,
endpoint_type_list=None):
# TODO(jkung): This method is currently only required by
# stx-openstack and is to be integrated with stx-openstack when
# that feature is enabled.
LOG.info("add_subcloud_sync_endpoint_type subcloud_name=%s "
"endpoint_type_list=%s" %
(subcloud_name, endpoint_type_list))
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
capabilities = sc.capabilities
c_endpoint_type_list = capabilities.get('endpoint_types', [])
# Update the DB first
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
if endpoint_type not in c_endpoint_type_list:
c_endpoint_type_list.append(endpoint_type)
if capabilities.get('endpoint_types') is None:
# assign back if 'endpoint_types' is not in capabilities
capabilities['endpoint_types'] = c_endpoint_type_list
sc.capabilities = capabilities
sc.save()
# Create objects for the endpoint types
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
# Check whether sync endpoint already exists
try:
subcloud_sync = db_api.subcloud_sync_get(
context, subcloud_name,
endpoint_type)
if subcloud_sync:
LOG.info("subcloud_sync subcloud=%s "
"endpoint_type=%s already exists" %
(subcloud_name, endpoint_type))
continue
except exceptions.SubcloudSyncNotFound:
pass
sync_obj = sync_object_class_map[endpoint_type](
subcloud_name, endpoint_type=endpoint_type)
# create the subcloud_sync !!!
db_api.subcloud_sync_create(
context, subcloud_name, endpoint_type,
values={'subcloud_id': sc.id}) # pylint: disable=E1101
if self.is_subcloud_ready(subcloud_name):
sync_obj.enable()
sync_obj.initial_sync()
def remove_subcloud_sync_endpoint_type(self, context, subcloud_name,
endpoint_type_list=None):
# TODO(jkung): This method is currently only required by
# stx-openstack and is to be integrated with stx-openstack when
# that feature is enabled and remove action performed.
# The subcloud_sync delete can be more graceful by ensuring the
# sync object is updated for each engine on delete.
LOG.info("remove_subcloud_sync_endpoint_type subcloud_name=%s "
"endpoint_type_list=%s" %
(subcloud_name, endpoint_type_list))
# Remove sync_objs and subcloud_sync for endpoint types to be removed
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
try:
db_api.subcloud_sync_delete(
context, subcloud_name, endpoint_type)
except exceptions.SubcloudSyncNotFound:
pass
# remove the endpoint types from subcloud capabilities
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
c_endpoint_type_list = sc.capabilities.get('endpoint_types', [])
if endpoint_type_list and c_endpoint_type_list:
for endpoint_type in endpoint_type_list:
if endpoint_type in c_endpoint_type_list:
c_endpoint_type_list.remove(endpoint_type)
sc.save()
def update_subcloud_version(self, context, subcloud_name, sw_version):
try:
sc = subcloud.Subcloud.get_by_name(context, subcloud_name)
sc.software_version = sw_version
sc.save()
except KeyError:
raise exceptions.SubcloudNotFound(region_name=subcloud_name)
def update_subcloud_endpoints(self, context, subcloud_name, endpoints):
try:
LOG.info("Updating service endpoints for subcloud %s in "
"endpoint cache" % subcloud_name)
endpoint_cache = sdk.OpenStackDriver(
region_name=dccommon_consts.CLOUD_0).keystone_client.endpoint_cache
endpoint_cache.update_master_service_endpoint_region(
subcloud_name, endpoints)
except (keystone_exceptions.EndpointNotFound,
keystone_exceptions.ConnectFailure,
IndexError):
LOG.error("Failed to update services endpoints for "
"subcloud: %s in dcorch." % subcloud_name)
def _audit_subcloud(self, context, subcloud_name, endpoint_type, sync_obj):
new_state = dco_consts.AUDIT_STATUS_COMPLETED
timeout = eventlet.timeout.Timeout(SYNC_TIMEOUT)
try:
sync_obj.run_sync_audit(self.engine_id)
except eventlet.timeout.Timeout as t:
if t is not timeout:
raise # not my timeout
new_state = dco_consts.AUDIT_STATUS_FAILED
except Exception as e:
LOG.exception('Audit failed for %s/%s: %s',
subcloud_name, endpoint_type, e)
new_state = dco_consts.AUDIT_STATUS_FAILED
finally:
timeout.cancel()
db_api.subcloud_sync_update(
context, subcloud_name, endpoint_type,
values={'audit_status': new_state})
def run_sync_audit(self, context, subcloud_sync_list):
# Clear the master resource cache
SyncThread.reset_master_resources_cache()
LOG.info('Engine id:(%s) Start to audit %d (subcloud, endpoint_type) pairs.'
% (self.engine_id, len(subcloud_sync_list)))
LOG.debug('Engine id:(%s) Start to audit %s.'
% (self.engine_id, subcloud_sync_list))
for sc_region_name, ept in subcloud_sync_list:
LOG.debug("Attempt audit_subcloud: %s/%s/%s",
self.engine_id, sc_region_name, ept)
try:
sync_obj = sync_object_class_map[ept](sc_region_name, ept)
self.thread_group_manager.start(self._audit_subcloud,
self.context,
sc_region_name,
ept,
sync_obj)
except exceptions.SubcloudSyncNotFound:
# The endpoint in subcloud_sync has been removed
LOG.debug("Engine id:(%s/%s/%s) SubcloudSyncNotFound: "
"The endpoint in subcloud_sync has been removed" %
(self.engine_id, sc_region_name, ept))
except Exception as e:
LOG.error("Exception occurred when running audit %s for "
"subcloud %s: %s" % (ept, sc_region_name, e))
db_api.subcloud_sync_update(
self.context, sc_region_name, ept,
values={'audit_status': dco_consts.AUDIT_STATUS_FAILED})

View File

@ -10,20 +10,19 @@
# License for the specific language governing permissions and limitations
# under the License.
#
# Copyright (c) 2020 Wind River Systems, Inc.
# Copyright (c) 2020, 2024 Wind River Systems, Inc.
#
import eventlet
from oslo_config import cfg
from oslo_log import log as logging
from dcorch.common import consts
from dcorch.common import context
from dcorch.db import api as db_api
from dcorch.engine import scheduler
from dcorch.engine import subcloud_lock
from dcorch.rpc import client
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
# How often the initial sync thread will wake up
@ -35,163 +34,94 @@ SYNC_FAIL_HOLD_OFF = 60
class InitialSyncManager(object):
"""Manages the initial sync for each subcloud."""
def __init__(self, gsm, fkm, *args, **kwargs):
def __init__(self, *args, **kwargs):
super(InitialSyncManager, self).__init__()
self.gsm = gsm
self.fkm = fkm
self.context = context.get_admin_context()
# Keeps track of greenthreads we create to do work.
self.thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=50)
# Track greenthreads created for each subcloud.
self.subcloud_threads = dict()
self.engine_worker_rpc_client = client.EngineWorkerClient()
def init_actions(self, engine_id):
def init_actions(self):
"""Perform actions on initialization"""
# Since we are starting up, any initial syncs that were in progress
# should be considered failed and must be redone.
for subcloud in db_api.subcloud_get_all(
self.context,
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS):
LOG.info('Engine id:(%s): Initial sync for subcloud %s was in '
'progress and will '
'be re-attempted' % (engine_id, subcloud.region_name))
self.gsm.update_subcloud_state(
subcloud.region_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
subclouds = db_api.subcloud_update_state_all(
self.context,
pre_initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
if subclouds > 0:
LOG.info("Initial sync for subclouds were in progress and "
"will be re-attempted.")
# Since we are starting up, any failed syncs won't be re-attempted
# because the timer will not be running. Reattempt them.
for subcloud in db_api.subcloud_get_all(
self.context,
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED):
LOG.info('Initial sync for subcloud %s was failed and will '
'be re-attempted' % subcloud.region_name)
self.gsm.update_subcloud_state(
subcloud.region_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
subclouds = db_api.subcloud_update_state_all(
self.context,
pre_initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
if subclouds > 0:
LOG.info(
"Initial sync for subclouds were failed and will be re-attempted.")
def initial_sync_thread(self, engine_id):
def initial_sync_thread(self):
"""Perform initial sync for subclouds as required."""
while True:
# Catch exceptions so the thread does not die.
try:
eventlet.greenthread.sleep(SYNC_INTERVAL)
self._initial_sync_subclouds(engine_id)
self._initial_sync_subclouds()
except eventlet.greenlet.GreenletExit:
# We have been told to exit
return
except Exception as e:
LOG.exception(e)
def _initial_sync_subclouds(self, engine_id):
def _initial_sync_subclouds(self):
"""Perform initial sync for subclouds that require it."""
LOG.debug('Engine id %s: Starting initial sync loop.' % engine_id)
for subcloud in db_api.subcloud_get_all(
self.context,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED):
# Create a new greenthread for each subcloud to allow the
# initial syncs to be done in parallel. If there are not enough
# greenthreads in the pool, this will block until one becomes
# available.
self.subcloud_threads[subcloud.region_name] = \
self.thread_group_manager.start(
self._initial_sync_subcloud, self.context, engine_id,
subcloud.region_name, 'none', 'none')
# Wait for all greenthreads to complete. This both throttles the
# initial syncs and ensures we don't attempt to do an initial sync
# for a subcloud before a previous initial sync completes.
LOG.debug('Waiting for initial syncs to complete.')
for thread in self.subcloud_threads.values():
thread.wait()
# Clear the list of threads before next audit
self.subcloud_threads = dict()
LOG.debug('All subcloud initial syncs have completed.')
@subcloud_lock.sync_subcloud
def _initial_sync_subcloud(self, context, engine_id, subcloud_name,
endpoint_type, action):
"""Perform initial sync for a subcloud.
This runs in a separate greenthread for each subcloud.
"""
LOG.info('Initial sync for subcloud %s' % subcloud_name)
# Verify that the sync state hasn't changed (there can be a delay
# before the greenthread runs).
if not self.gsm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED):
# Sync is no longer required
LOG.info('Initial sync for subcloud %s no longer required' %
subcloud_name)
return
# Indicate that initial sync has started
self.gsm.update_subcloud_state(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS)
# Initial sync. It's synchronous so that identity
# get synced before fernet token keys are synced. This is
# necessary since we want to revoke all existing tokens on
# this subcloud after its services user IDs and project
# IDs are changed. Otherwise subcloud services will fail
# authentication since they keep on using their existing tokens
# issued before these IDs change, until these tokens expires.
new_state = consts.INITIAL_SYNC_STATE_COMPLETED
try:
self.gsm.initial_sync(self.context, subcloud_name)
self.fkm.distribute_keys(self.context, subcloud_name)
self.gsm.init_subcloud_sync_audit(subcloud_name)
except Exception as e:
LOG.exception('Initial sync failed for %s: %s', subcloud_name, e)
# We need to try again
new_state = consts.INITIAL_SYNC_STATE_FAILED
# Verify that the sync wasn't cancelled while we did the sync (for
# example, the subcloud could have been unmanaged).
if self.gsm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS):
# Update initial sync state
self.gsm.update_subcloud_state(subcloud_name,
initial_sync_state=new_state)
if new_state == consts.INITIAL_SYNC_STATE_COMPLETED:
# The initial sync was completed and we have updated the
# subcloud state. Now we can enable syncing for the subcloud.
self.gsm.enable_subcloud(self.context, subcloud_name)
elif new_state == consts.INITIAL_SYNC_STATE_FAILED:
# Start a "timer" to wait a bit before re-attempting the sync.
# This thread is not taken from the thread pool, because we
# don't want a large number of failed syncs to prevent new
# subclouds from syncing.
eventlet.greenthread.spawn_after(SYNC_FAIL_HOLD_OFF,
self._reattempt_sync,
subcloud_name)
pass
else:
LOG.error('Unexpected new_state %s for subcloud %s' %
(new_state, subcloud_name))
else:
LOG.info('Initial sync was cancelled for subcloud %s while in '
'progress' % subcloud_name)
def _reattempt_sync(self, subcloud_name):
# Verify that the sync state hasn't changed since the last attempt.
if not self.gsm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED):
# Sync is no longer required
LOG.info('Reattempt initial sync for subcloud %s no longer '
'required' % subcloud_name)
return
self.gsm.update_subcloud_state(
subcloud_name,
subclouds = db_api.subcloud_capabilities_get_all(
self.context,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
if not subclouds:
LOG.info("No eligible subclouds for initial sync.")
return
LOG.info("Starting initial sync loop.")
# We want a chunksize of least 1 so add the number of workers.
chunksize = (len(subclouds) + CONF.worker_workers) // (CONF.worker_workers)
subcloud_capabilities = {}
for region_name, capabilities in subclouds.items():
subcloud_capabilities[region_name] = capabilities
if len(subcloud_capabilities) == chunksize:
# We've gathered a batch of subclouds, send it to engine worker
# to process.
try:
self.engine_worker_rpc_client.initial_sync_subclouds(
self.context,
subcloud_capabilities)
LOG.debug(
"Sent initial sync request message for %d subclouds"
% len(subcloud_capabilities)
)
except Exception as e:
LOG.error("Exception occurred in initial_sync for "
"subclouds %s: %s" %
(list(subcloud_capabilities.keys()), e))
subcloud_capabilities = {}
if len(subcloud_capabilities) > 0:
# We've got a partial batch...send it off for processing.
try:
self.engine_worker_rpc_client.initial_sync_subclouds(
self.context,
subcloud_capabilities)
LOG.debug(
"Sent initial sync request message for %d subclouds"
% len(subcloud_capabilities)
)
except Exception as e:
LOG.error("Exception occurred in initial_sync for "
"subclouds %s: %s" %
(list(subcloud_capabilities.keys()), e))
else:
LOG.debug("Done sending initial sync request messages.")

View File

@ -0,0 +1,172 @@
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import eventlet
from oslo_log import log as logging
from dcorch.common import consts
from dcorch.common import context
from dcorch.db import api as db_api
from dcorch.engine.fernet_key_manager import FernetKeyManager
from dcorch.engine import scheduler
LOG = logging.getLogger(__name__)
# How often the initial sync thread will wake up
SYNC_INTERVAL = 10
# How long to wait after a failed sync before retrying
SYNC_FAIL_HOLD_OFF = 60
class InitialSyncWorkerManager(object):
"""Manages the initial sync for each subcloud."""
def __init__(self, gswm, engine_id, *args, **kwargs):
super(InitialSyncWorkerManager, self).__init__()
self.gswm = gswm
self.engine_id = engine_id
self.context = context.get_admin_context()
# Keeps track of greenthreads we create to do work.
self.thread_group_manager = scheduler.ThreadGroupManager(
thread_pool_size=100)
def initial_sync_subclouds(self, context, subcloud_capabilities):
"""Perform initial sync for subclouds that require it."""
LOG.info('Engine id:(%s) Start initial sync for %d subclouds.'
% (self.engine_id, len(subcloud_capabilities)))
LOG.debug('Engine id:(%s) Start initial sync for subclouds %s.'
% (self.engine_id, list(subcloud_capabilities.keys())))
for sc_region_name, sc_capabilities in subcloud_capabilities.items():
# Create a new greenthread for each subcloud to allow the
# initial syncs to be done in parallel. If there are not enough
# greenthreads in the pool, this will block until one becomes
# available.
try:
self.thread_group_manager.start(
self._initial_sync_subcloud,
self.context,
sc_region_name,
sc_capabilities)
except Exception as e:
LOG.error("Exception occurred when running initial_sync for "
"subcloud %s: %s" % (sc_region_name, e))
def _initial_sync_subcloud(self, context, subcloud_name, subcloud_capabilities):
"""Perform initial sync for a subcloud.
This runs in a separate greenthread for each subcloud.
"""
LOG.debug('Initial sync for subcloud %s' % subcloud_name)
# Verify that the sync state hasn't changed (there can be a delay
# before the greenthread runs).
if not self.gswm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED):
# Sync is no longer required
LOG.debug('Initial sync for subcloud %s no longer required' %
subcloud_name)
return
# Indicate that initial sync has started
self.gswm.update_subcloud_state(
context,
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS)
# sync_objs stores the sync object per endpoint
sync_objs = self.gswm.create_sync_objects(
subcloud_name, subcloud_capabilities)
# Initial sync. It's synchronous so that identity
# get synced before fernet token keys are synced. This is
# necessary since we want to revoke all existing tokens on
# this subcloud after its services user IDs and project
# IDs are changed. Otherwise subcloud services will fail
# authentication since they keep on using their existing tokens
# issued before these IDs change, until these tokens expires.
new_state = consts.INITIAL_SYNC_STATE_COMPLETED
try:
self.initial_sync(subcloud_name, sync_objs)
FernetKeyManager.distribute_keys(subcloud_name)
self.init_subcloud_sync_audit(subcloud_name)
except Exception as e:
LOG.exception('Initial sync failed for %s: %s', subcloud_name, e)
# We need to try again
new_state = consts.INITIAL_SYNC_STATE_FAILED
# Verify that the sync wasn't cancelled while we did the sync (for
# example, the subcloud could have been unmanaged).
if self.gswm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS):
# Update initial sync state
self.gswm.update_subcloud_state(context,
subcloud_name,
initial_sync_state=new_state)
if new_state == consts.INITIAL_SYNC_STATE_COMPLETED:
# The initial sync was completed and we have updated the
# subcloud state. Now we can enable syncing for the subcloud.
self.enable_subcloud(subcloud_name, sync_objs)
elif new_state == consts.INITIAL_SYNC_STATE_FAILED:
# Start a "timer" to wait a bit before re-attempting the sync.
# This thread is not taken from the thread pool, because we
# don't want a large number of failed syncs to prevent new
# subclouds from syncing.
eventlet.greenthread.spawn_after(SYNC_FAIL_HOLD_OFF,
self._reattempt_sync,
subcloud_name)
pass
else:
LOG.error('Unexpected new_state %s for subcloud %s' %
(new_state, subcloud_name))
else:
LOG.debug('Initial sync was cancelled for subcloud %s while in '
'progress' % subcloud_name)
def _reattempt_sync(self, subcloud_name):
# Verify that the sync state hasn't changed since the last attempt.
if not self.gswm.subcloud_state_matches(
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED):
# Sync is no longer required
LOG.debug('Reattempt initial sync for subcloud %s no longer required'
% subcloud_name)
return
self.gswm.update_subcloud_state(
self.context,
subcloud_name,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
def enable_subcloud(self, subcloud_name, sync_objs):
LOG.debug('enabling subcloud %(sc)s' % {'sc': subcloud_name})
for endpoint_type, sync_obj in sync_objs.items():
LOG.debug('Engine id: %s enabling sync thread for '
'subcloud %s and endpoint type %s' %
(self.engine_id, subcloud_name, endpoint_type))
sync_obj.enable()
def init_subcloud_sync_audit(self, subcloud_name):
LOG.debug('Initialize subcloud sync audit for subcloud %(sc)s'
% {'sc': subcloud_name})
for endpoint_type in consts.SYNC_ENDPOINT_TYPES_LIST:
db_api.subcloud_sync_update(
self.context, subcloud_name, endpoint_type,
values={'audit_status': consts.AUDIT_STATUS_NONE,
'sync_status_reported': consts.SYNC_STATUS_NONE,
'sync_status_report_time': None,
'last_audit_time': None})
def initial_sync(self, subcloud_name, sync_objs):
LOG.debug('Initial sync subcloud %(sc)s %(id)s' %
{'sc': subcloud_name, 'id': self.engine_id})
for sync_obj in sync_objs.values():
sync_obj.initial_sync()

View File

@ -18,7 +18,6 @@ import collections
import copy
import re
import threading
import time
from oslo_config import cfg
from oslo_log import log as logging
@ -34,7 +33,6 @@ from dcorch.common import manager
from dcorch.common import utils
from dcorch.db import api as db_api
from dcorch.drivers.openstack import sdk
from dcorch.engine import dc_orch_lock
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
@ -74,11 +72,6 @@ class QuotaManager(manager.Manager):
self.context = context.get_admin_context()
self.endpoints = endpoint_cache.EndpointCache()
# This lock is used to ensure we only have one quota sync audit at
# a time. For better efficiency we could use per-project locks
# and/or the ReaderWriterLock from the "fastener" package.
self.quota_audit_lock = threading.Lock()
@classmethod
def calculate_subcloud_project_quotas(cls, project_id, user_id,
new_global_quotas, subcloud):
@ -132,19 +125,8 @@ class QuotaManager(manager.Manager):
pass
return list(project_user_list)
def periodic_balance_all(self, engine_id):
def periodic_balance_all(self):
LOG.info("periodically balance quota for all keystone tenants")
lock = dc_orch_lock.sync_lock_acquire(engine_id, TASK_TYPE,
self.quota_audit_lock)
if not lock:
LOG.error("Not able to acquire lock for %(task_type)s, may"
" be Previous sync job has not finished yet, "
"Aborting this run at: %(time)s ",
{'task_type': TASK_TYPE,
'time': time.strftime("%c")}
)
return
LOG.info("Successfully acquired lock")
projects_thread_list = []
# Generate a list of project_id/user_id tuples that need to have their
@ -192,8 +174,6 @@ class QuotaManager(manager.Manager):
# the job(sync all projects quota)
for current_thread in projects_thread_list:
current_thread.join()
dc_orch_lock.sync_lock_release(engine_id, TASK_TYPE,
self.quota_audit_lock)
def read_quota_usage(self, project_id, user_id, region, usage_queue):
# Writes usage dict to the Queue in the following format

View File

@ -21,7 +21,6 @@ from oslo_config import cfg
from oslo_log import log as logging
import oslo_messaging
from oslo_service import service
from oslo_utils import timeutils
from oslo_utils import uuidutils
import six
@ -31,13 +30,13 @@ from dcorch.common import context
from dcorch.common import exceptions
from dcorch.common.i18n import _
from dcorch.common import messaging as rpc_messaging
from dcorch.db import api as db_api
from dcorch.engine.fernet_key_manager import FernetKeyManager
from dcorch.engine.generic_sync_manager import GenericSyncManager
from dcorch.engine.generic_sync_worker_manager import GenericSyncWorkerManager
from dcorch.engine.initial_sync_manager import InitialSyncManager
from dcorch.engine.initial_sync_worker_manager import InitialSyncWorkerManager
from dcorch.engine.quota_manager import QuotaManager
from dcorch.engine import scheduler
from dcorch.objects import service as service_obj
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
@ -57,16 +56,9 @@ def request_context(func):
class EngineService(service.Service):
"""Lifecycle manager for a running service engine.
"""Lifecycle manager for a running audit service."""
- All the methods in here are called from the RPC client.
- If a RPC call does not have a corresponding method here, an exceptions
will be thrown.
- Arguments to these calls are added dynamically and will be treated as
keyword arguments by the RPC client.
"""
def __init__(self, host, topic, manager=None):
def __init__(self):
super(EngineService, self).__init__()
self.host = cfg.CONF.host
@ -74,10 +66,8 @@ class EngineService(service.Service):
self.topic = consts.TOPIC_ORCH_ENGINE
# The following are initialized here, but assigned in start() which
# happens after the fork when spawning multiple worker processes
self.engine_id = None
self.TG = None
self.periodic_enable = cfg.CONF.scheduler.periodic_enable
self.periodic_interval = cfg.CONF.scheduler.periodic_interval
self.target = None
self._rpc_server = None
self.qm = None
@ -85,33 +75,10 @@ class EngineService(service.Service):
self.fkm = None
self.ism = None
def init_tgm(self):
self.TG = scheduler.ThreadGroupManager()
def init_qm(self):
self.qm = QuotaManager()
def init_gsm(self):
ctxt = context.get_admin_context()
self.gsm = GenericSyncManager(self.engine_id)
self.gsm.init_from_db(ctxt)
self.TG.start(self.gsm.sync_job_thread, self.engine_id)
self.TG.start(self.gsm.sync_audit_thread, self.engine_id)
def init_fkm(self):
self.fkm = FernetKeyManager(self.gsm)
def init_ism(self):
self.ism = InitialSyncManager(self.gsm, self.fkm)
self.ism.init_actions(self.engine_id)
self.TG.start(self.ism.initial_sync_thread, self.engine_id)
def start(self):
LOG.info("Starting %s", self.__class__.__name__)
self.engine_id = uuidutils.generate_uuid()
target = oslo_messaging.Target(version=self.rpc_api_version,
server=self.host,
topic=self.topic)
target = oslo_messaging.Target(
version=self.rpc_api_version, server=self.host, topic=self.topic
)
self.target = target
self._rpc_server = rpc_messaging.get_rpc_server(self.target, self)
self._rpc_server.start()
@ -122,17 +89,8 @@ class EngineService(service.Service):
self.init_fkm()
self.init_ism()
self.service_registry_cleanup()
self.set_resource_limit()
self.TG.add_timer(cfg.CONF.report_interval,
self.service_registry_report)
self.TG.add_timer(2 * self.periodic_interval,
self.sync_lock_cleanup)
super(EngineService, self).start()
if self.periodic_enable:
LOG.info("Adding periodic tasks for the engine to perform")
self.TG.add_timer(CONF.fernet.key_rotation_interval *
@ -141,63 +99,37 @@ class EngineService(service.Service):
initial_delay=(CONF.fernet.key_rotation_interval
* dccommon_consts.SECONDS_IN_HOUR))
def service_registry_report(self):
ctx = context.get_admin_context()
try:
svc = service_obj.Service.update(ctx, self.engine_id)
# if svc is None, means it's not created.
if svc is None:
service_obj.Service.create(ctx, self.engine_id, self.host,
'dcorch-engine', self.topic)
except Exception as ex:
LOG.error('Service %(service_id)s update failed: %(error)s',
{'service_id': self.engine_id, 'error': ex})
def init_tgm(self):
self.TG = scheduler.ThreadGroupManager()
def service_registry_cleanup(self):
ctx = context.get_admin_context()
time_window = (2 * cfg.CONF.report_interval)
services = service_obj.Service.get_all(ctx)
for svc in services:
if svc['id'] == self.engine_id:
continue
if timeutils.is_older_than(svc['updated_at'], time_window):
# < time_line:
# hasn't been updated, assuming it's died.
LOG.info('Service %s was aborted', svc['id'])
service_obj.Service.delete(ctx, svc['id'])
# Delete sync locks where service ID no longer exists. This could
# happen if the process is terminated abnormally e.g. poweroff
db_api.purge_stale_sync_lock(ctx)
def init_qm(self):
self.qm = QuotaManager()
def sync_lock_cleanup(self):
ctx = context.get_admin_context()
time_window = (2 * cfg.CONF.report_interval)
services = service_obj.Service.get_all(ctx)
for svc in services:
if svc['id'] == self.engine_id:
continue
if timeutils.is_older_than(svc['updated_at'], time_window):
# delete the stale sync lock if any
LOG.debug("To delete the stale locks")
db_api.sync_lock_delete_by_engine_id(ctx, svc['id'])
def init_gsm(self):
self.gsm = GenericSyncManager()
self.TG.start(self.gsm.sync_job_thread)
self.TG.start(self.gsm.sync_audit_thread)
def set_resource_limit(self):
try:
resource.setrlimit(resource.RLIMIT_NOFILE, (cfg.CONF.rlimit_nofile,
cfg.CONF.rlimit_nofile))
except Exception as ex:
LOG.error('Engine id %s: failed to set the NOFILE resource limit: '
'%s' % (self.engine_id, ex))
def init_fkm(self):
self.fkm = FernetKeyManager(self.gsm)
def delete_sync_lock(self, service_id):
ctx = context.get_admin_context()
db_api.sync_lock_delete_by_engine_id(ctx, service_id)
def init_ism(self):
self.ism = InitialSyncManager()
self.ism.init_actions()
self.TG.start(self.ism.initial_sync_thread)
def periodic_balance_all(self, engine_id):
@request_context
# The sync job info has been written to the DB, alert the sync engine
# that there is work to do.
# TODO(lzhu1): add authentication since ctxt not actually needed later
def sync_request(self, ctxt, endpoint_type):
self.gsm.sync_request(ctxt, endpoint_type)
def periodic_balance_all(self):
# Automated Quota Sync for all the keystone projects
LOG.info("Periodic quota sync job started at: %s",
time.strftime("%c"))
self.qm.periodic_balance_all(engine_id)
self.qm.periodic_balance_all()
@request_context
def get_usage_for_project_and_user(self, context, endpoint_type,
@ -214,106 +146,15 @@ class EngineService(service.Service):
project_id, user_id)
self.qm.quota_sync_for_project(project_id, user_id)
@request_context
def add_subcloud(self, ctxt, subcloud_name, sw_version):
self.gsm.add_subcloud(ctxt, subcloud_name, sw_version)
@request_context
# todo: add authentication since ctxt not actually needed later
def del_subcloud(self, ctxt, subcloud_name):
self.gsm.del_subcloud(ctxt, subcloud_name)
@request_context
# todo: add authentication since ctxt not actually needed later
def update_subcloud_states(self, ctxt, subcloud_name,
management_state,
availability_status):
"""Handle subcloud state updates from dcmanager
These state updates must be processed quickly. Any work triggered by
these state updates must be done asynchronously, without delaying the
reply to the dcmanager. For example, it is not acceptable to
communicate with a subcloud while handling the state update.
"""
# Check if state has changed before doing anything
if self.gsm.subcloud_state_matches(
subcloud_name,
management_state=management_state,
availability_status=availability_status):
# No change in state - nothing to do.
LOG.debug('Ignoring unchanged state update for %s' % subcloud_name)
return
# Check if the subcloud is ready to sync.
if (management_state == dccommon_consts.MANAGEMENT_MANAGED) and \
(availability_status == dccommon_consts.AVAILABILITY_ONLINE):
# Update the subcloud state and schedule an initial sync
self.gsm.update_subcloud_state(
subcloud_name,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
else:
# Update the subcloud state and cancel the initial sync
self.gsm.update_subcloud_state(
subcloud_name,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
@request_context
def add_subcloud_sync_endpoint_type(self, ctxt, subcloud_name,
endpoint_type_list=None):
try:
self.gsm.add_subcloud_sync_endpoint_type(
ctxt, subcloud_name,
endpoint_type_list=endpoint_type_list)
except Exception as ex:
LOG.warning('Add subcloud endpoint type failed for %s: %s',
subcloud_name, six.text_type(ex))
raise
@request_context
def remove_subcloud_sync_endpoint_type(self, ctxt, subcloud_name,
endpoint_type_list=None):
try:
self.gsm.remove_subcloud_sync_endpoint_type(
ctxt, subcloud_name,
endpoint_type_list=endpoint_type_list)
except Exception as ex:
LOG.warning('Remove subcloud endpoint type failed for %s: %s',
subcloud_name, six.text_type(ex))
raise
@request_context
# todo: add authentication since ctxt not actually needed later
def update_subcloud_version(self, ctxt, subcloud_name, sw_version):
self.gsm.update_subcloud_version(ctxt, subcloud_name, sw_version)
@request_context
def update_subcloud_endpoints(self, ctxt, subcloud_name, endpoints):
self.gsm.update_subcloud_endpoints(ctxt, subcloud_name, endpoints)
@request_context
# The sync job info has been written to the DB, alert the sync engine
# that there is work to do.
# todo: add authentication since ctxt not actually needed later
def sync_request(self, ctxt, endpoint_type):
self.gsm.sync_request(ctxt, endpoint_type)
def _stop_rpc_server(self):
# Stop RPC connection to prevent new requests
LOG.debug(_("Attempting to stop engine service..."))
try:
if self._rpc_server:
self._rpc_server.stop()
self._rpc_server.wait()
LOG.info('Engine service stopped successfully')
self._rpc_server.stop()
self._rpc_server.wait()
LOG.info("Engine service stopped successfully")
except Exception as ex:
LOG.error('Failed to stop engine service: %s',
six.text_type(ex))
LOG.error(f"Failed to stop engine service: {six.text_type(ex)}")
def stop(self):
self._stop_rpc_server()
@ -329,3 +170,191 @@ class EngineService(service.Service):
"""Periodic key rotation."""
LOG.info("Periodic key rotation started at: %s", time.strftime("%c"))
return self.fkm.rotate_fernet_keys()
class EngineWorkerService(service.Service):
"""Lifecycle manager for a running service engine.
- All the methods in here are called from the RPC client.
- If a RPC call does not have a corresponding method here, an exceptions
will be thrown.
- Arguments to these calls are added dynamically and will be treated as
keyword arguments by the RPC client.
"""
def __init__(self):
super(EngineWorkerService, self).__init__()
self.host = cfg.CONF.host
self.rpc_api_version = consts.RPC_API_VERSION
self.topic = consts.TOPIC_ORCH_ENGINE_WORKER
# The following are initialized here, but assigned in start() which
# happens after the fork when spawning multiple worker processes
self.engine_id = None
self.TG = None
self.target = None
self._rpc_server = None
self.gswm = None
self.iswm = None
def init_tgm(self):
self.TG = scheduler.ThreadGroupManager()
def init_gswm(self):
self.gswm = GenericSyncWorkerManager(self.engine_id)
def init_iswm(self):
self.iswm = InitialSyncWorkerManager(self.gswm, self.engine_id)
def start(self):
LOG.info("Starting %s", self.__class__.__name__)
self.engine_id = uuidutils.generate_uuid()
target = oslo_messaging.Target(version=self.rpc_api_version,
server=self.host,
topic=self.topic)
self.target = target
self._rpc_server = rpc_messaging.get_rpc_server(self.target, self)
self._rpc_server.start()
self.init_tgm()
self.init_gswm()
self.init_iswm()
self.set_resource_limit()
super(EngineWorkerService, self).start()
def set_resource_limit(self):
try:
resource.setrlimit(resource.RLIMIT_NOFILE, (cfg.CONF.rlimit_nofile,
cfg.CONF.rlimit_nofile))
except Exception as ex:
LOG.error('Engine id %s: failed to set the NOFILE resource limit: '
'%s' % (self.engine_id, ex))
@request_context
def add_subcloud(self, ctxt, subcloud_name, sw_version):
self.gswm.add_subcloud(ctxt, subcloud_name, sw_version)
@request_context
# todo: add authentication since ctxt not actually needed later
def del_subcloud(self, ctxt, subcloud_name):
self.gswm.del_subcloud(ctxt, subcloud_name)
@request_context
# todo: add authentication since ctxt not actually needed later
def update_subcloud_states(self, ctxt, subcloud_name,
management_state,
availability_status):
"""Handle subcloud state updates from dcmanager
These state updates must be processed quickly. Any work triggered by
these state updates must be done asynchronously, without delaying the
reply to the dcmanager. For example, it is not acceptable to
communicate with a subcloud while handling the state update.
"""
# Check if state has changed before doing anything
if self.gswm.subcloud_state_matches(
subcloud_name,
management_state=management_state,
availability_status=availability_status):
# No change in state - nothing to do.
LOG.debug('Ignoring unchanged state update for %s' % subcloud_name)
return
# Check if the subcloud is ready to sync.
if (management_state == dccommon_consts.MANAGEMENT_MANAGED) and \
(availability_status == dccommon_consts.AVAILABILITY_ONLINE):
# Update the subcloud state and schedule an initial sync
self.gswm.update_subcloud_state(
ctxt,
subcloud_name,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
else:
# Update the subcloud state and cancel the initial sync
self.gswm.update_subcloud_state(
ctxt,
subcloud_name,
management_state=management_state,
availability_status=availability_status,
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
@request_context
def update_subcloud_state(self, ctxt, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
LOG.info("Trigger update state for subcloud %s", subcloud_name)
self.gswm.update_subcloud_state(ctxt, subcloud_name,
management_state,
availability_status,
initial_sync_state)
@request_context
def add_subcloud_sync_endpoint_type(self, ctxt, subcloud_name,
endpoint_type_list=None):
try:
self.gswm.add_subcloud_sync_endpoint_type(
ctxt, subcloud_name,
endpoint_type_list=endpoint_type_list)
except Exception as ex:
LOG.warning('Add subcloud endpoint type failed for %s: %s',
subcloud_name, six.text_type(ex))
raise
@request_context
def remove_subcloud_sync_endpoint_type(self, ctxt, subcloud_name,
endpoint_type_list=None):
try:
self.gswm.remove_subcloud_sync_endpoint_type(
ctxt, subcloud_name,
endpoint_type_list=endpoint_type_list)
except Exception as ex:
LOG.warning('Remove subcloud endpoint type failed for %s: %s',
subcloud_name, six.text_type(ex))
raise
@request_context
def sync_subclouds(self, ctxt, subcloud_sync_list):
self.gswm.sync_subclouds(ctxt, subcloud_sync_list)
@request_context
def run_sync_audit(self, ctxt, subcloud_sync_list):
self.gswm.run_sync_audit(ctxt, subcloud_sync_list)
@request_context
def initial_sync_subclouds(self, ctxt, subcloud_capabilities):
self.iswm.initial_sync_subclouds(ctxt, subcloud_capabilities)
@request_context
# todo: add authentication since ctxt not actually needed later
def update_subcloud_version(self, ctxt, subcloud_name, sw_version):
self.gswm.update_subcloud_version(ctxt, subcloud_name, sw_version)
@request_context
def update_subcloud_endpoints(self, ctxt, subcloud_name, endpoints):
self.gswm.update_subcloud_endpoints(ctxt, subcloud_name, endpoints)
def _stop_rpc_server(self):
# Stop RPC connection to prevent new requests
LOG.debug(_("Attempting to stop engine-worker service..."))
try:
if self._rpc_server:
self._rpc_server.stop()
self._rpc_server.wait()
LOG.info('Engine-worker service stopped successfully')
except Exception as ex:
LOG.error(f"Failed to stop engine-worker service: {six.text_type(ex)}")
def stop(self):
self._stop_rpc_server()
if self.TG:
self.TG.stop()
# Terminate the engine process
LOG.info("All threads were gone, terminating engine-worker")
super(EngineWorkerService, self).stop()

View File

@ -1,94 +0,0 @@
# Copyright 2020 Wind River Inc.
# All Rights Reserved.
#
# Copyright 2016 Ericsson AB
#
# 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.
from oslo_db import exception as db_exc
from oslo_log import log as logging
from dcorch.db import api as db_api
LOG = logging.getLogger(__name__)
def sync_subcloud(func):
"""Synchronized lock decorator for _update_subcloud_endpoint_status. """
def _get_lock_and_call(*args, **kwargs):
"""Get a single fair lock per subcloud based on subcloud name. """
# context is the 2nd argument
# engine_id is the 3rd argument
# subcloud name is the 4rd argument
# endpoint_type is the 5th argument
# action is the 6th argument
def _call_func(*args, **kwargs):
if sync_lock_acquire(args[1], args[2], args[3], args[4], args[5]):
try:
result = func(*args, **kwargs)
return result
finally:
sync_lock_release(args[1], args[2], args[3], args[4],
args[5])
return _call_func(*args, **kwargs)
return _get_lock_and_call
def sync_lock_acquire(context, engine_id, name, endpoint_type, action):
"""Try to lock with specified engine_id.
:param context: the security context
:param engine_id: ID of the engine which wants to lock the projects.
:param name: the name of the resource to lock
:param endpoint_type: service type of a subcloud
:param action: action to be performed (i.e. audit or sync)
:returns: True if lock is acquired, or False otherwise.
"""
LOG.debug('Trying to acquire lock with %(engId)s for Resource: %(name)s '
'Type: %(type)s, action: %(action)s',
{'engId': engine_id,
'name': name,
'type': endpoint_type,
'action': action
}
)
try:
lock_status = db_api.sync_lock_acquire(context, engine_id, name,
endpoint_type, action)
except db_exc.DBDuplicateEntry:
return False
if lock_status:
return True
return False
def sync_lock_release(context, engine_id, name, endpoint_type, action):
"""Release the lock for the projects"""
LOG.debug('Releasing acquired lock with %(engId)s for subcloud: %(name)s '
'%(type)s, %(action)s',
{'engId': engine_id,
'name': name,
'type': endpoint_type,
'action': action
}
)
return db_api.sync_lock_release(context, name, endpoint_type, action)

View File

@ -87,7 +87,7 @@ class IdentitySyncThread(SyncThread):
self.log_extra = {"instance": "{}/{}: ".format(
self.region_name, self.endpoint_type)}
LOG.info("IdentitySyncThread initialized", extra=self.log_extra)
LOG.debug("IdentitySyncThread initialized", extra=self.log_extra)
@staticmethod
def get_os_client(region):
@ -285,7 +285,7 @@ class IdentitySyncThread(SyncThread):
# service recovery time at subcloud.
# get users from master cloud
m_users = self.get_master_resources(
m_users = self.get_cached_master_resources(
consts.RESOURCE_TYPE_IDENTITY_USERS)
if not m_users:
@ -305,7 +305,7 @@ class IdentitySyncThread(SyncThread):
self._initial_sync_users(m_users, sc_users)
# get groups from master cloud
m_groups = self.get_master_resources(
m_groups = self.get_cached_master_resources(
consts.RESOURCE_TYPE_IDENTITY_GROUPS)
if not m_groups:
@ -324,7 +324,7 @@ class IdentitySyncThread(SyncThread):
self._initial_sync_groups(m_groups, sc_groups)
# get projects from master cloud
m_projects = self.get_master_resources(
m_projects = self.get_cached_master_resources(
consts.RESOURCE_TYPE_IDENTITY_PROJECTS)
if not m_projects:
@ -344,7 +344,7 @@ class IdentitySyncThread(SyncThread):
self._initial_sync_projects(m_projects, sc_projects)
# get roles from master cloud
m_roles = self.get_master_resources(
m_roles = self.get_cached_master_resources(
consts.RESOURCE_TYPE_IDENTITY_ROLES)
if not m_roles:
@ -2057,7 +2057,8 @@ class IdentitySyncThread(SyncThread):
consts.RESOURCE_TYPE_IDENTITY_TOKEN_REVOKE_EVENTS_FOR_USER)\
and resource.user_id and resource.issued_before:
event_id = "{}_{}".format(resource.user_id, resource.issued_before)
return base64.urlsafe_b64encode(event_id)
return base64.urlsafe_b64encode(
event_id.encode('utf-8')).decode('utf-8')
# Default id field retrieved from master cloud
return resource.id

View File

@ -81,7 +81,7 @@ class SysinvSyncThread(SyncThread):
consts.RESOURCE_TYPE_SYSINV_FERNET_REPO,
]
LOG.info("SysinvSyncThread initialized", extra=self.log_extra)
LOG.debug("SysinvSyncThread initialized", extra=self.log_extra)
def sync_platform_resource(self, request, rsrc):
try:
@ -518,7 +518,8 @@ class SysinvSyncThread(SyncThread):
return None
def post_audit(self):
super(SysinvSyncThread, self).post_audit()
# TODO(lzhu1): This should be revisited once the master cache service
# is implemented.
sdk.OpenStackDriver.delete_region_clients_for_thread(
self.region_name, 'audit')
sdk.OpenStackDriver.delete_region_clients_for_thread(

View File

@ -343,9 +343,9 @@ class SyncThread(object):
# Early exit in case there are no pending sync requests
if not sync_requests:
LOG.info("Sync resources done for subcloud - "
"no sync requests",
extra=self.log_extra)
LOG.debug("Sync resources done for subcloud - "
"no sync requests",
extra=self.log_extra)
self.set_sync_status(dccommon_consts.SYNC_STATUS_IN_SYNC)
return
@ -508,8 +508,8 @@ class SyncThread(object):
if self.endpoint_type in cfg.CONF.disable_audit_endpoints:
LOG.warn("Audit disabled!", extra=self.log_extra)
return
LOG.info("Engine id={}: sync_audit started".format(engine_id),
extra=self.log_extra)
LOG.debug("Engine id={}: sync_audit started".format(engine_id),
extra=self.log_extra)
self.sync_audit(engine_id)
def sync_audit(self, engine_id):
@ -580,8 +580,8 @@ class SyncThread(object):
# Extra resources in subcloud are not impacted by the audit.
if not num_of_audit_jobs:
LOG.info("Clean audit run for {}".format(resource_type),
extra=self.log_extra)
LOG.debug("Clean audit run for {}".format(resource_type),
extra=self.log_extra)
else:
LOG.info("{} num_of_audit_jobs for {}".
format(num_of_audit_jobs, resource_type),
@ -610,17 +610,20 @@ class SyncThread(object):
LOG.debug("{}: done sync audit".format(
threading.currentThread().getName()), extra=self.log_extra)
from dcorch.engine.generic_sync_manager import GenericSyncManager
GenericSyncManager.set_sync_request(self.ctxt, self.subcloud_name,
self.endpoint_type)
SyncThread.set_sync_request(
self.ctxt, self.subcloud_name, self.endpoint_type)
self.post_audit()
@lockutils.synchronized(AUDIT_LOCK_NAME)
def post_audit(self):
# Some specific SyncThread subclasses may perform post audit actions
pass
@classmethod
@lockutils.synchronized(AUDIT_LOCK_NAME)
def reset_master_resources_cache(cls):
# reset the cached master resources
LOG.debug("Reset the cached master resources.")
SyncThread.master_resources_dict = collections.defaultdict(dict)
# The specific SyncThread subclasses may perform additional post
# audit actions
def audit_find_missing(self, resource_type, m_resources,
db_resources, sc_resources,
@ -931,3 +934,9 @@ class SyncThread(object):
# exists in subcloud resources.
def resource_exists_in_subcloud(self, subcloud_rsrc, sc_resources):
return True
@classmethod
def set_sync_request(cls, ctxt, subcloud_name, endpoint_type):
db_api.subcloud_sync_update(
ctxt, subcloud_name, endpoint_type,
values={'sync_request': consts.SYNC_STATUS_REQUESTED})

View File

@ -1,4 +1,4 @@
# Copyright (c) 2017-2023 Wind River Systems, Inc.
# Copyright (c) 2017-2024 Wind River Systems, Inc.
# 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
@ -15,13 +15,9 @@
Client side of the DC Orchestrator RPC API.
"""
from oslo_log import log as logging
from dcorch.common import consts
from dcorch.common import messaging
LOG = logging.getLogger(__name__)
class EngineClient(object):
"""Client side of the DC orchestrator engine rpc API.
@ -57,6 +53,12 @@ class EngineClient(object):
client = self._client
return client.cast(ctxt, method, **kwargs)
# The sync job info has been written to the DB, alert the sync engine
# that there is work to do.
def sync_request(self, ctxt, endpoint_type):
return self.cast(
ctxt, self.make_msg('sync_request', endpoint_type=endpoint_type))
def get_usage_for_project_and_user(self, ctxt, endpoint_type,
project_id, user_id=None):
return self.call(ctxt, self.make_msg('get_usage_for_project_and_user',
@ -69,6 +71,41 @@ class EngineClient(object):
project_id=project_id,
user_id=user_id))
class EngineWorkerClient(object):
"""Client side of the DC orchestrator engine worker rpc API.
Version History:
1.0 - Initial version
"""
BASE_RPC_API_VERSION = '1.0'
def __init__(self):
self._client = messaging.get_rpc_client(
topic=consts.TOPIC_ORCH_ENGINE_WORKER,
version=self.BASE_RPC_API_VERSION)
@staticmethod
def make_msg(method, **kwargs):
return method, kwargs
def call(self, ctxt, msg, version=None):
method, kwargs = msg
if version is not None:
client = self._client.prepare(version=version)
else:
client = self._client
return client.call(ctxt, method, **kwargs)
def cast(self, ctxt, msg, fanout=None, version=None):
method, kwargs = msg
if version or fanout:
client = self._client.prepare(fanout=fanout, version=version)
else:
client = self._client
return client.cast(ctxt, method, **kwargs)
def keypair_sync_for_user(self, ctxt, job_id, payload):
return self.cast(
ctxt,
@ -116,6 +153,24 @@ class EngineClient(object):
subcloud_name=subcloud_name,
endpoint_type_list=endpoint_type_list))
def sync_subclouds(self, ctxt, subcloud_sync_list):
return self.cast(
ctxt,
self.make_msg('sync_subclouds',
subcloud_sync_list=subcloud_sync_list))
def run_sync_audit(self, ctxt, subcloud_sync_list):
return self.cast(
ctxt,
self.make_msg('run_sync_audit',
subcloud_sync_list=subcloud_sync_list))
def initial_sync_subclouds(self, ctxt, subcloud_capabilities):
return self.cast(
ctxt,
self.make_msg('initial_sync_subclouds',
subcloud_capabilities=subcloud_capabilities))
def update_subcloud_version(self, ctxt, subcloud_name, sw_version):
return self.call(
ctxt,
@ -126,9 +181,3 @@ class EngineClient(object):
return self.cast(ctxt, self.make_msg(
'update_subcloud_endpoints', subcloud_name=subcloud_name,
endpoints=endpoints), fanout=True, version=self.BASE_RPC_API_VERSION)
# The sync job info has been written to the DB, alert the sync engine
# that there is work to do.
def sync_request(self, ctxt, endpoint_type):
return self.cast(
ctxt, self.make_msg('sync_request', endpoint_type=endpoint_type))

View File

@ -24,6 +24,7 @@ from oslo_db import options
from oslotest import base
import sqlalchemy
from dccommon import consts as dccommon_consts
from dcmanager.rpc import client as dcmanager_rpc_client
from dcorch.db import api
from dcorch.db.sqlalchemy import api as db_api
@ -34,6 +35,12 @@ from dcorch.tests import utils
get_engine = api.get_engine
CAPABILITES = {
'endpoint_types':
[dccommon_consts.ENDPOINT_TYPE_PLATFORM,
dccommon_consts.ENDPOINT_TYPE_IDENTITY]}
class FakeException(Exception):
"""Exception used to throw a generic exception in the application
@ -81,7 +88,7 @@ class OrchestratorTestCase(base.BaseTestCase):
def _mock_rpc_client(self):
"""Mock rpc's manager client"""
mock_patch = mock.patch.object(rpc_client, 'EngineClient')
mock_patch = mock.patch.object(rpc_client, 'EngineWorkerClient')
self.mock_rpc_client = mock_patch.start()
self.addCleanup(mock_patch.stop)

View File

@ -129,6 +129,8 @@ class BaseTestIdentitySyncThread(OrchestratorTestCase, mixins.BaseMixin):
self.log.info.assert_called_with(message, extra=extra)
elif level == 'error':
self.log.error.assert_called_with(message, extra=extra)
elif level == 'debug':
self.log.debug.assert_called_with(message, extra=extra)
class BaseTestIdentitySyncThreadUsers(BaseTestIdentitySyncThread):
@ -553,7 +555,7 @@ class TestIdentitySyncThreadProjectRoleAssignmentsPut(
self._execute()
self._assert_log('info', 'IdentitySyncThread initialized')
self._assert_log('debug', 'IdentitySyncThread initialized')
self.log.error.assert_not_called()

View File

@ -12,199 +12,261 @@
# under the License.
#
from datetime import timedelta
import math
import mock
from oslo_utils import uuidutils
from oslo_config import cfg
from oslo_utils import timeutils
from dccommon import consts as dccommon_consts
from dcorch.common import consts
from dcorch.common import exceptions
from dcorch.db.sqlalchemy import api as db_api
from dcorch.engine import generic_sync_manager
from dcorch.engine.sync_services import sysinv
from dcorch.tests import base
from dcorch.tests import utils
class FakeSyncThread(object):
def __init__(self):
self.start = mock.MagicMock()
CONF = cfg.CONF
class TestGenericSyncManager(base.OrchestratorTestCase):
def setUp(self):
super(TestGenericSyncManager, self).setUp()
self.engine_id = uuidutils.generate_uuid()
# Mock the sysinv sync methods
self.fake_sync_thread_sysinv = FakeSyncThread()
p = mock.patch.object(sysinv, 'SysinvSyncThread')
self.mock_sync_service_sysinv = p.start()
self.mock_sync_service_sysinv.return_value = self.fake_sync_thread_sysinv
# Mock the DCorch engine-worker API client
p = mock.patch('dcorch.rpc.client.EngineWorkerClient')
self.mock_dcorch_api = p.start()
self.addCleanup(p.stop)
@staticmethod
def create_subcloud_static(ctxt, name, **kwargs):
values = {
'software_version': '10.04',
'management_state': dccommon_consts.MANAGEMENT_MANAGED,
'availability_status': dccommon_consts.AVAILABILITY_ONLINE,
'initial_sync_state': '',
'capabilities': {},
}
values.update(kwargs)
return db_api.subcloud_create(ctxt, name, values=values)
def test_init(self):
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
gsm = generic_sync_manager.GenericSyncManager()
self.assertIsNotNone(gsm)
def test_init_from_db(self):
def test_process_subclouds(self):
# Create a list including 22 (subcloud, endpoint_type) pairs and
# distribute them into chunks based on the number of workers.
subcloud_sync_list = list()
chunks = list()
chunk_num = -1
for i in range(1, 23):
region_name = 'subcloud' + str(i)
subcloud_sync_identity = \
(region_name, dccommon_consts.ENDPOINT_TYPE_IDENTITY)
subcloud_sync_list.append(subcloud_sync_identity)
if (i - 1) % CONF.worker_workers == 0:
chunk_num += 1
chunks.insert(chunk_num, list())
chunks[chunk_num].append(subcloud_sync_identity)
self.create_subcloud_static(
self.ctx,
name='subcloud1')
self.create_subcloud_static(
self.ctx,
name='subcloud2')
self.create_subcloud_static(
self.ctx,
name='subcloud3')
gsm = generic_sync_manager.GenericSyncManager()
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
rpc_method = mock.MagicMock()
rpc_method.__name__ = 'mock_rpc_method'
gsm._process_subclouds(rpc_method, subcloud_sync_list)
# Initialize from the DB
gsm.init_from_db(self.ctx)
# Verify the number of chunks
self.assertEqual(math.ceil(len(subcloud_sync_list) / CONF.worker_workers),
len(chunks))
# Verify rpc call for each chunk of subclouds
for chunk in chunks:
rpc_method.assert_any_call(mock.ANY, chunk)
# Verify the engines were created
self.assertEqual(gsm.sync_objs['subcloud1'], {})
self.assertEqual(gsm.sync_objs['subcloud2'], {})
self.assertEqual(gsm.sync_objs['subcloud3'], {})
def test_subcloud_state_matches(self):
self.create_subcloud_static(
def test_sync_subclouds(self):
# Create subcloud1 not eligible for sync due to initial_sync_state
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
# Initialize from the DB
gsm.init_from_db(self.ctx)
# Compare all states (match)
match = gsm.subcloud_state_matches(
'subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud1',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
sync_request=consts.SYNC_STATUS_REQUESTED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud1',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
sync_request=consts.SYNC_STATUS_REQUESTED)
# Create subcloud2 not eligible for sync due to sync_request
utils.create_subcloud_static(
self.ctx,
name='subcloud2',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertTrue(match)
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud2',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
sync_request=consts.SYNC_STATUS_IN_PROGRESS)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud2',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
sync_request=consts.SYNC_STATUS_IN_PROGRESS)
# Create 22 eligible subclouds
subcloud_sync_list = []
for i in range(3, 25):
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud' + str(i),
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud' + str(i),
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
sync_request='requested')
subcloud_sync_list.append((subcloud.region_name,
dccommon_consts.ENDPOINT_TYPE_IDENTITY))
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud' + str(i),
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
sync_request='requested')
subcloud_sync_list.append((subcloud.region_name,
dccommon_consts.ENDPOINT_TYPE_PLATFORM))
# Compare all states (not a match)
match = gsm.subcloud_state_matches(
'subcloud1',
gsm = generic_sync_manager.GenericSyncManager()
gsm._process_subclouds = mock.MagicMock()
gsm.sync_subclouds()
gsm._process_subclouds.assert_called_once_with(
self.mock_dcorch_api().sync_subclouds, subcloud_sync_list)
# Verify the sync_request of the subclouds were updated to in-progress
for i in range(3, 25):
subcloud_sync_identity = db_api.subcloud_sync_get(
self.ctx,
'subcloud' + str(i),
dccommon_consts.ENDPOINT_TYPE_IDENTITY)
self.assertEqual(consts.SYNC_STATUS_IN_PROGRESS,
subcloud_sync_identity.sync_request)
subcloud_sync_platform = db_api.subcloud_sync_get(
self.ctx,
'subcloud' + str(i),
dccommon_consts.ENDPOINT_TYPE_PLATFORM)
self.assertEqual(consts.SYNC_STATUS_IN_PROGRESS,
subcloud_sync_platform.sync_request)
def test_run_sync_audit(self):
# Create subcloud1 not eligible for audit due to initial_sync_state
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertFalse(match)
# Compare one state (match)
match = gsm.subcloud_state_matches(
'subcloud1',
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud1',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
audit_status=consts.AUDIT_STATUS_NONE)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud1',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
audit_status=consts.AUDIT_STATUS_NONE)
# Create subcloud2 not eligible for audit due to management_state
utils.create_subcloud_static(
self.ctx,
name='subcloud2',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE)
self.assertTrue(match)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud2',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
audit_status=consts.AUDIT_STATUS_FAILED)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud2',
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
audit_status=consts.AUDIT_STATUS_FAILED)
# Create 22 eligible subclouds
subcloud_sync_list = []
for i in range(3, 25):
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud' + str(i),
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
last_audit_time = timeutils.utcnow() - \
timedelta(seconds=generic_sync_manager.AUDIT_INTERVAL)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud' + str(i),
endpoint_type=dccommon_consts.ENDPOINT_TYPE_IDENTITY,
audit_status=consts.AUDIT_STATUS_COMPLETED,
last_audit_time=last_audit_time)
subcloud_sync_list.append((subcloud.region_name,
dccommon_consts.ENDPOINT_TYPE_IDENTITY))
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud' + str(i),
endpoint_type=dccommon_consts.ENDPOINT_TYPE_PLATFORM,
audit_status=consts.AUDIT_STATUS_COMPLETED,
last_audit_time=last_audit_time)
subcloud_sync_list.append((subcloud.region_name,
dccommon_consts.ENDPOINT_TYPE_PLATFORM))
# Compare one state (not a match)
match = gsm.subcloud_state_matches(
'subcloud1',
initial_sync_state='')
self.assertFalse(match)
gsm = generic_sync_manager.GenericSyncManager()
gsm._process_subclouds = mock.MagicMock()
gsm.run_sync_audit()
def test_subcloud_state_matches_missing(self):
gsm._process_subclouds.assert_called_once_with(
self.mock_dcorch_api().run_sync_audit, subcloud_sync_list)
self.create_subcloud_static(
# Verify the audit_status of the subclouds were updated to in-progress
for i in range(3, 25):
subcloud_sync_identity = db_api.subcloud_sync_get(
self.ctx,
'subcloud' + str(i),
dccommon_consts.ENDPOINT_TYPE_IDENTITY)
self.assertEqual(consts.AUDIT_STATUS_IN_PROGRESS,
subcloud_sync_identity.audit_status)
subcloud_sync_platform = db_api.subcloud_sync_get(
self.ctx,
'subcloud' + str(i),
dccommon_consts.ENDPOINT_TYPE_PLATFORM)
self.assertEqual(consts.AUDIT_STATUS_IN_PROGRESS,
subcloud_sync_platform.audit_status)
def test_sync_request(self):
subcloud1 = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
# Initialize from the DB
gsm.init_from_db(self.ctx)
# Compare all states for missing subcloud
self.assertRaises(
exceptions.SubcloudNotFound,
gsm.subcloud_state_matches,
'subcloud2',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
def test_update_subcloud_state(self):
self.create_subcloud_static(
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
utils.create_subcloud_sync_static(
self.ctx,
name='subcloud1',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
subcloud1.region_name,
dccommon_consts.ENDPOINT_TYPE_IDENTITY,
subcloud_id=subcloud1.id)
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
# Initialize from the DB
gsm.init_from_db(self.ctx)
# Update all states
gsm.update_subcloud_state(
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
# Compare all states (match)
match = gsm.subcloud_state_matches(
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
self.assertTrue(match)
# Update one state
gsm.update_subcloud_state(
'subcloud1',
availability_status=dccommon_consts.AVAILABILITY_ONLINE)
# Compare all states (match)
match = gsm.subcloud_state_matches(
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
self.assertTrue(match)
def test_update_subcloud_state_missing(self):
self.create_subcloud_static(
subcloud2 = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
name='subcloud2',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
utils.create_subcloud_sync_static(
self.ctx,
subcloud2.region_name,
dccommon_consts.ENDPOINT_TYPE_IDENTITY,
vsubcloud_id=subcloud2.id)
gsm = generic_sync_manager.GenericSyncManager(self.engine_id)
gsm = generic_sync_manager.GenericSyncManager()
gsm.sync_request(self.ctx, dccommon_consts.ENDPOINT_TYPE_IDENTITY)
# Initialize from the DB
gsm.init_from_db(self.ctx)
# Update all states for missing subcloud
self.assertRaises(
exceptions.SubcloudNotFound,
gsm.update_subcloud_state,
'subcloud2',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
# Verify the sync_request of the subclouds were updated to requested
subcloud_sync = db_api.subcloud_sync_get(
self.ctx, 'subcloud1', dccommon_consts.ENDPOINT_TYPE_IDENTITY)
self.assertEqual(consts.SYNC_STATUS_REQUESTED,
subcloud_sync.sync_request)
subcloud_sync = db_api.subcloud_sync_get(
self.ctx, 'subcloud2', dccommon_consts.ENDPOINT_TYPE_IDENTITY)
self.assertEqual(consts.SYNC_STATUS_REQUESTED,
subcloud_sync.sync_request)

View File

@ -0,0 +1,133 @@
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import mock
from oslo_service import threadgroup
from oslo_utils import uuidutils
from dccommon import consts as dccommon_consts
from dcorch.common import consts
from dcorch.engine import generic_sync_worker_manager
from dcorch.tests import base
from dcorch.tests import utils
SUBCLOUD_SYNC_LIST = [
('subcloud1', dccommon_consts.ENDPOINT_TYPE_IDENTITY),
('subcloud1', dccommon_consts.ENDPOINT_TYPE_PLATFORM),
('subcloud2', dccommon_consts.ENDPOINT_TYPE_IDENTITY),
('subcloud2', dccommon_consts.ENDPOINT_TYPE_PLATFORM)
]
class TestGenericSyncWorkerManager(base.OrchestratorTestCase):
def setUp(self):
super(TestGenericSyncWorkerManager, self).setUp()
self.engine_id = uuidutils.generate_uuid()
# Mock sync_object_class_map
p = mock.patch.object(generic_sync_worker_manager,
'sync_object_class_map',
{dccommon_consts.ENDPOINT_TYPE_PLATFORM:
mock.MagicMock(),
dccommon_consts.ENDPOINT_TYPE_IDENTITY:
mock.MagicMock(),
dccommon_consts.ENDPOINT_TYPE_IDENTITY_OS:
mock.MagicMock()})
self.mock_sync_object_class_map = p.start()
self.addCleanup(mock.patch.stopall)
# Mock thread
p = mock.patch.object(threadgroup, 'Thread')
self.mock_thread = p.start()
self.addCleanup(p.stop)
# Mock ThreadGroupManager start
p = mock.patch('dcorch.engine.scheduler.ThreadGroupManager.start')
self.mock_thread_start = p.start()
self.mock_thread_start.return_value = self.mock_thread
self.addCleanup(p.stop)
def test_init(self):
gswm = generic_sync_worker_manager.GenericSyncWorkerManager(self.engine_id)
self.assertIsNotNone(gswm)
def test_create_sync_objects(self):
gswm = generic_sync_worker_manager.GenericSyncWorkerManager(self.engine_id)
sync_objs = gswm.create_sync_objects('subcloud1', base.CAPABILITES)
# Verify both endpoint types have corresponding sync object
self.assertEqual(len(sync_objs), 2)
self.assertIn(dccommon_consts.ENDPOINT_TYPE_PLATFORM, sync_objs)
self.assertIn(dccommon_consts.ENDPOINT_TYPE_IDENTITY, sync_objs)
def test_update_subcloud_state(self):
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
management_state=dccommon_consts.MANAGEMENT_MANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
gswm = generic_sync_worker_manager.GenericSyncWorkerManager(self.engine_id)
# Update all states
gswm.update_subcloud_state(
self.ctx,
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
# Compare all states (match)
match = gswm.subcloud_state_matches(
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_OFFLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
self.assertTrue(match)
# Update one state
gswm.update_subcloud_state(
self.ctx,
'subcloud1',
availability_status=dccommon_consts.AVAILABILITY_ONLINE)
# Compare all states (match)
match = gswm.subcloud_state_matches(
'subcloud1',
management_state=dccommon_consts.MANAGEMENT_UNMANAGED,
availability_status=dccommon_consts.AVAILABILITY_ONLINE,
initial_sync_state=consts.INITIAL_SYNC_STATE_COMPLETED)
self.assertTrue(match)
def test_sync_subclouds(self):
gswm = generic_sync_worker_manager.GenericSyncWorkerManager(self.engine_id)
gswm._sync_subcloud = mock.MagicMock()
gswm.sync_subclouds(self.ctx, SUBCLOUD_SYNC_LIST)
# Verify 4 threads started, one for each endpoint_type of a subcloud
for subcloud_name, endpoint_type in SUBCLOUD_SYNC_LIST:
self.mock_thread_start.assert_any_call(
gswm._sync_subcloud,
mock.ANY,
subcloud_name,
endpoint_type)
def test_run_sync_audit(self):
gswm = generic_sync_worker_manager.GenericSyncWorkerManager(self.engine_id)
gswm._audit_subcloud = mock.MagicMock()
gswm.run_sync_audit(self.ctx, SUBCLOUD_SYNC_LIST)
# Verify 4 threads started, one for each endpoint_type of a subcloud
for subcloud_name, endpoint_type in SUBCLOUD_SYNC_LIST:
self.mock_thread_start.assert_any_call(
gswm._audit_subcloud,
mock.ANY,
subcloud_name,
endpoint_type,
mock.ANY)

View File

@ -13,98 +13,59 @@
# License for the specific language governing permissions and limitations
# under the License.
import math
import mock
from oslo_utils import uuidutils
from oslo_config import cfg
from dccommon import consts as dccommon_consts
from dcorch.common import consts
from dcorch.db.sqlalchemy import api as db_api
from dcorch.engine import initial_sync_manager
from dcorch.rpc import client
from dcorch.tests import base
from dcorch.tests import utils
class FakeGSM(object):
def __init__(self, ctx):
self.ctx = ctx
self.initial_sync = mock.MagicMock()
self.enable_subcloud = mock.MagicMock()
self.init_subcloud_sync_audit = mock.MagicMock()
def update_subcloud_state(self, name, initial_sync_state):
db_api.subcloud_update(
self.ctx,
name,
values={'initial_sync_state': initial_sync_state})
def subcloud_state_matches(self, name, initial_sync_state):
subcloud = db_api.subcloud_get(self.ctx, name)
return subcloud.initial_sync_state == initial_sync_state
class FakeFKM(object):
def __init__(self):
self.distribute_keys = mock.MagicMock()
CONF = cfg.CONF
class TestInitialSyncManager(base.OrchestratorTestCase):
def setUp(self):
super(TestInitialSyncManager, self).setUp()
self.engine_id = uuidutils.generate_uuid()
# Mock eventlet
p = mock.patch('eventlet.greenthread.spawn_after')
self.mock_eventlet_spawn_after = p.start()
self.addCleanup(p.stop)
# Mock the context
p = mock.patch.object(initial_sync_manager, 'context')
self.mock_context = p.start()
self.mock_context.get_admin_context.return_value = self.ctx
self.addCleanup(p.stop)
# Mock the GSM and FKM
self.fake_gsm = FakeGSM(self.ctx)
self.fake_fkm = FakeFKM()
@staticmethod
def create_subcloud_static(ctxt, name, **kwargs):
values = {
'software_version': '10.04',
'availability_status': dccommon_consts.AVAILABILITY_ONLINE,
}
values.update(kwargs)
return db_api.subcloud_create(ctxt, name, values=values)
# Mock the DCorch engine-worker API client
mock_patch = mock.patch.object(client, 'EngineWorkerClient')
self.mock_rpc_client = mock_patch.start()
self.addCleanup(mock_patch.stop)
def test_init(self):
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
ism = initial_sync_manager.InitialSyncManager()
self.assertIsNotNone(ism)
self.assertEqual(self.ctx, ism.context)
def test_init_actions(self):
subcloud = self.create_subcloud_static(
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
subcloud = self.create_subcloud_static(
utils.create_subcloud_static(
self.ctx,
name='subcloud2',
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS)
subcloud = self.create_subcloud_static(
utils.create_subcloud_static(
self.ctx,
name='subcloud3',
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
subcloud = self.create_subcloud_static(
utils.create_subcloud_static(
self.ctx,
name='subcloud4',
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
utils.create_subcloud_static(
self.ctx,
name='subcloud5',
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS)
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
ism = initial_sync_manager.InitialSyncManager()
# Perform init actions
ism.init_actions(self.engine_id)
ism.init_actions()
# Verify the subclouds are in the correct initial sync state
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
@ -119,118 +80,37 @@ class TestInitialSyncManager(base.OrchestratorTestCase):
subcloud = db_api.subcloud_get(self.ctx, 'subcloud4')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_REQUESTED)
def test_initial_sync_subcloud(self):
subcloud = self.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertIsNotNone(subcloud)
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
# Initial sync the subcloud
ism._initial_sync_subcloud(self.ctx,
self.engine_id,
subcloud.region_name, None, None)
# Verify that the initial sync steps were done
self.fake_gsm.initial_sync.assert_called_with(self.ctx,
subcloud.region_name)
self.fake_fkm.distribute_keys.assert_called_with(self.ctx,
subcloud.region_name)
# Verify that the subcloud was enabled
self.fake_gsm.enable_subcloud.assert_called_with(self.ctx,
subcloud.region_name)
# Verify the initial sync was completed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_COMPLETED)
def test_initial_sync_subcloud_not_required(self):
subcloud = self.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state='')
self.assertIsNotNone(subcloud)
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
# Initial sync the subcloud
ism._initial_sync_subcloud(self.ctx,
self.engine_id,
subcloud.region_name, None, None)
# Verify that the initial sync steps were not done
self.fake_gsm.initial_sync.assert_not_called()
# Verify the initial sync state was not changed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state, '')
def test_initial_sync_subcloud_failed(self):
subcloud = self.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertIsNotNone(subcloud)
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
# Force a failure
self.fake_gsm.initial_sync.side_effect = Exception('fake_exception')
# Initial sync the subcloud
ism._initial_sync_subcloud(self.ctx,
self.engine_id,
subcloud.region_name, None, None)
# Verify the initial sync was failed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_FAILED)
# Verify that the subcloud was not enabled
self.fake_gsm.enable_subcloud.assert_not_called()
# Verify the initial sync was retried
self.mock_eventlet_spawn_after.assert_called_with(
initial_sync_manager.SYNC_FAIL_HOLD_OFF, mock.ANY, 'subcloud1')
def test_reattempt_sync(self):
subcloud = self.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
subcloud = self.create_subcloud_static(
self.ctx,
name='subcloud2',
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
ism = initial_sync_manager.InitialSyncManager(self.fake_gsm,
self.fake_fkm)
# Reattempt sync success
ism._reattempt_sync('subcloud2')
# Verify the subcloud is in the correct initial sync state
subcloud = db_api.subcloud_get(self.ctx, 'subcloud2')
subcloud = db_api.subcloud_get(self.ctx, 'subcloud5')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_REQUESTED)
# Reattempt sync when not needed
ism._reattempt_sync('subcloud1')
def test_initial_sync_subclouds(self):
# Create subcloud1 not eligible for initial sync due to initial_sync_state
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_IN_PROGRESS)
chunks = list()
chunk_num = -1
# Create 21 eligible subclouds
for i in range(2, 23):
if (i - 1) % CONF.worker_workers == 1:
chunk_num += 1
chunks.insert(chunk_num, dict())
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud' + str(i),
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
chunks[chunk_num][subcloud.region_name] = base.CAPABILITES
# Verify the subcloud is in the correct initial sync state
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_NONE)
ism = initial_sync_manager.InitialSyncManager()
# Perform initial sync for subclouds
ism._initial_sync_subclouds()
# Verify the number of chunks
self.assertEqual(math.ceil(21 / CONF.worker_workers), len(chunks))
# Verify a thread started for each chunk of subclouds
for chunk in chunks:
self.mock_rpc_client().initial_sync_subclouds.assert_any_call(
mock.ANY, chunk)

View File

@ -0,0 +1,241 @@
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
import mock
from oslo_service import threadgroup
from oslo_utils import uuidutils
from dcorch.common import consts
from dcorch.db.sqlalchemy import api as db_api
from dcorch.engine import initial_sync_worker_manager
from dcorch.tests import base
from dcorch.tests import utils
class FakeSyncObject(object):
def initial_sync(self):
pass
def enable(self):
pass
class FakeGSWM(object):
def __init__(self, ctx, engine_id):
self.ctx = ctx
self.engine_id = engine_id
def update_subcloud_state(self, ctx, subcloud_name, initial_sync_state):
db_api.subcloud_update(
ctx,
subcloud_name,
values={'initial_sync_state': initial_sync_state})
def create_sync_objects(self, subcloud_name, capabilities):
sync_objs = {}
endpoint_type_list = capabilities.get('endpoint_types', None)
if endpoint_type_list:
for endpoint_type in endpoint_type_list:
sync_obj = FakeSyncObject()
sync_objs.update({endpoint_type: sync_obj})
return sync_objs
def subcloud_state_matches(self, subcloud_name,
management_state=None,
availability_status=None,
initial_sync_state=None):
# compare subcloud states
match = True
sc = db_api.subcloud_get(self.ctx, subcloud_name)
if management_state is not None and \
sc.management_state != management_state:
match = False
if match and availability_status is not None and \
sc.availability_status != availability_status:
match = False
if match and initial_sync_state is not None and \
sc.initial_sync_state != initial_sync_state:
match = False
return match
class TestInitialSyncWorkerManager(base.OrchestratorTestCase):
def setUp(self):
super(TestInitialSyncWorkerManager, self).setUp()
self.engine_id = uuidutils.generate_uuid()
self.fake_gswm = FakeGSWM(self.ctx, self.engine_id)
# Mock eventlet
p = mock.patch('eventlet.greenthread.spawn_after')
self.mock_eventlet_spawn_after = p.start()
self.addCleanup(p.stop)
# Mock FernetKeyManager distribute_Keys
p = mock.patch(
'dcorch.engine.fernet_key_manager.FernetKeyManager.distribute_keys')
self.mock_distribute_keys = p.start()
self.addCleanup(p.stop)
# Mock db_api subcloud_sync_update
p = mock.patch('dcorch.db.api.subcloud_sync_update')
self.mock_subcloud_sync_update = p.start()
self.addCleanup(p.stop)
# Mock thread
p = mock.patch.object(threadgroup, 'Thread')
self.mock_thread = p.start()
self.addCleanup(p.stop)
# Mock ThreadGroupManager start
p = mock.patch('dcorch.engine.scheduler.ThreadGroupManager.start')
self.mock_thread_start = p.start()
self.mock_thread_start.return_value = self.mock_thread
self.addCleanup(p.stop)
def test_init(self):
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
self.assertIsNotNone(iswm)
def test_initial_sync_subcloud(self):
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertIsNotNone(subcloud)
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
# Initial sync the subcloud
iswm._initial_sync_subcloud(self.ctx,
subcloud.region_name,
base.CAPABILITES)
self.mock_distribute_keys.assert_called_once()
# Verify subcloud_sync_update called twice due to two endpoint types
self.assertEqual(2, self.mock_subcloud_sync_update.call_count)
# Verify the initial sync was completed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_COMPLETED)
def test_initial_sync_subcloud_not_required(self):
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state='')
self.assertIsNotNone(subcloud)
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
iswm.initial_sync = mock.MagicMock()
# Initial sync the subcloud
iswm._initial_sync_subcloud(self.ctx,
subcloud.region_name,
base.CAPABILITES)
# Verify that the initial sync steps were not done
iswm.initial_sync.assert_not_called()
self.mock_distribute_keys.assert_not_called()
self.mock_subcloud_sync_update.assert_not_called()
# Verify the initial sync state was not changed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state, '')
def test_initial_sync_subcloud_failed(self):
subcloud = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_REQUESTED)
self.assertIsNotNone(subcloud)
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
iswm.enable_subcloud = mock.MagicMock()
# Force a failure
self.mock_distribute_keys.side_effect = Exception('fake_exception')
# Initial sync the subcloud
iswm._initial_sync_subcloud(self.ctx,
subcloud.region_name,
base.CAPABILITES)
# Verify the initial sync was failed
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_FAILED)
# Verify that the subcloud was not enabled
iswm.enable_subcloud.assert_not_called()
# Verify the initial sync was retried
self.mock_eventlet_spawn_after.assert_called_with(
initial_sync_worker_manager.SYNC_FAIL_HOLD_OFF, mock.ANY, 'subcloud1')
def test_reattempt_sync(self):
utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state=consts.INITIAL_SYNC_STATE_NONE)
utils.create_subcloud_static(
self.ctx,
name='subcloud2',
initial_sync_state=consts.INITIAL_SYNC_STATE_FAILED)
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
# Reattempt sync success
iswm._reattempt_sync('subcloud2')
# Verify the subcloud is in the correct initial sync state
subcloud = db_api.subcloud_get(self.ctx, 'subcloud2')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_REQUESTED)
# Reattempt sync when not needed
iswm._reattempt_sync('subcloud1')
# Verify the subcloud is in the correct initial sync state
subcloud = db_api.subcloud_get(self.ctx, 'subcloud1')
self.assertEqual(subcloud.initial_sync_state,
consts.INITIAL_SYNC_STATE_NONE)
def test_initial_sync_subclouds(self):
subcloud1 = utils.create_subcloud_static(
self.ctx,
name='subcloud1',
initial_sync_state='')
subcloud2 = utils.create_subcloud_static(
self.ctx,
name='subcloud2',
initial_sync_state='')
subcloud_capabilities = {subcloud1.region_name: base.CAPABILITES,
subcloud2.region_name: base.CAPABILITES}
iswm = initial_sync_worker_manager.InitialSyncWorkerManager(
self.fake_gswm, self.engine_id)
iswm.initial_sync_subclouds(self.ctx, subcloud_capabilities)
# Verify 2 threads started, one for each of the subcloud
self.mock_thread_start.assert_any_call(iswm._initial_sync_subcloud,
mock.ANY,
subcloud1.region_name,
subcloud_capabilities.get(
subcloud1.region_name))
self.mock_thread_start.assert_called_with(iswm._initial_sync_subcloud,
mock.ANY,
subcloud2.region_name,
subcloud_capabilities.get(
subcloud2.region_name))

View File

@ -23,8 +23,10 @@ from oslo_config import cfg
from oslo_db import options
import sqlalchemy
from dccommon import consts as dccommon_consts
from dcorch.common import context
from dcorch.db import api as db_api
from dcorch.tests import base
get_engine = db_api.get_engine
@ -95,3 +97,26 @@ def wait_until_true(predicate, timeout=60, sleep=1, exception=None):
with eventlet.timeout.Timeout(timeout, exception):
while not predicate():
eventlet.sleep(sleep)
def create_subcloud_static(ctxt, name, **kwargs):
values = {
'software_version': '10.04',
'management_state': dccommon_consts.MANAGEMENT_MANAGED,
'availability_status': dccommon_consts.AVAILABILITY_ONLINE,
'initial_sync_state': '',
'capabilities': base.CAPABILITES
}
values.update(kwargs)
return db_api.subcloud_create(ctxt, name, values=values)
def create_subcloud_sync_static(ctxt, name, endpoint_type, **kwargs):
values = {
'subcloud_name': name,
'endpoint_type': endpoint_type,
'subcloud_id': '',
'sync_request': ''
}
values.update(kwargs)
return db_api.subcloud_sync_create(ctxt, name, endpoint_type, values=values)

View File

@ -3,11 +3,13 @@ usr/bin/clean-dcorch
usr/bin/dcorch-api
usr/bin/dcorch-api-proxy
usr/bin/dcorch-engine
usr/bin/dcorch-engine-worker
usr/bin/dcorch-manage
usr/lib/ocf/resource.d/openstack/dcorch-*
usr/lib/python3/dist-packages/dcorch/*
usr/lib/systemd/system/dcorch-api.service
usr/lib/systemd/system/dcorch-engine.service
usr/lib/systemd/system/dcorch-engine-worker.service
usr/lib/systemd/system/dcorch-sysinv-api-proxy.service
usr/lib/systemd/system/dcorch-identity-api-proxy.service
usr/lib/tmpfiles.d/dcorch.conf

View File

@ -33,6 +33,7 @@ override_dh_install:
install -p -D -m 644 files/dcorch-api.service $(SYSTEMD_DIR)/dcorch-api.service
install -p -D -m 644 files/dcorch-engine.service $(SYSTEMD_DIR)/dcorch-engine.service
install -p -D -m 644 files/dcorch-engine-worker.service $(SYSTEMD_DIR)/dcorch-engine-worker.service
install -p -D -m 644 files/dcorch-sysinv-api-proxy.service $(SYSTEMD_DIR)/dcorch-sysinv-api-proxy.service
install -p -D -m 644 files/dcorch-identity-api-proxy.service $(SYSTEMD_DIR)/dcorch-identity-api-proxy.service

View File

@ -0,0 +1,12 @@
[Unit]
Description=DC Orchestrator Engine-worker Service
After=syslog.target network.target mysqld.service openstack-keystone.service
[Service]
Type=simple
User=root
ExecStart=/usr/bin/dcorch-engine-worker --config-file /etc/dcorch/dcorch.conf
Restart=on-failure
[Install]
WantedBy=multi-user.target

View File

@ -1,5 +1,5 @@
[Unit]
Description=DC Manager Service
Description=DC Orchestrator Engine Service
After=syslog.target network.target mysqld.service openstack-keystone.service
[Service]

View File

@ -233,12 +233,12 @@ dcorch_engine_confirm_stop() {
local my_processes
my_binary=`which ${OCF_RESKEY_binary}`
my_processes=`pgrep -l -f "^(python|/usr/bin/python|/usr/bin/python2) ${my_binary}([^\w-]|$)"`
my_processes=`pgrep -l -f "^(python|/usr/bin/python|/usr/bin/python3) ${my_binary}([^\w-]|$)"`
if [ -n "${my_processes}" ]
then
ocf_log info "About to SIGKILL the following: ${my_processes}"
pkill -KILL -f "^(python|/usr/bin/python|/usr/bin/python2) ${my_binary}([^\w-]|$)"
pkill -KILL -f "^(python|/usr/bin/python|/usr/bin/python3) ${my_binary}([^\w-]|$)"
fi
}

View File

@ -0,0 +1,323 @@
#!/bin/sh
# OpenStack DC Orchestrator Engine-worker Service (dcorch-engine-worker)
#
# Description: Manages an OpenStack DC Orchestrator Engine-worker Service (dcorch-engine-worker) process as an HA resource
#
# Copyright (c) 2024 Wind River Systems, Inc.
#
# SPDX-License-Identifier: Apache-2.0
#
#
# See usage() function below for more details ...
#
# OCF instance parameters:
# OCF_RESKEY_binary
# OCF_RESKEY_config
# OCF_RESKEY_user
# OCF_RESKEY_pid
# OCF_RESKEY_additional_parameters
#######################################################################
# Initialization:
: ${OCF_FUNCTIONS_DIR=${OCF_ROOT}/lib/heartbeat}
. ${OCF_FUNCTIONS_DIR}/ocf-shellfuncs
#######################################################################
# Fill in some defaults if no values are specified
OCF_RESKEY_binary_default="/usr/bin/dcorch-engine-worker"
OCF_RESKEY_config_default="/etc/dcorch/dcorch.conf"
OCF_RESKEY_user_default="root"
OCF_RESKEY_pid_default="$HA_RSCTMP/$OCF_RESOURCE_INSTANCE.pid"
: ${OCF_RESKEY_binary=${OCF_RESKEY_binary_default}}
: ${OCF_RESKEY_config=${OCF_RESKEY_config_default}}
: ${OCF_RESKEY_user=${OCF_RESKEY_user_default}}
: ${OCF_RESKEY_pid=${OCF_RESKEY_pid_default}}
#######################################################################
usage() {
cat <<UEND
usage: $0 (start|stop|validate-all|meta-data|status|monitor)
$0 manages an OpenStack DC Orchestrator Engine-worker (dcorch-engine-worker) process as an HA resource
The 'start' operation starts the dcorch-engine-worker service.
The 'stop' operation stops the dcorch-engine-worker service.
The 'validate-all' operation reports whether the parameters are valid
The 'meta-data' operation reports this RA's meta-data information
The 'status' operation reports whether the dcorch-engine-worker service is running
The 'monitor' operation reports whether the dcorch-engine-worker service seems to be working
UEND
}
meta_data() {
cat <<END
<?xml version="1.0"?>
<!DOCTYPE resource-agent SYSTEM "ra-api-1.dtd">
<resource-agent name="dcorch-engine-worker">
<version>1.0</version>
<longdesc lang="en">
Resource agent for the DC Orchestrator Engine-worker Service (dcorch-engine-worker)
</longdesc>
<shortdesc lang="en">Manages the OpenStack DC Orchestrator Engine-worker Service(dcorch-engine-worker)</shortdesc>
<parameters>
<parameter name="binary" unique="0" required="0">
<longdesc lang="en">
Location of the DC Orchestrator Engine-worker binary (dcorch-engine-worker)
</longdesc>
<shortdesc lang="en">DC Orchestrator Engine-worker binary (dcorch-engine-worker)</shortdesc>
<content type="string" default="${OCF_RESKEY_binary_default}" />
</parameter>
<parameter name="config" unique="0" required="0">
<longdesc lang="en">
Location of the DC Orchestrator Engine-worker (dcorch-engine-worker) configuration file
</longdesc>
<shortdesc lang="en">DC Orchestrator Engine-worker (dcorch-engine-worker registry) config file</shortdesc>
<content type="string" default="${OCF_RESKEY_config_default}" />
</parameter>
<parameter name="user" unique="0" required="0">
<longdesc lang="en">
User running DC Orchestrator Engine-worker (dcorch-engine-worker)
</longdesc>
<shortdesc lang="en">DC Orchestrator Engine-worker (dcorch-engine-worker) user</shortdesc>
<content type="string" default="${OCF_RESKEY_user_default}" />
</parameter>
<parameter name="pid" unique="0" required="0">
<longdesc lang="en">
The pid file to use for this DC Orchestrator Engine-worker (dcorch-engine-worker) instance
</longdesc>
<shortdesc lang="en">DC Orchestrator Engine-worker (dcorch-engine-worker) pid file</shortdesc>
<content type="string" default="${OCF_RESKEY_pid_default}" />
</parameter>
<parameter name="additional_parameters" unique="0" required="0">
<longdesc lang="en">
Additional parameters to pass on to the OpenStack DC Orchestrator Engine-worker (dcorch-engine-worker)
</longdesc>
<shortdesc lang="en">Additional parameters for dcorch-engine-worker</shortdesc>
<content type="string" />
</parameter>
</parameters>
<actions>
<action name="start" timeout="20" />
<action name="stop" timeout="20" />
<action name="status" timeout="20" />
<action name="monitor" timeout="10" interval="5" />
<action name="validate-all" timeout="5" />
<action name="meta-data" timeout="5" />
</actions>
</resource-agent>
END
}
#######################################################################
# Functions invoked by resource manager actions
dcorch_engine_worker_validate() {
local rc
check_binary $OCF_RESKEY_binary
check_binary curl
check_binary tr
check_binary grep
check_binary cut
check_binary head
# A config file on shared storage that is not available
# during probes is OK.
if [ ! -f $OCF_RESKEY_config ]; then
if ! ocf_is_probe; then
ocf_log err "Config $OCF_RESKEY_config doesn't exist"
return $OCF_ERR_INSTALLED
fi
ocf_log_warn "Config $OCF_RESKEY_config not available during a probe"
fi
getent passwd $OCF_RESKEY_user >/dev/null 2>&1
rc=$?
if [ $rc -ne 0 ]; then
ocf_log err "User $OCF_RESKEY_user doesn't exist"
return $OCF_ERR_INSTALLED
fi
true
}
dcorch_engine_worker_status() {
local pid
local rc
if [ ! -f $OCF_RESKEY_pid ]; then
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) is not running"
return $OCF_NOT_RUNNING
else
pid=`cat $OCF_RESKEY_pid`
fi
ocf_run -warn kill -s 0 $pid
rc=$?
if [ $rc -eq 0 ]; then
return $OCF_SUCCESS
else
ocf_log info "Old PID file found, but DC Orchestrator Engine-worker (dcorch-engine-worker) is not running"
rm -f $OCF_RESKEY_pid
return $OCF_NOT_RUNNING
fi
}
dcorch_engine_worker_monitor() {
local rc
dcorch_engine_worker_status
rc=$?
# If status returned anything but success, return that immediately
if [ $rc -ne $OCF_SUCCESS ]; then
return $rc
fi
# Further verify the service availibility.
ocf_log debug "DC Orchestrator Engine-worker (dcorch-engine-worker) monitor succeeded"
return $OCF_SUCCESS
}
dcorch_engine_worker_start() {
local rc
dcorch_engine_worker_status
rc=$?
if [ $rc -eq $OCF_SUCCESS ]; then
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) already running"
return $OCF_SUCCESS
fi
# Change the working dir to /, to be sure it's accesible
cd /
# run the actual dcorch-engine-worker daemon. Don't use ocf_run as we're sending the tool's output
# straight to /dev/null anyway and using ocf_run would break stdout-redirection here.
su ${OCF_RESKEY_user} -s /bin/sh -c "${OCF_RESKEY_binary} --config-file=$OCF_RESKEY_config \
$OCF_RESKEY_additional_parameters"' >> /dev/null 2>&1 & echo $!' > $OCF_RESKEY_pid
# Spin waiting for the server to come up.
# Let the CRM/LRM time us out if required
while true; do
dcorch_engine_worker_monitor
rc=$?
[ $rc -eq $OCF_SUCCESS ] && break
if [ $rc -ne $OCF_NOT_RUNNING ]; then
ocf_log err "DC Orchestrator Engine-worker (dcorch-engine-worker) start failed"
exit $OCF_ERR_GENERIC
fi
sleep 1
done
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) started"
return $OCF_SUCCESS
}
dcorch_engine_worker_confirm_stop() {
local my_bin
local my_processes
my_binary=`which ${OCF_RESKEY_binary}`
my_processes=`pgrep -l -f "^(python|/usr/bin/python|/usr/bin/python3) ${my_binary}([^\w-]|$)"`
if [ -n "${my_processes}" ]
then
ocf_log info "About to SIGKILL the following: ${my_processes}"
pkill -KILL -f "^(python|/usr/bin/python|/usr/bin/python3) ${my_binary}([^\w-]|$)"
fi
}
dcorch_engine_worker_stop() {
local rc
local pid
dcorch_engine_worker_status
rc=$?
if [ $rc -eq $OCF_NOT_RUNNING ]; then
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) already stopped"
dcorch_engine_worker_confirm_stop
return $OCF_SUCCESS
fi
# Try SIGTERM
pid=`cat $OCF_RESKEY_pid`
ocf_run kill -s TERM $pid
rc=$?
if [ $rc -ne 0 ]; then
ocf_log err "DC Orchestrator Engine-worker (dcorch-engine-worker) couldn't be stopped"
dcorch_engine_worker_confirm_stop
exit $OCF_ERR_GENERIC
fi
# stop waiting
shutdown_timeout=15
if [ -n "$OCF_RESKEY_CRM_meta_timeout" ]; then
shutdown_timeout=$((($OCF_RESKEY_CRM_meta_timeout/1000)-5))
fi
count=0
while [ $count -lt $shutdown_timeout ]; do
dcorch_engine_worker_status
rc=$?
if [ $rc -eq $OCF_NOT_RUNNING ]; then
break
fi
count=`expr $count + 1`
sleep 1
ocf_log debug "DC Orchestrator Engine-worker (dcorch-engine-worker) still hasn't stopped yet. Waiting ..."
done
dcorch_engine_worker_status
rc=$?
if [ $rc -ne $OCF_NOT_RUNNING ]; then
# SIGTERM didn't help either, try SIGKILL
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) failed to stop after ${shutdown_timeout}s \
using SIGTERM. Trying SIGKILL ..."
ocf_run kill -s KILL $pid
fi
dcorch_engine_worker_confirm_stop
ocf_log info "DC Orchestrator Engine-worker (dcorch-engine-worker) stopped"
rm -f $OCF_RESKEY_pid
return $OCF_SUCCESS
}
#######################################################################
case "$1" in
meta-data) meta_data
exit $OCF_SUCCESS;;
usage|help) usage
exit $OCF_SUCCESS;;
esac
# Anything except meta-data and help must pass validation
dcorch_engine_worker_validate || exit $?
# What kind of method was invoked?
case "$1" in
start) dcorch_engine_worker_start;;
stop) dcorch_engine_worker_stop;;
status) dcorch_engine_worker_status;;
monitor) dcorch_engine_worker_monitor;;
validate-all) ;;
*) usage
exit $OCF_ERR_UNIMPLEMENTED;;
esac

View File

@ -37,6 +37,7 @@ console_scripts =
dcmanager-state = dcmanager.cmd.state:main
dcorch-api = dcorch.cmd.api:main
dcorch-engine = dcorch.cmd.engine:main
dcorch-engine-worker = dcorch.cmd.engine_worker:main
dcorch-manage = dcorch.cmd.manage:main
dcorch-api-proxy = dcorch.cmd.api_proxy:main
dcdbsync-api = dcdbsync.cmd.api:main
@ -45,7 +46,6 @@ oslo.config.opts =
dcorch.common.config = dcorch.common.config:list_opts
dcorch.common.api.api_config = dcorch.api.api_config:list_opts
dcorch.engine.quota_manager = dcorch.engine.quota_manager:list_opts
dcorch.engine.dcorch_lock = dcorch.engine.dcorch_lock:list_opts
dcmanager.common.config = dcmanager.common.config:list_opts
dcmanager.common.api.api_config = dcmanager.api.api_config:list_opts
dcdbsync.common.config = dcdbsync.common.config:list_opts