Merge "Allocation API: conductor API (without HA and take over)"
This commit is contained in:
commit
680e5b5687
@ -803,3 +803,7 @@ class AllocationDuplicateName(Conflict):
|
|||||||
|
|
||||||
class AllocationAlreadyExists(Conflict):
|
class AllocationAlreadyExists(Conflict):
|
||||||
_msg_fmt = _("An allocation with UUID %(uuid)s already exists.")
|
_msg_fmt = _("An allocation with UUID %(uuid)s already exists.")
|
||||||
|
|
||||||
|
|
||||||
|
class AllocationFailed(IronicException):
|
||||||
|
_msg_fmt = _("Failed to process allocation %(uuid)s: %(error)s.")
|
||||||
|
@ -132,7 +132,7 @@ RELEASE_MAPPING = {
|
|||||||
},
|
},
|
||||||
'master': {
|
'master': {
|
||||||
'api': '1.50',
|
'api': '1.50',
|
||||||
'rpc': '1.47',
|
'rpc': '1.48',
|
||||||
'objects': {
|
'objects': {
|
||||||
'Allocation': ['1.0'],
|
'Allocation': ['1.0'],
|
||||||
'Node': ['1.31', '1.30', '1.29', '1.28'],
|
'Node': ['1.31', '1.30', '1.29', '1.28'],
|
||||||
|
@ -262,6 +262,14 @@ SOFT_REBOOT = 'soft rebooting'
|
|||||||
SOFT_POWER_OFF = 'soft power off'
|
SOFT_POWER_OFF = 'soft power off'
|
||||||
""" Node is in the process of soft power off. """
|
""" Node is in the process of soft power off. """
|
||||||
|
|
||||||
|
###################
|
||||||
|
# Allocation states
|
||||||
|
###################
|
||||||
|
|
||||||
|
ALLOCATING = 'allocating'
|
||||||
|
|
||||||
|
# States ERROR and ACTIVE are reused.
|
||||||
|
|
||||||
|
|
||||||
#####################
|
#####################
|
||||||
# State machine model
|
# State machine model
|
||||||
|
233
ironic/conductor/allocations.py
Normal file
233
ironic/conductor/allocations.py
Normal file
@ -0,0 +1,233 @@
|
|||||||
|
# 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.
|
||||||
|
|
||||||
|
"""Functionality related to allocations."""
|
||||||
|
|
||||||
|
import random
|
||||||
|
|
||||||
|
from ironic_lib import metrics_utils
|
||||||
|
from oslo_config import cfg
|
||||||
|
from oslo_log import log
|
||||||
|
import retrying
|
||||||
|
|
||||||
|
from ironic.common import exception
|
||||||
|
from ironic.common.i18n import _
|
||||||
|
from ironic.common import states
|
||||||
|
from ironic.conductor import task_manager
|
||||||
|
from ironic import objects
|
||||||
|
|
||||||
|
|
||||||
|
CONF = cfg.CONF
|
||||||
|
LOG = log.getLogger(__name__)
|
||||||
|
METRICS = metrics_utils.get_metrics_logger(__name__)
|
||||||
|
|
||||||
|
|
||||||
|
def do_allocate(context, allocation):
|
||||||
|
"""Process the allocation.
|
||||||
|
|
||||||
|
This call runs in a separate thread on a conductor. It finds suitable
|
||||||
|
nodes for the allocation and reserves one of them.
|
||||||
|
|
||||||
|
This call does not raise exceptions since it's designed to work
|
||||||
|
asynchronously.
|
||||||
|
|
||||||
|
:param context: an admin context
|
||||||
|
:param allocation: an allocation object
|
||||||
|
"""
|
||||||
|
try:
|
||||||
|
nodes = _candidate_nodes(context, allocation)
|
||||||
|
_allocate_node(context, allocation, nodes)
|
||||||
|
except exception.AllocationFailed as exc:
|
||||||
|
LOG.error(str(exc))
|
||||||
|
_allocation_failed(allocation, exc)
|
||||||
|
except Exception as exc:
|
||||||
|
LOG.exception("Unexpected exception during processing of "
|
||||||
|
"allocation %s", allocation.uuid)
|
||||||
|
reason = _("Unexpected exception during allocation: %s") % exc
|
||||||
|
_allocation_failed(allocation, reason)
|
||||||
|
|
||||||
|
|
||||||
|
def verify_node_for_deallocation(node, allocation):
|
||||||
|
"""Verify that allocation can be removed for the node.
|
||||||
|
|
||||||
|
:param node: a node object
|
||||||
|
:param allocation: an allocation object associated with the node
|
||||||
|
"""
|
||||||
|
if node.maintenance:
|
||||||
|
# Allocations can always be removed in the maintenance mode.
|
||||||
|
return
|
||||||
|
|
||||||
|
if (node.target_provision_state
|
||||||
|
and node.provision_state not in states.UPDATE_ALLOWED_STATES):
|
||||||
|
msg = (_("Cannot remove allocation %(uuid)s for node %(node)s, "
|
||||||
|
"because the node is in state %(state)s where updates are "
|
||||||
|
"not allowed (and maintenance mode is off)") %
|
||||||
|
{'node': node.uuid, 'uuid': allocation.uuid,
|
||||||
|
'state': node.provision_state})
|
||||||
|
raise exception.InvalidState(msg)
|
||||||
|
|
||||||
|
if node.provision_state == states.ACTIVE:
|
||||||
|
msg = (_("Cannot remove allocation %(uuid)s for node %(node)s, "
|
||||||
|
"because the node is active (and maintenance mode is off)") %
|
||||||
|
{'node': node.uuid, 'uuid': allocation.uuid})
|
||||||
|
raise exception.InvalidState(msg)
|
||||||
|
|
||||||
|
|
||||||
|
def _allocation_failed(allocation, reason):
|
||||||
|
"""Failure handler for the allocation."""
|
||||||
|
try:
|
||||||
|
allocation.state = states.ERROR
|
||||||
|
allocation.last_error = str(reason)
|
||||||
|
allocation.save()
|
||||||
|
except Exception:
|
||||||
|
LOG.exception('Could not save the failed allocation %s',
|
||||||
|
allocation.uuid)
|
||||||
|
|
||||||
|
|
||||||
|
def _traits_match(traits, node):
|
||||||
|
return {t.trait for t in node.traits.objects}.issuperset(traits)
|
||||||
|
|
||||||
|
|
||||||
|
def _candidate_nodes(context, allocation):
|
||||||
|
"""Get a list of candidate nodes for the allocation."""
|
||||||
|
filters = {'resource_class': allocation.resource_class,
|
||||||
|
'provision_state': states.AVAILABLE,
|
||||||
|
'associated': False,
|
||||||
|
'with_power_state': True,
|
||||||
|
'maintenance': False}
|
||||||
|
if allocation.candidate_nodes:
|
||||||
|
# NOTE(dtantsur): we assume that candidate_nodes were converted to
|
||||||
|
# UUIDs on the API level.
|
||||||
|
filters['uuid_in'] = allocation.candidate_nodes
|
||||||
|
|
||||||
|
nodes = objects.Node.list(context, filters=filters)
|
||||||
|
|
||||||
|
if not nodes:
|
||||||
|
if allocation.candidate_nodes:
|
||||||
|
error = _("none of the requested nodes are available and match "
|
||||||
|
"the resource class %s") % allocation.resource_class
|
||||||
|
else:
|
||||||
|
error = _("no available nodes match the resource class %s") % (
|
||||||
|
allocation.resource_class)
|
||||||
|
raise exception.AllocationFailed(uuid=allocation.uuid, error=error)
|
||||||
|
|
||||||
|
# TODO(dtantsur): database-level filtering?
|
||||||
|
if allocation.traits:
|
||||||
|
traits = set(allocation.traits)
|
||||||
|
nodes = [n for n in nodes if _traits_match(traits, n)]
|
||||||
|
if not nodes:
|
||||||
|
error = (_("no suitable nodes have the requested traits %s") %
|
||||||
|
', '.join(traits))
|
||||||
|
raise exception.AllocationFailed(uuid=allocation.uuid, error=error)
|
||||||
|
|
||||||
|
# NOTE(dtantsur): make sure that parallel allocations do not try the nodes
|
||||||
|
# in the same order.
|
||||||
|
random.shuffle(nodes)
|
||||||
|
|
||||||
|
LOG.debug('%(count)d nodes are candidates for allocation %(uuid)s',
|
||||||
|
{'count': len(nodes), 'uuid': allocation.uuid})
|
||||||
|
return nodes
|
||||||
|
|
||||||
|
|
||||||
|
def _verify_node(node, allocation):
|
||||||
|
"""Check that the node still satisfiest the request."""
|
||||||
|
if node.maintenance:
|
||||||
|
LOG.debug('Node %s is now in maintenance, skipping',
|
||||||
|
node.uuid)
|
||||||
|
return False
|
||||||
|
|
||||||
|
if node.instance_uuid:
|
||||||
|
LOG.debug('Node %(node)s is already associated with instance '
|
||||||
|
'%(inst)s, skipping',
|
||||||
|
{'node': node.uuid, 'inst': node.instance_uuid})
|
||||||
|
return False
|
||||||
|
|
||||||
|
if node.provision_state != states.AVAILABLE:
|
||||||
|
LOG.debug('Node %s is no longer available, skipping',
|
||||||
|
node.uuid)
|
||||||
|
return False
|
||||||
|
|
||||||
|
if node.resource_class != allocation.resource_class:
|
||||||
|
LOG.debug('Resource class of node %(node)s no longer '
|
||||||
|
'matches requested resource class %(rsc)s for '
|
||||||
|
'allocation %(uuid)s, skipping',
|
||||||
|
{'node': node.uuid,
|
||||||
|
'rsc': allocation.resource_class,
|
||||||
|
'uuid': allocation.uuid})
|
||||||
|
return False
|
||||||
|
|
||||||
|
if allocation.traits and not _traits_match(set(allocation.traits), node):
|
||||||
|
LOG.debug('List of traits of node %(node)s no longer '
|
||||||
|
'matches requested traits %(traits)s for '
|
||||||
|
'allocation %(uuid)s, skipping',
|
||||||
|
{'node': node.uuid,
|
||||||
|
'traits': allocation.traits,
|
||||||
|
'uuid': allocation.uuid})
|
||||||
|
return False
|
||||||
|
|
||||||
|
return True
|
||||||
|
|
||||||
|
|
||||||
|
# NOTE(dtantsur): instead of trying to allocate each node
|
||||||
|
# node_locked_retry_attempt times, we try to allocate *any* node the same
|
||||||
|
# number of times. This avoids getting stuck on a node reserved e.g. for power
|
||||||
|
# sync periodic task.
|
||||||
|
@retrying.retry(
|
||||||
|
retry_on_exception=lambda e: isinstance(e, exception.AllocationFailed),
|
||||||
|
stop_max_attempt_number=CONF.conductor.node_locked_retry_attempts,
|
||||||
|
wait_fixed=CONF.conductor.node_locked_retry_interval * 1000)
|
||||||
|
def _allocate_node(context, allocation, nodes):
|
||||||
|
"""Go through the list of nodes and try to allocate one of them."""
|
||||||
|
retry_nodes = []
|
||||||
|
for node in nodes:
|
||||||
|
try:
|
||||||
|
# NOTE(dtantsur): retries are done for all nodes above, so disable
|
||||||
|
# per-node retry. Also disable loading the driver, since the
|
||||||
|
# current conductor may not have the requried hardware type or
|
||||||
|
# interfaces (it's picked at random).
|
||||||
|
with task_manager.acquire(context, node.uuid, shared=False,
|
||||||
|
retry=False, load_driver=False,
|
||||||
|
purpose='allocating') as task:
|
||||||
|
# NOTE(dtantsur): double-check the node details, since they
|
||||||
|
# could have changed before we acquired the lock.
|
||||||
|
if not _verify_node(task.node, allocation):
|
||||||
|
continue
|
||||||
|
|
||||||
|
allocation.node_id = task.node.id
|
||||||
|
allocation.state = states.ACTIVE
|
||||||
|
# NOTE(dtantsur): the node.instance_uuid and allocation_id are
|
||||||
|
# updated inside of the save() call within the same
|
||||||
|
# transaction to avoid races. NodeAssociated can be raised if
|
||||||
|
# another process allocates this node first.
|
||||||
|
allocation.save()
|
||||||
|
LOG.info('Node %(node)s has been successfully reserved for '
|
||||||
|
'allocation %(uuid)s',
|
||||||
|
{'node': node.uuid, 'uuid': allocation.uuid})
|
||||||
|
return allocation
|
||||||
|
except exception.NodeLocked:
|
||||||
|
LOG.debug('Node %s is currently locked, moving to the next one',
|
||||||
|
node.uuid)
|
||||||
|
retry_nodes.append(node)
|
||||||
|
except exception.NodeAssociated:
|
||||||
|
LOG.debug('Node %s is already associated, moving to the next one',
|
||||||
|
node.uuid)
|
||||||
|
|
||||||
|
# NOTE(dtantsur): rewrite the passed list to only contain the nodes that
|
||||||
|
# are worth retrying. Do not include nodes that are no longer suitable.
|
||||||
|
nodes[:] = retry_nodes
|
||||||
|
|
||||||
|
if nodes:
|
||||||
|
error = _('could not reserve any of %d suitable nodes') % len(nodes)
|
||||||
|
else:
|
||||||
|
error = _('all nodes were filtered out during reservation')
|
||||||
|
|
||||||
|
raise exception.AllocationFailed(uuid=allocation.uuid, error=error)
|
@ -66,6 +66,7 @@ from ironic.common import network
|
|||||||
from ironic.common import release_mappings as versions
|
from ironic.common import release_mappings as versions
|
||||||
from ironic.common import states
|
from ironic.common import states
|
||||||
from ironic.common import swift
|
from ironic.common import swift
|
||||||
|
from ironic.conductor import allocations
|
||||||
from ironic.conductor import base_manager
|
from ironic.conductor import base_manager
|
||||||
from ironic.conductor import notification_utils as notify_utils
|
from ironic.conductor import notification_utils as notify_utils
|
||||||
from ironic.conductor import task_manager
|
from ironic.conductor import task_manager
|
||||||
@ -100,7 +101,7 @@ class ConductorManager(base_manager.BaseConductorManager):
|
|||||||
# NOTE(rloo): This must be in sync with rpcapi.ConductorAPI's.
|
# NOTE(rloo): This must be in sync with rpcapi.ConductorAPI's.
|
||||||
# NOTE(pas-ha): This also must be in sync with
|
# NOTE(pas-ha): This also must be in sync with
|
||||||
# ironic.common.release_mappings.RELEASE_MAPPING['master']
|
# ironic.common.release_mappings.RELEASE_MAPPING['master']
|
||||||
RPC_API_VERSION = '1.47'
|
RPC_API_VERSION = '1.48'
|
||||||
|
|
||||||
target = messaging.Target(version=RPC_API_VERSION)
|
target = messaging.Target(version=RPC_API_VERSION)
|
||||||
|
|
||||||
@ -244,6 +245,25 @@ class ConductorManager(base_manager.BaseConductorManager):
|
|||||||
'allowed': ', '.join(allowed_update_states),
|
'allowed': ', '.join(allowed_update_states),
|
||||||
'field': 'resource_class'})
|
'field': 'resource_class'})
|
||||||
|
|
||||||
|
if ('instance_uuid' in delta and task.node.allocation_id
|
||||||
|
and not node_obj.instance_uuid):
|
||||||
|
if (not task.node.maintenance and task.node.provision_state
|
||||||
|
not in allowed_update_states):
|
||||||
|
action = _("Node %(node)s with an allocation can not have "
|
||||||
|
"instance_uuid removed unless it is in one of "
|
||||||
|
"allowed (%(allowed)s) states or in "
|
||||||
|
"maintenance mode.")
|
||||||
|
raise exception.InvalidState(
|
||||||
|
action % {'node': node_obj.uuid,
|
||||||
|
'allowed': ', '.join(allowed_update_states)})
|
||||||
|
|
||||||
|
try:
|
||||||
|
allocation = objects.Allocation.get_by_id(
|
||||||
|
context, task.node.allocation_id)
|
||||||
|
allocation.destroy()
|
||||||
|
except exception.AllocationNotFound:
|
||||||
|
pass
|
||||||
|
|
||||||
node_obj.save()
|
node_obj.save()
|
||||||
|
|
||||||
return node_obj
|
return node_obj
|
||||||
@ -1023,6 +1043,13 @@ class ConductorManager(base_manager.BaseConductorManager):
|
|||||||
node.driver_internal_info = driver_internal_info
|
node.driver_internal_info = driver_internal_info
|
||||||
network.remove_vifs_from_node(task)
|
network.remove_vifs_from_node(task)
|
||||||
node.save()
|
node.save()
|
||||||
|
if node.allocation_id:
|
||||||
|
allocation = objects.Allocation.get_by_id(task.context,
|
||||||
|
node.allocation_id)
|
||||||
|
allocation.destroy()
|
||||||
|
# The destroy() call above removes allocation_id and
|
||||||
|
# instance_uuid, refresh the node to get these changes.
|
||||||
|
node.refresh()
|
||||||
|
|
||||||
# Begin cleaning
|
# Begin cleaning
|
||||||
task.process_event('clean')
|
task.process_event('clean')
|
||||||
@ -3410,6 +3437,54 @@ class ConductorManager(base_manager.BaseConductorManager):
|
|||||||
objects.Trait.destroy(context, node_id=node_id,
|
objects.Trait.destroy(context, node_id=node_id,
|
||||||
trait=trait)
|
trait=trait)
|
||||||
|
|
||||||
|
@METRICS.timer('ConductorManager.create_allocation')
|
||||||
|
@messaging.expected_exceptions(exception.InvalidParameterValue)
|
||||||
|
def create_allocation(self, context, allocation):
|
||||||
|
"""Create an allocation in database.
|
||||||
|
|
||||||
|
:param context: an admin context
|
||||||
|
:param allocation: a created (but not saved to the database)
|
||||||
|
allocation object.
|
||||||
|
:returns: created allocation object.
|
||||||
|
:raises: InvalidParameterValue if some fields fail validation.
|
||||||
|
"""
|
||||||
|
LOG.debug("RPC create_allocation called for allocation %s.",
|
||||||
|
allocation.uuid)
|
||||||
|
allocation.conductor_affinity = self.conductor.id
|
||||||
|
allocation.create()
|
||||||
|
|
||||||
|
# Spawn an asynchronous worker to process the allocation. Copy it to
|
||||||
|
# avoid data races.
|
||||||
|
self._spawn_worker(allocations.do_allocate,
|
||||||
|
context, allocation.obj_clone())
|
||||||
|
|
||||||
|
# Return the unfinished allocation
|
||||||
|
return allocation
|
||||||
|
|
||||||
|
@METRICS.timer('ConductorManager.destroy_allocation')
|
||||||
|
@messaging.expected_exceptions(exception.InvalidState)
|
||||||
|
def destroy_allocation(self, context, allocation):
|
||||||
|
"""Delete an allocation.
|
||||||
|
|
||||||
|
:param context: request context.
|
||||||
|
:param allocation: allocation object.
|
||||||
|
:raises: InvalidState if the associated node is in the wrong provision
|
||||||
|
state to perform deallocation.
|
||||||
|
"""
|
||||||
|
if allocation.node_id:
|
||||||
|
with task_manager.acquire(context, allocation.node_id,
|
||||||
|
purpose='allocation deletion',
|
||||||
|
shared=False) as task:
|
||||||
|
allocations.verify_node_for_deallocation(task.node, allocation)
|
||||||
|
# NOTE(dtantsur): remove the allocation while still holding
|
||||||
|
# the node lock to avoid races.
|
||||||
|
allocation.destroy()
|
||||||
|
else:
|
||||||
|
allocation.destroy()
|
||||||
|
|
||||||
|
LOG.info('Successfully deleted allocation %(uuid)s',
|
||||||
|
allocation.uuid)
|
||||||
|
|
||||||
|
|
||||||
@METRICS.timer('get_vendor_passthru_metadata')
|
@METRICS.timer('get_vendor_passthru_metadata')
|
||||||
def get_vendor_passthru_metadata(route_dict):
|
def get_vendor_passthru_metadata(route_dict):
|
||||||
|
@ -97,13 +97,14 @@ class ConductorAPI(object):
|
|||||||
| 1.45 - Added continue_node_deploy
|
| 1.45 - Added continue_node_deploy
|
||||||
| 1.46 - Added reset_interfaces to update_node
|
| 1.46 - Added reset_interfaces to update_node
|
||||||
| 1.47 - Added support for conductor groups
|
| 1.47 - Added support for conductor groups
|
||||||
|
| 1.48 - Added allocation API
|
||||||
|
|
||||||
"""
|
"""
|
||||||
|
|
||||||
# NOTE(rloo): This must be in sync with manager.ConductorManager's.
|
# NOTE(rloo): This must be in sync with manager.ConductorManager's.
|
||||||
# NOTE(pas-ha): This also must be in sync with
|
# NOTE(pas-ha): This also must be in sync with
|
||||||
# ironic.common.release_mappings.RELEASE_MAPPING['master']
|
# ironic.common.release_mappings.RELEASE_MAPPING['master']
|
||||||
RPC_API_VERSION = '1.47'
|
RPC_API_VERSION = '1.48'
|
||||||
|
|
||||||
def __init__(self, topic=None):
|
def __init__(self, topic=None):
|
||||||
super(ConductorAPI, self).__init__()
|
super(ConductorAPI, self).__init__()
|
||||||
@ -1105,3 +1106,25 @@ class ConductorAPI(object):
|
|||||||
cctxt = self.client.prepare(topic=topic or self.topic, version='1.44')
|
cctxt = self.client.prepare(topic=topic or self.topic, version='1.44')
|
||||||
return cctxt.call(context, 'remove_node_traits', node_id=node_id,
|
return cctxt.call(context, 'remove_node_traits', node_id=node_id,
|
||||||
traits=traits)
|
traits=traits)
|
||||||
|
|
||||||
|
def create_allocation(self, context, allocation, topic=None):
|
||||||
|
"""Create an allocation.
|
||||||
|
|
||||||
|
:param context: request context.
|
||||||
|
:param allocation: an allocation object.
|
||||||
|
:param topic: RPC topic. Defaults to self.topic.
|
||||||
|
"""
|
||||||
|
cctxt = self.client.prepare(topic=topic or self.topic, version='1.48')
|
||||||
|
return cctxt.call(context, 'create_allocation', allocation=allocation)
|
||||||
|
|
||||||
|
def destroy_allocation(self, context, allocation, topic=None):
|
||||||
|
"""Delete an allocation.
|
||||||
|
|
||||||
|
:param context: request context.
|
||||||
|
:param allocation: an allocation object.
|
||||||
|
:param topic: RPC topic. Defaults to self.topic.
|
||||||
|
:raises: InvalidState if the associated node is in the wrong provision
|
||||||
|
state to perform deallocation.
|
||||||
|
"""
|
||||||
|
cctxt = self.client.prepare(topic=topic or self.topic, version='1.48')
|
||||||
|
return cctxt.call(context, 'destroy_allocation', allocation=allocation)
|
||||||
|
@ -1602,6 +1602,8 @@ class Connection(api.Connection):
|
|||||||
"""
|
"""
|
||||||
if not values.get('uuid'):
|
if not values.get('uuid'):
|
||||||
values['uuid'] = uuidutils.generate_uuid()
|
values['uuid'] = uuidutils.generate_uuid()
|
||||||
|
if not values.get('state'):
|
||||||
|
values['state'] = states.ALLOCATING
|
||||||
|
|
||||||
allocation = models.Allocation()
|
allocation = models.Allocation()
|
||||||
allocation.update(values)
|
allocation.update(values)
|
||||||
|
362
ironic/tests/unit/conductor/test_allocations.py
Normal file
362
ironic/tests/unit/conductor/test_allocations.py
Normal file
@ -0,0 +1,362 @@
|
|||||||
|
# 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.
|
||||||
|
|
||||||
|
"""Unit tests for functionality related to allocations."""
|
||||||
|
|
||||||
|
import mock
|
||||||
|
import oslo_messaging as messaging
|
||||||
|
from oslo_utils import uuidutils
|
||||||
|
|
||||||
|
from ironic.common import exception
|
||||||
|
from ironic.conductor import allocations
|
||||||
|
from ironic.conductor import manager
|
||||||
|
from ironic.conductor import task_manager
|
||||||
|
from ironic import objects
|
||||||
|
from ironic.tests.unit.conductor import mgr_utils
|
||||||
|
from ironic.tests.unit.db import base as db_base
|
||||||
|
from ironic.tests.unit.db import utils as db_utils
|
||||||
|
from ironic.tests.unit.objects import utils as obj_utils
|
||||||
|
|
||||||
|
|
||||||
|
@mgr_utils.mock_record_keepalive
|
||||||
|
class AllocationTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
||||||
|
@mock.patch.object(manager.ConductorManager, '_spawn_worker',
|
||||||
|
autospec=True)
|
||||||
|
def test_create_allocation(self, mock_spawn):
|
||||||
|
# In this test we mock spawn_worker, so that the actual processing does
|
||||||
|
# not happen, and the allocation stays in the "allocating" state.
|
||||||
|
allocation = obj_utils.get_test_allocation(self.context,
|
||||||
|
extra={'test': 'one'})
|
||||||
|
self._start_service()
|
||||||
|
mock_spawn.reset_mock()
|
||||||
|
|
||||||
|
res = self.service.create_allocation(self.context, allocation)
|
||||||
|
|
||||||
|
self.assertEqual({'test': 'one'}, res['extra'])
|
||||||
|
self.assertEqual('allocating', res['state'])
|
||||||
|
self.assertIsNotNone(res['uuid'])
|
||||||
|
self.assertEqual(self.service.conductor.id, res['conductor_affinity'])
|
||||||
|
res = objects.Allocation.get_by_uuid(self.context, allocation['uuid'])
|
||||||
|
self.assertEqual({'test': 'one'}, res['extra'])
|
||||||
|
self.assertEqual('allocating', res['state'])
|
||||||
|
self.assertIsNotNone(res['uuid'])
|
||||||
|
self.assertEqual(self.service.conductor.id, res['conductor_affinity'])
|
||||||
|
|
||||||
|
mock_spawn.assert_called_once_with(self.service,
|
||||||
|
allocations.do_allocate,
|
||||||
|
self.context, mock.ANY)
|
||||||
|
|
||||||
|
def test_destroy_allocation_without_node(self):
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context)
|
||||||
|
self.service.destroy_allocation(self.context, allocation)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_uuid,
|
||||||
|
self.context, allocation['uuid'])
|
||||||
|
|
||||||
|
def test_destroy_allocation_with_node(self):
|
||||||
|
node = obj_utils.create_test_node(self.context)
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
node_id=node['id'])
|
||||||
|
node.instance_uuid = allocation['uuid']
|
||||||
|
node.allocation_id = allocation['id']
|
||||||
|
node.save()
|
||||||
|
|
||||||
|
self.service.destroy_allocation(self.context, allocation)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_uuid,
|
||||||
|
self.context, allocation['uuid'])
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertIsNone(node['instance_uuid'])
|
||||||
|
self.assertIsNone(node['allocation_id'])
|
||||||
|
|
||||||
|
def test_destroy_allocation_with_active_node(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
provision_state='active')
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
node_id=node['id'])
|
||||||
|
node.instance_uuid = allocation['uuid']
|
||||||
|
node.allocation_id = allocation['id']
|
||||||
|
node.save()
|
||||||
|
|
||||||
|
exc = self.assertRaises(messaging.rpc.ExpectedException,
|
||||||
|
self.service.destroy_allocation,
|
||||||
|
self.context, allocation)
|
||||||
|
# Compare true exception hidden by @messaging.expected_exceptions
|
||||||
|
self.assertEqual(exception.InvalidState, exc.exc_info[0])
|
||||||
|
|
||||||
|
objects.Allocation.get_by_uuid(self.context, allocation['uuid'])
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertEqual(allocation['uuid'], node['instance_uuid'])
|
||||||
|
self.assertEqual(allocation['id'], node['allocation_id'])
|
||||||
|
|
||||||
|
def test_destroy_allocation_with_transient_node(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
target_provision_state='active',
|
||||||
|
provision_state='deploying')
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
node_id=node['id'])
|
||||||
|
node.instance_uuid = allocation['uuid']
|
||||||
|
node.allocation_id = allocation['id']
|
||||||
|
node.save()
|
||||||
|
|
||||||
|
exc = self.assertRaises(messaging.rpc.ExpectedException,
|
||||||
|
self.service.destroy_allocation,
|
||||||
|
self.context, allocation)
|
||||||
|
# Compare true exception hidden by @messaging.expected_exceptions
|
||||||
|
self.assertEqual(exception.InvalidState, exc.exc_info[0])
|
||||||
|
|
||||||
|
objects.Allocation.get_by_uuid(self.context, allocation['uuid'])
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertEqual(allocation['uuid'], node['instance_uuid'])
|
||||||
|
self.assertEqual(allocation['id'], node['allocation_id'])
|
||||||
|
|
||||||
|
def test_destroy_allocation_with_node_in_maintenance(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
provision_state='active',
|
||||||
|
maintenance=True)
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
node_id=node['id'])
|
||||||
|
node.instance_uuid = allocation['uuid']
|
||||||
|
node.allocation_id = allocation['id']
|
||||||
|
node.save()
|
||||||
|
|
||||||
|
self.service.destroy_allocation(self.context, allocation)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_uuid,
|
||||||
|
self.context, allocation['uuid'])
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertIsNone(node['instance_uuid'])
|
||||||
|
self.assertIsNone(node['allocation_id'])
|
||||||
|
|
||||||
|
|
||||||
|
@mock.patch('time.sleep', lambda _: None)
|
||||||
|
class DoAllocateTestCase(db_base.DbTestCase):
|
||||||
|
def test_success(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-large',
|
||||||
|
provision_state='available')
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
resource_class='x-large')
|
||||||
|
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
|
||||||
|
allocation = objects.Allocation.get_by_uuid(self.context,
|
||||||
|
allocation['uuid'])
|
||||||
|
self.assertIsNone(allocation['last_error'])
|
||||||
|
self.assertEqual('active', allocation['state'])
|
||||||
|
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertEqual(allocation['uuid'], node['instance_uuid'])
|
||||||
|
self.assertEqual(allocation['id'], node['allocation_id'])
|
||||||
|
|
||||||
|
def test_with_traits(self):
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-large',
|
||||||
|
provision_state='available')
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-large',
|
||||||
|
provision_state='available')
|
||||||
|
db_utils.create_test_node_traits(['tr1', 'tr2'], node_id=node.id)
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
resource_class='x-large',
|
||||||
|
traits=['tr2'])
|
||||||
|
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
|
||||||
|
allocation = objects.Allocation.get_by_uuid(self.context,
|
||||||
|
allocation['uuid'])
|
||||||
|
self.assertIsNone(allocation['last_error'])
|
||||||
|
self.assertEqual('active', allocation['state'])
|
||||||
|
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertEqual(allocation['uuid'], node['instance_uuid'])
|
||||||
|
self.assertEqual(allocation['id'], node['allocation_id'])
|
||||||
|
self.assertEqual(allocation['traits'], ['tr2'])
|
||||||
|
|
||||||
|
def test_with_candidates(self):
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-large',
|
||||||
|
provision_state='available')
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-large',
|
||||||
|
provision_state='available')
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(
|
||||||
|
self.context, resource_class='x-large',
|
||||||
|
candidate_nodes=[node['uuid']])
|
||||||
|
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
|
||||||
|
allocation = objects.Allocation.get_by_uuid(self.context,
|
||||||
|
allocation['uuid'])
|
||||||
|
self.assertIsNone(allocation['last_error'])
|
||||||
|
self.assertEqual('active', allocation['state'])
|
||||||
|
|
||||||
|
node = objects.Node.get_by_uuid(self.context, node['uuid'])
|
||||||
|
self.assertEqual(allocation['uuid'], node['instance_uuid'])
|
||||||
|
self.assertEqual(allocation['id'], node['allocation_id'])
|
||||||
|
self.assertEqual([node['uuid']], allocation['candidate_nodes'])
|
||||||
|
|
||||||
|
@mock.patch.object(task_manager, 'acquire', autospec=True,
|
||||||
|
side_effect=task_manager.acquire)
|
||||||
|
def test_nodes_filtered_out(self, mock_acquire):
|
||||||
|
# Resource class does not match
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-small',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available')
|
||||||
|
# Provision state is not available
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='manageable')
|
||||||
|
# Power state is undefined
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state=None,
|
||||||
|
provision_state='available')
|
||||||
|
# Maintenance mode is on
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
maintenance=True,
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available')
|
||||||
|
# Already associated
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
instance_uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available')
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
resource_class='x-large')
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
self.assertIn('no available nodes', allocation['last_error'])
|
||||||
|
self.assertIn('x-large', allocation['last_error'])
|
||||||
|
self.assertEqual('error', allocation['state'])
|
||||||
|
|
||||||
|
# All nodes are filtered out on the database level.
|
||||||
|
self.assertFalse(mock_acquire.called)
|
||||||
|
|
||||||
|
@mock.patch.object(task_manager, 'acquire', autospec=True,
|
||||||
|
side_effect=task_manager.acquire)
|
||||||
|
def test_nodes_locked(self, mock_acquire):
|
||||||
|
self.config(node_locked_retry_attempts=2, group='conductor')
|
||||||
|
node1 = obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
maintenance=False,
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available',
|
||||||
|
reservation='example.com')
|
||||||
|
node2 = obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available',
|
||||||
|
reservation='example.com')
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
resource_class='x-large')
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
self.assertIn('could not reserve any of 2', allocation['last_error'])
|
||||||
|
self.assertEqual('error', allocation['state'])
|
||||||
|
|
||||||
|
self.assertEqual(6, mock_acquire.call_count)
|
||||||
|
# NOTE(dtantsur): node are tried in random order by design, so we
|
||||||
|
# cannot directly use assert_has_calls. Check that all nodes are tried
|
||||||
|
# before going into retries (rather than each tried 3 times in a row).
|
||||||
|
nodes = [call[0][1] for call in mock_acquire.call_args_list]
|
||||||
|
for offset in (0, 2, 4):
|
||||||
|
self.assertEqual(set(nodes[offset:offset + 2]),
|
||||||
|
{node1.uuid, node2.uuid})
|
||||||
|
|
||||||
|
@mock.patch.object(task_manager, 'acquire', autospec=True)
|
||||||
|
def test_nodes_changed_after_lock(self, mock_acquire):
|
||||||
|
nodes = [obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available')
|
||||||
|
for _ in range(5)]
|
||||||
|
for node in nodes:
|
||||||
|
db_utils.create_test_node_trait(trait='tr1', node_id=node.id)
|
||||||
|
|
||||||
|
# Modify nodes in-memory so that they no longer match the allocation:
|
||||||
|
|
||||||
|
# Resource class does not match
|
||||||
|
nodes[0].resource_class = 'x-small'
|
||||||
|
# Provision state is not available
|
||||||
|
nodes[1].provision_state = 'deploying'
|
||||||
|
# Maintenance mode is on
|
||||||
|
nodes[2].maintenance = True
|
||||||
|
# Already associated
|
||||||
|
nodes[3].instance_uuid = uuidutils.generate_uuid()
|
||||||
|
# Traits changed
|
||||||
|
nodes[4].traits.objects[:] = []
|
||||||
|
|
||||||
|
mock_acquire.side_effect = [
|
||||||
|
mock.MagicMock(**{'__enter__.return_value.node': node})
|
||||||
|
for node in nodes
|
||||||
|
]
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(self.context,
|
||||||
|
resource_class='x-large',
|
||||||
|
traits=['tr1'])
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
self.assertIn('all nodes were filtered out', allocation['last_error'])
|
||||||
|
self.assertEqual('error', allocation['state'])
|
||||||
|
|
||||||
|
# No retries for these failures.
|
||||||
|
self.assertEqual(5, mock_acquire.call_count)
|
||||||
|
|
||||||
|
@mock.patch.object(task_manager, 'acquire', autospec=True,
|
||||||
|
side_effect=task_manager.acquire)
|
||||||
|
def test_nodes_candidates_do_not_match(self, mock_acquire):
|
||||||
|
obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
resource_class='x-large',
|
||||||
|
power_state='power off',
|
||||||
|
provision_state='available')
|
||||||
|
# Resource class does not match
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
uuid=uuidutils.generate_uuid(),
|
||||||
|
power_state='power on',
|
||||||
|
resource_class='x-small',
|
||||||
|
provision_state='available')
|
||||||
|
|
||||||
|
allocation = obj_utils.create_test_allocation(
|
||||||
|
self.context, resource_class='x-large',
|
||||||
|
candidate_nodes=[node['uuid']])
|
||||||
|
|
||||||
|
allocations.do_allocate(self.context, allocation)
|
||||||
|
self.assertIn('none of the requested nodes', allocation['last_error'])
|
||||||
|
self.assertIn('x-large', allocation['last_error'])
|
||||||
|
self.assertEqual('error', allocation['state'])
|
||||||
|
|
||||||
|
# All nodes are filtered out on the database level.
|
||||||
|
self.assertFalse(mock_acquire.called)
|
@ -811,6 +811,74 @@ class UpdateNodeTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
self.assertEqual(new_hardware, node.driver)
|
self.assertEqual(new_hardware, node.driver)
|
||||||
self.assertEqual(new_interface, node.boot_interface)
|
self.assertEqual(new_interface, node.boot_interface)
|
||||||
|
|
||||||
|
def test_update_node_deleting_allocation(self):
|
||||||
|
node = obj_utils.create_test_node(self.context)
|
||||||
|
alloc = obj_utils.create_test_allocation(self.context)
|
||||||
|
# Establish cross-linking between the node and the allocation
|
||||||
|
alloc.node_id = node.id
|
||||||
|
alloc.save()
|
||||||
|
node.refresh()
|
||||||
|
self.assertEqual(alloc.id, node.allocation_id)
|
||||||
|
self.assertEqual(alloc.uuid, node.instance_uuid)
|
||||||
|
|
||||||
|
node.instance_uuid = None
|
||||||
|
res = self.service.update_node(self.context, node)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_id,
|
||||||
|
self.context, alloc.id)
|
||||||
|
self.assertIsNone(res['instance_uuid'])
|
||||||
|
self.assertIsNone(res['allocation_id'])
|
||||||
|
|
||||||
|
node.refresh()
|
||||||
|
self.assertIsNone(node.instance_uuid)
|
||||||
|
self.assertIsNone(node.allocation_id)
|
||||||
|
|
||||||
|
def test_update_node_deleting_allocation_forbidden(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
provision_state='active',
|
||||||
|
maintenance=False)
|
||||||
|
alloc = obj_utils.create_test_allocation(self.context)
|
||||||
|
# Establish cross-linking between the node and the allocation
|
||||||
|
alloc.node_id = node.id
|
||||||
|
alloc.save()
|
||||||
|
node.refresh()
|
||||||
|
self.assertEqual(alloc.id, node.allocation_id)
|
||||||
|
self.assertEqual(alloc.uuid, node.instance_uuid)
|
||||||
|
|
||||||
|
node.instance_uuid = None
|
||||||
|
exc = self.assertRaises(messaging.rpc.ExpectedException,
|
||||||
|
self.service.update_node,
|
||||||
|
self.context, node)
|
||||||
|
self.assertEqual(exception.InvalidState, exc.exc_info[0])
|
||||||
|
|
||||||
|
node.refresh()
|
||||||
|
self.assertEqual(alloc.id, node.allocation_id)
|
||||||
|
self.assertEqual(alloc.uuid, node.instance_uuid)
|
||||||
|
|
||||||
|
def test_update_node_deleting_allocation_in_maintenance(self):
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
provision_state='active',
|
||||||
|
maintenance=True)
|
||||||
|
alloc = obj_utils.create_test_allocation(self.context)
|
||||||
|
# Establish cross-linking between the node and the allocation
|
||||||
|
alloc.node_id = node.id
|
||||||
|
alloc.save()
|
||||||
|
node.refresh()
|
||||||
|
self.assertEqual(alloc.id, node.allocation_id)
|
||||||
|
self.assertEqual(alloc.uuid, node.instance_uuid)
|
||||||
|
|
||||||
|
node.instance_uuid = None
|
||||||
|
res = self.service.update_node(self.context, node)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_id,
|
||||||
|
self.context, alloc.id)
|
||||||
|
self.assertIsNone(res['instance_uuid'])
|
||||||
|
self.assertIsNone(res['allocation_id'])
|
||||||
|
|
||||||
|
node.refresh()
|
||||||
|
self.assertIsNone(node.instance_uuid)
|
||||||
|
self.assertIsNone(node.allocation_id)
|
||||||
|
|
||||||
|
|
||||||
@mgr_utils.mock_record_keepalive
|
@mgr_utils.mock_record_keepalive
|
||||||
class VendorPassthruTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
class VendorPassthruTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
||||||
@ -2736,13 +2804,15 @@ class DoNodeTearDownTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
@mock.patch('ironic.drivers.modules.fake.FakeDeploy.tear_down')
|
@mock.patch('ironic.drivers.modules.fake.FakeDeploy.tear_down')
|
||||||
def _test__do_node_tear_down_ok(self, mock_tear_down, mock_clean,
|
def _test__do_node_tear_down_ok(self, mock_tear_down, mock_clean,
|
||||||
mock_unbind, mock_console,
|
mock_unbind, mock_console,
|
||||||
enabled_console=False):
|
enabled_console=False,
|
||||||
|
with_allocation=False):
|
||||||
# test when driver.deploy.tear_down succeeds
|
# test when driver.deploy.tear_down succeeds
|
||||||
node = obj_utils.create_test_node(
|
node = obj_utils.create_test_node(
|
||||||
self.context, driver='fake-hardware',
|
self.context, driver='fake-hardware',
|
||||||
provision_state=states.DELETING,
|
provision_state=states.DELETING,
|
||||||
target_provision_state=states.AVAILABLE,
|
target_provision_state=states.AVAILABLE,
|
||||||
instance_uuid=uuidutils.generate_uuid(),
|
instance_uuid=(uuidutils.generate_uuid()
|
||||||
|
if not with_allocation else None),
|
||||||
instance_info={'foo': 'bar'},
|
instance_info={'foo': 'bar'},
|
||||||
console_enabled=enabled_console,
|
console_enabled=enabled_console,
|
||||||
driver_internal_info={'is_whole_disk_image': False,
|
driver_internal_info={'is_whole_disk_image': False,
|
||||||
@ -2752,6 +2822,12 @@ class DoNodeTearDownTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
port = obj_utils.create_test_port(
|
port = obj_utils.create_test_port(
|
||||||
self.context, node_id=node.id,
|
self.context, node_id=node.id,
|
||||||
internal_info={'tenant_vif_port_id': 'foo'})
|
internal_info={'tenant_vif_port_id': 'foo'})
|
||||||
|
if with_allocation:
|
||||||
|
alloc = obj_utils.create_test_allocation(self.context)
|
||||||
|
# Establish cross-linking between the node and the allocation
|
||||||
|
alloc.node_id = node.id
|
||||||
|
alloc.save()
|
||||||
|
node.refresh()
|
||||||
|
|
||||||
task = task_manager.TaskManager(self.context, node.uuid)
|
task = task_manager.TaskManager(self.context, node.uuid)
|
||||||
self._start_service()
|
self._start_service()
|
||||||
@ -2763,6 +2839,7 @@ class DoNodeTearDownTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
self.assertEqual(states.AVAILABLE, node.target_provision_state)
|
self.assertEqual(states.AVAILABLE, node.target_provision_state)
|
||||||
self.assertIsNone(node.last_error)
|
self.assertIsNone(node.last_error)
|
||||||
self.assertIsNone(node.instance_uuid)
|
self.assertIsNone(node.instance_uuid)
|
||||||
|
self.assertIsNone(node.allocation_id)
|
||||||
self.assertEqual({}, node.instance_info)
|
self.assertEqual({}, node.instance_info)
|
||||||
self.assertNotIn('instance', node.driver_internal_info)
|
self.assertNotIn('instance', node.driver_internal_info)
|
||||||
self.assertNotIn('clean_steps', node.driver_internal_info)
|
self.assertNotIn('clean_steps', node.driver_internal_info)
|
||||||
@ -2776,6 +2853,10 @@ class DoNodeTearDownTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
mock_console.assert_called_once_with(task)
|
mock_console.assert_called_once_with(task)
|
||||||
else:
|
else:
|
||||||
self.assertFalse(mock_console.called)
|
self.assertFalse(mock_console.called)
|
||||||
|
if with_allocation:
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
objects.Allocation.get_by_id,
|
||||||
|
self.context, alloc.id)
|
||||||
|
|
||||||
def test__do_node_tear_down_ok_without_console(self):
|
def test__do_node_tear_down_ok_without_console(self):
|
||||||
self._test__do_node_tear_down_ok(enabled_console=False)
|
self._test__do_node_tear_down_ok(enabled_console=False)
|
||||||
@ -2783,6 +2864,9 @@ class DoNodeTearDownTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
def test__do_node_tear_down_ok_with_console(self):
|
def test__do_node_tear_down_ok_with_console(self):
|
||||||
self._test__do_node_tear_down_ok(enabled_console=True)
|
self._test__do_node_tear_down_ok(enabled_console=True)
|
||||||
|
|
||||||
|
def test__do_node_tear_down_with_allocation(self):
|
||||||
|
self._test__do_node_tear_down_ok(with_allocation=True)
|
||||||
|
|
||||||
@mock.patch('ironic.drivers.modules.fake.FakeRescue.clean_up')
|
@mock.patch('ironic.drivers.modules.fake.FakeRescue.clean_up')
|
||||||
@mock.patch('ironic.conductor.manager.ConductorManager._do_node_clean')
|
@mock.patch('ironic.conductor.manager.ConductorManager._do_node_clean')
|
||||||
@mock.patch('ironic.drivers.modules.fake.FakeDeploy.tear_down')
|
@mock.patch('ironic.drivers.modules.fake.FakeDeploy.tear_down')
|
||||||
@ -4950,6 +5034,25 @@ class DestroyNodeTestCase(mgr_utils.ServiceSetUpMixin, db_base.DbTestCase):
|
|||||||
node.refresh()
|
node.refresh()
|
||||||
self.assertIsNone(node.reservation)
|
self.assertIsNone(node.reservation)
|
||||||
|
|
||||||
|
def test_destroy_node_with_allocation(self):
|
||||||
|
# Nodes with allocations can be deleted in maintenance
|
||||||
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
provision_state=states.ACTIVE,
|
||||||
|
maintenance=True)
|
||||||
|
alloc = obj_utils.create_test_allocation(self.context)
|
||||||
|
# Establish cross-linking between the node and the allocation
|
||||||
|
alloc.node_id = node.id
|
||||||
|
alloc.save()
|
||||||
|
node.refresh()
|
||||||
|
|
||||||
|
self.service.destroy_node(self.context, node.uuid)
|
||||||
|
self.assertRaises(exception.NodeNotFound,
|
||||||
|
self.dbapi.get_node_by_uuid,
|
||||||
|
node.uuid)
|
||||||
|
self.assertRaises(exception.AllocationNotFound,
|
||||||
|
self.dbapi.get_allocation_by_id,
|
||||||
|
alloc.id)
|
||||||
|
|
||||||
def test_destroy_node_invalid_provision_state(self):
|
def test_destroy_node_invalid_provision_state(self):
|
||||||
self._start_service()
|
self._start_service()
|
||||||
node = obj_utils.create_test_node(self.context,
|
node = obj_utils.create_test_node(self.context,
|
||||||
|
@ -598,3 +598,15 @@ class RPCAPITestCase(db_base.DbTestCase):
|
|||||||
node_id='fake-node',
|
node_id='fake-node',
|
||||||
traits=None,
|
traits=None,
|
||||||
version='1.44')
|
version='1.44')
|
||||||
|
|
||||||
|
def test_create_allocation(self):
|
||||||
|
self._test_rpcapi('create_allocation',
|
||||||
|
'call',
|
||||||
|
allocation='fake-allocation',
|
||||||
|
version='1.48')
|
||||||
|
|
||||||
|
def test_destroy_allocation(self):
|
||||||
|
self._test_rpcapi('destroy_allocation',
|
||||||
|
'call',
|
||||||
|
allocation='fake-allocation',
|
||||||
|
version='1.48')
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
from oslo_utils import uuidutils
|
from oslo_utils import uuidutils
|
||||||
|
|
||||||
from ironic.common import exception
|
from ironic.common import exception
|
||||||
|
from ironic.db import api as db_api
|
||||||
from ironic.tests.unit.db import base
|
from ironic.tests.unit.db import base
|
||||||
from ironic.tests.unit.db import utils as db_utils
|
from ironic.tests.unit.db import utils as db_utils
|
||||||
|
|
||||||
@ -26,6 +27,12 @@ class AllocationsTestCase(base.DbTestCase):
|
|||||||
self.node = db_utils.create_test_node()
|
self.node = db_utils.create_test_node()
|
||||||
self.allocation = db_utils.create_test_allocation(name='host1')
|
self.allocation = db_utils.create_test_allocation(name='host1')
|
||||||
|
|
||||||
|
def test_create(self):
|
||||||
|
dbapi = db_api.get_instance()
|
||||||
|
allocation = dbapi.create_allocation({'resource_class': 'bm'})
|
||||||
|
self.assertIsNotNone(allocation.uuid)
|
||||||
|
self.assertEqual('allocating', allocation.state)
|
||||||
|
|
||||||
def _create_test_allocation_range(self, count, **kw):
|
def _create_test_allocation_range(self, count, **kw):
|
||||||
"""Create the specified number of test allocation entries in DB
|
"""Create the specified number of test allocation entries in DB
|
||||||
|
|
||||||
|
@ -265,6 +265,37 @@ def create_test_conductor(ctxt, **kw):
|
|||||||
return conductor
|
return conductor
|
||||||
|
|
||||||
|
|
||||||
|
def get_test_allocation(ctxt, **kw):
|
||||||
|
"""Return an Allocation object with appropriate attributes.
|
||||||
|
|
||||||
|
NOTE: The object leaves the attributes marked as changed, such
|
||||||
|
that a create() could be used to commit it to the DB.
|
||||||
|
"""
|
||||||
|
kw['object_type'] = 'allocation'
|
||||||
|
get_db_allocation_checked = check_keyword_arguments(
|
||||||
|
db_utils.get_test_allocation)
|
||||||
|
db_allocation = get_db_allocation_checked(**kw)
|
||||||
|
|
||||||
|
# Let DB generate ID if it isn't specified explicitly
|
||||||
|
if 'id' not in kw:
|
||||||
|
del db_allocation['id']
|
||||||
|
allocation = objects.Allocation(ctxt)
|
||||||
|
for key in db_allocation:
|
||||||
|
setattr(allocation, key, db_allocation[key])
|
||||||
|
return allocation
|
||||||
|
|
||||||
|
|
||||||
|
def create_test_allocation(ctxt, **kw):
|
||||||
|
"""Create and return a test allocation object.
|
||||||
|
|
||||||
|
Create an allocation in the DB and return an Allocation object with
|
||||||
|
appropriate attributes.
|
||||||
|
"""
|
||||||
|
allocation = get_test_allocation(ctxt, **kw)
|
||||||
|
allocation.create()
|
||||||
|
return allocation
|
||||||
|
|
||||||
|
|
||||||
def get_payloads_with_schemas(from_module):
|
def get_payloads_with_schemas(from_module):
|
||||||
"""Get the Payload classes with SCHEMAs defined.
|
"""Get the Payload classes with SCHEMAs defined.
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user