Change compute RPC to use alternates for resize

The 'prep_resize()' method of compute now accepts a 'host_list'
parameter that supplies alternate hosts for retrying. This required
bumping the RPC version to 4.21 for compute. The
MigrationTask._execute() method is also changed to request alternates
from the scheduler.

Blueprint: return-alternate-hosts

Change-Id: If6a0bb766e70ab6f1c313da38bb2b0756a2e8772
This commit is contained in:
Ed Leafe 2017-12-05 18:31:56 +00:00
parent a40c00957e
commit 941b309154
12 changed files with 290 additions and 75 deletions

View File

@ -485,7 +485,7 @@ class ComputeVirtAPI(virtapi.VirtAPI):
class ComputeManager(manager.Manager): class ComputeManager(manager.Manager):
"""Manages the running instances from creation to destruction.""" """Manages the running instances from creation to destruction."""
target = messaging.Target(version='4.20') target = messaging.Target(version='4.21')
# How long to wait in seconds before re-issuing a shutdown # How long to wait in seconds before re-issuing a shutdown
# signal to an instance during power off. The overall # signal to an instance during power off. The overall
@ -1330,7 +1330,7 @@ class ComputeManager(manager.Manager):
def _reschedule(self, context, request_spec, filter_properties, def _reschedule(self, context, request_spec, filter_properties,
instance, reschedule_method, method_args, task_state, instance, reschedule_method, method_args, task_state,
exc_info=None): exc_info=None, host_list=None):
"""Attempt to re-schedule a compute operation.""" """Attempt to re-schedule a compute operation."""
instance_uuid = instance.uuid instance_uuid = instance.uuid
@ -1358,7 +1358,7 @@ class ComputeManager(manager.Manager):
retry['exc'] = traceback.format_exception_only(exc_info[0], retry['exc'] = traceback.format_exception_only(exc_info[0],
exc_info[1]) exc_info[1])
reschedule_method(context, *method_args) reschedule_method(context, *method_args, host_list=host_list)
return True return True
@periodic_task.periodic_task @periodic_task.periodic_task
@ -4072,7 +4072,7 @@ class ComputeManager(manager.Manager):
@wrap_instance_fault @wrap_instance_fault
def prep_resize(self, context, image, instance, instance_type, def prep_resize(self, context, image, instance, instance_type,
reservations, request_spec, filter_properties, node, reservations, request_spec, filter_properties, node,
clean_shutdown, migration=None): clean_shutdown, migration=None, host_list=None):
"""Initiates the process of moving a running instance to another host. """Initiates the process of moving a running instance to another host.
Possibly changes the VCPU, RAM and disk size in the process. Possibly changes the VCPU, RAM and disk size in the process.
@ -4123,7 +4123,7 @@ class ComputeManager(manager.Manager):
exc_info = sys.exc_info() exc_info = sys.exc_info()
self._reschedule_resize_or_reraise(context, image, instance, self._reschedule_resize_or_reraise(context, image, instance,
exc_info, instance_type, request_spec, exc_info, instance_type, request_spec,
filter_properties) filter_properties, host_list)
finally: finally:
extra_usage_info = dict( extra_usage_info = dict(
new_instance_type=instance_type.name, new_instance_type=instance_type.name,
@ -4134,7 +4134,7 @@ class ComputeManager(manager.Manager):
extra_usage_info=extra_usage_info) extra_usage_info=extra_usage_info)
def _reschedule_resize_or_reraise(self, context, image, instance, exc_info, def _reschedule_resize_or_reraise(self, context, image, instance, exc_info,
instance_type, request_spec, filter_properties): instance_type, request_spec, filter_properties, host_list):
"""Try to re-schedule the resize or re-raise the original error to """Try to re-schedule the resize or re-raise the original error to
error out the instance. error out the instance.
""" """
@ -4154,7 +4154,7 @@ class ComputeManager(manager.Manager):
rescheduled = self._reschedule(context, request_spec, rescheduled = self._reschedule(context, request_spec,
filter_properties, instance, reschedule_method, filter_properties, instance, reschedule_method,
method_args, task_state, exc_info) method_args, task_state, exc_info, host_list=host_list)
except Exception as error: except Exception as error:
rescheduled = False rescheduled = False
LOG.exception("Error trying to reschedule", LOG.exception("Error trying to reschedule",

View File

@ -338,6 +338,8 @@ class ComputeAPI(object):
representing potential alternate hosts for retries within a representing potential alternate hosts for retries within a
cell. cell.
* 4.20 - Add multiattach argument to reserve_block_device_name(). * 4.20 - Add multiattach argument to reserve_block_device_name().
* 4.21 - prep_resize() now gets a 'host_list' parameter representing
potential alternate hosts for retries within a cell.
''' '''
VERSION_ALIASES = { VERSION_ALIASES = {
@ -768,7 +770,7 @@ class ComputeAPI(object):
def prep_resize(self, ctxt, instance, image, instance_type, host, def prep_resize(self, ctxt, instance, image, instance_type, host,
migration, reservations=None, request_spec=None, migration, reservations=None, request_spec=None,
filter_properties=None, node=None, filter_properties=None, node=None,
clean_shutdown=True): clean_shutdown=True, host_list=None):
image_p = jsonutils.to_primitive(image) image_p = jsonutils.to_primitive(image)
msg_args = {'instance': instance, msg_args = {'instance': instance,
'instance_type': instance_type, 'instance_type': instance_type,
@ -778,9 +780,13 @@ class ComputeAPI(object):
'filter_properties': filter_properties, 'filter_properties': filter_properties,
'node': node, 'node': node,
'migration': migration, 'migration': migration,
'clean_shutdown': clean_shutdown} 'clean_shutdown': clean_shutdown,
'host_list': host_list}
client = self.router.client(ctxt) client = self.router.client(ctxt)
version = '4.18' version = '4.21'
if not client.can_send_version(version):
version = '4.18'
del msg_args['host_list']
if not client.can_send_version(version): if not client.can_send_version(version):
version = '4.1' version = '4.1'
del msg_args['migration'] del msg_args['migration']

View File

@ -86,7 +86,8 @@ class ComputeTaskAPI(object):
def resize_instance(self, context, instance, extra_instance_updates, def resize_instance(self, context, instance, extra_instance_updates,
scheduler_hint, flavor, reservations=None, scheduler_hint, flavor, reservations=None,
clean_shutdown=True, request_spec=None): clean_shutdown=True, request_spec=None,
host_list=None):
# NOTE(comstud): 'extra_instance_updates' is not used here but is # NOTE(comstud): 'extra_instance_updates' is not used here but is
# needed for compatibility with the cells_rpcapi version of this # needed for compatibility with the cells_rpcapi version of this
# method. # method.
@ -94,7 +95,7 @@ class ComputeTaskAPI(object):
context, instance, scheduler_hint, live=False, rebuild=False, context, instance, scheduler_hint, live=False, rebuild=False,
flavor=flavor, block_migration=None, disk_over_commit=None, flavor=flavor, block_migration=None, disk_over_commit=None,
reservations=reservations, clean_shutdown=clean_shutdown, reservations=reservations, clean_shutdown=clean_shutdown,
request_spec=request_spec) request_spec=request_spec, host_list=host_list)
def live_migrate_instance(self, context, instance, host_name, def live_migrate_instance(self, context, instance, host_name,
block_migration, disk_over_commit, block_migration, disk_over_commit,

View File

@ -219,7 +219,7 @@ class ComputeTaskManager(base.Base):
may involve coordinating activities on multiple compute nodes. may involve coordinating activities on multiple compute nodes.
""" """
target = messaging.Target(namespace='compute_task', version='1.19') target = messaging.Target(namespace='compute_task', version='1.20')
def __init__(self): def __init__(self):
super(ComputeTaskManager, self).__init__() super(ComputeTaskManager, self).__init__()
@ -258,7 +258,7 @@ class ComputeTaskManager(base.Base):
@wrap_instance_event(prefix='conductor') @wrap_instance_event(prefix='conductor')
def migrate_server(self, context, instance, scheduler_hint, live, rebuild, def migrate_server(self, context, instance, scheduler_hint, live, rebuild,
flavor, block_migration, disk_over_commit, reservations=None, flavor, block_migration, disk_over_commit, reservations=None,
clean_shutdown=True, request_spec=None): clean_shutdown=True, request_spec=None, host_list=None):
if instance and not isinstance(instance, nova_object.NovaObject): if instance and not isinstance(instance, nova_object.NovaObject):
# NOTE(danms): Until v2 of the RPC API, we need to tolerate # NOTE(danms): Until v2 of the RPC API, we need to tolerate
# old-world instance objects here # old-world instance objects here
@ -281,12 +281,13 @@ class ComputeTaskManager(base.Base):
instance_uuid): instance_uuid):
self._cold_migrate(context, instance, flavor, self._cold_migrate(context, instance, flavor,
scheduler_hint['filter_properties'], scheduler_hint['filter_properties'],
reservations, clean_shutdown, request_spec) reservations, clean_shutdown, request_spec,
host_list)
else: else:
raise NotImplementedError() raise NotImplementedError()
def _cold_migrate(self, context, instance, flavor, filter_properties, def _cold_migrate(self, context, instance, flavor, filter_properties,
reservations, clean_shutdown, request_spec): reservations, clean_shutdown, request_spec, host_list):
image = utils.get_image_from_system_metadata( image = utils.get_image_from_system_metadata(
instance.system_metadata) instance.system_metadata)
@ -307,8 +308,7 @@ class ComputeTaskManager(base.Base):
request_spec.flavor = flavor request_spec.flavor = flavor
task = self._build_cold_migrate_task(context, instance, flavor, task = self._build_cold_migrate_task(context, instance, flavor,
request_spec, request_spec, reservations, clean_shutdown, host_list)
reservations, clean_shutdown)
try: try:
task.execute() task.execute()
except exception.NoValidHost as ex: except exception.NoValidHost as ex:
@ -461,14 +461,13 @@ class ComputeTaskManager(base.Base):
self.scheduler_client, self.scheduler_client,
request_spec) request_spec)
def _build_cold_migrate_task(self, context, instance, flavor, def _build_cold_migrate_task(self, context, instance, flavor, request_spec,
request_spec, reservations, reservations, clean_shutdown, host_list):
clean_shutdown):
return migrate.MigrationTask(context, instance, flavor, return migrate.MigrationTask(context, instance, flavor,
request_spec, request_spec,
reservations, clean_shutdown, reservations, clean_shutdown,
self.compute_rpcapi, self.compute_rpcapi,
self.scheduler_client) self.scheduler_client, host_list)
def _destroy_build_request(self, context, instance): def _destroy_build_request(self, context, instance):
# The BuildRequest needs to be stored until the instance is mapped to # The BuildRequest needs to be stored until the instance is mapped to

View File

@ -278,6 +278,8 @@ class ComputeTaskAPI(object):
1.19 - build_instances() now gets a 'host_lists' parameter that represents 1.19 - build_instances() now gets a 'host_lists' parameter that represents
potential alternate hosts for retries within a cell for each potential alternate hosts for retries within a cell for each
instance. instance.
1.20 - migrate_server() now gets a 'host_list' parameter that represents
potential alternate hosts for retries within a cell.
""" """
def __init__(self): def __init__(self):
@ -303,7 +305,8 @@ class ComputeTaskAPI(object):
# RPC API. # RPC API.
def migrate_server(self, context, instance, scheduler_hint, live, rebuild, def migrate_server(self, context, instance, scheduler_hint, live, rebuild,
flavor, block_migration, disk_over_commit, flavor, block_migration, disk_over_commit,
reservations=None, clean_shutdown=True, request_spec=None): reservations=None, clean_shutdown=True, request_spec=None,
host_list=None):
kw = {'instance': instance, 'scheduler_hint': scheduler_hint, kw = {'instance': instance, 'scheduler_hint': scheduler_hint,
'live': live, 'rebuild': rebuild, 'flavor': flavor, 'live': live, 'rebuild': rebuild, 'flavor': flavor,
'block_migration': block_migration, 'block_migration': block_migration,
@ -311,8 +314,12 @@ class ComputeTaskAPI(object):
'reservations': reservations, 'reservations': reservations,
'clean_shutdown': clean_shutdown, 'clean_shutdown': clean_shutdown,
'request_spec': request_spec, 'request_spec': request_spec,
'host_list': host_list,
} }
version = '1.13' version = '1.20'
if not self.client.can_send_version(version):
del kw['host_list']
version = '1.13'
if not self.client.can_send_version(version): if not self.client.can_send_version(version):
del kw['request_spec'] del kw['request_spec']
version = '1.11' version = '1.11'

View File

@ -110,7 +110,7 @@ def should_do_migration_allocation(context):
class MigrationTask(base.TaskBase): class MigrationTask(base.TaskBase):
def __init__(self, context, instance, flavor, def __init__(self, context, instance, flavor,
request_spec, reservations, clean_shutdown, compute_rpcapi, request_spec, reservations, clean_shutdown, compute_rpcapi,
scheduler_client): scheduler_client, host_list):
super(MigrationTask, self).__init__(context, instance) super(MigrationTask, self).__init__(context, instance)
self.clean_shutdown = clean_shutdown self.clean_shutdown = clean_shutdown
self.request_spec = request_spec self.request_spec = request_spec
@ -119,6 +119,8 @@ class MigrationTask(base.TaskBase):
self.compute_rpcapi = compute_rpcapi self.compute_rpcapi = compute_rpcapi
self.scheduler_client = scheduler_client self.scheduler_client = scheduler_client
self.reportclient = scheduler_client.reportclient
self.host_list = host_list
# Persist things from the happy path so we don't have to look # Persist things from the happy path so we don't have to look
# them up if we need to roll back # them up if we need to roll back
@ -132,16 +134,26 @@ class MigrationTask(base.TaskBase):
# old computes. Let the compute do it (legacy behavior). # old computes. Let the compute do it (legacy behavior).
return None return None
migration = objects.Migration(context=self.context.elevated()) # If this is a rescheduled migration, don't create a new record.
migration.old_instance_type_id = self.instance.flavor.id migration_type = ("resize" if self.instance.flavor.id != self.flavor.id
migration.new_instance_type_id = self.flavor.id else "migration")
migration.status = 'pre-migrating' filters = {"instance_uuid": self.instance.uuid,
migration.instance_uuid = self.instance.uuid "migration_type": migration_type,
migration.source_compute = self.instance.host "status": "pre-migrating"}
migration.source_node = self.instance.node migrations = objects.MigrationList.get_by_filters(self.context,
migration.migration_type = (self.instance.flavor.id != self.flavor.id filters).objects
and 'resize' or 'migration') if migrations:
migration.create() migration = migrations[0]
else:
migration = objects.Migration(context=self.context.elevated())
migration.old_instance_type_id = self.instance.flavor.id
migration.new_instance_type_id = self.flavor.id
migration.status = 'pre-migrating'
migration.instance_uuid = self.instance.uuid
migration.source_compute = self.instance.host
migration.source_node = self.instance.node
migration.migration_type = migration_type
migration.create()
self._migration = migration self._migration = migration
@ -194,17 +206,73 @@ class MigrationTask(base.TaskBase):
self.request_spec.requested_destination = objects.Destination( self.request_spec.requested_destination = objects.Destination(
cell=instance_mapping.cell_mapping) cell=instance_mapping.cell_mapping)
# Once _preallocate_migration() is done, the source node allocation is
# moved from the instance consumer to the migration record consumer,
# and the instance consumer doesn't have any allocations. If this is
# the first time through here (not a reschedule), select_destinations
# below will allocate resources on the selected destination node for
# the instance consumer. If we're rescheduling, host_list is not None
# and we'll call claim_resources for the instance and the selected
# alternate. If we exhaust our alternates and raise MaxRetriesExceeded,
# the rollback() method should revert the allocation swaparoo and move
# the source node allocation from the migration record back to the
# instance record.
migration = self._preallocate_migration() migration = self._preallocate_migration()
self.request_spec.ensure_project_id(self.instance) self.request_spec.ensure_project_id(self.instance)
# For now, don't request alternates. A later patch in the series will # On an initial call to migrate, 'self.host_list' will be None, so we
# modify migration to use alternates instead of calling the scheduler # have to call the scheduler to get a list of acceptable hosts to
# again. # migrate to. That list will consist of a selected host, along with
selection_lists = self.scheduler_client.select_destinations( # zero or more alternates. On a reschedule, though, the alternates will
self.context, self.request_spec, [self.instance.uuid], # be passed to this object and stored in 'self.host_list', so we can
return_objects=True, return_alternates=False) # pop the first alternate from the list to use for the destination, and
# We only need the first item in the first list, as there is only one # pass the remaining alternates to the compute.
# instance, and we don't care about any alternates. if self.host_list is None:
selection = selection_lists[0][0] selection_lists = self.scheduler_client.select_destinations(
self.context, self.request_spec, [self.instance.uuid],
return_objects=True, return_alternates=True)
# Since there is only ever one instance to migrate per call, we
# just need the first returned element.
selection_list = selection_lists[0]
# The selected host is the first item in the list, with the
# alternates being the remainder of the list.
selection, self.host_list = selection_list[0], selection_list[1:]
else:
# This is a reschedule that will use the supplied alternate hosts
# in the host_list as destinations. Since the resources on these
# alternates may have been consumed and might not be able to
# support the migrated instance, we need to first claim the
# resources to verify the host still has sufficient availabile
# resources.
elevated = self.context.elevated()
host_available = False
while self.host_list and not host_available:
selection = self.host_list.pop(0)
if selection.allocation_request:
alloc_req = jsonutils.loads(selection.allocation_request)
else:
alloc_req = None
if alloc_req:
# If this call succeeds, the resources on the destination
# host will be claimed by the instance.
host_available = scheduler_utils.claim_resources(
elevated, self.reportclient, self.request_spec,
self.instance.uuid, alloc_req,
selection.allocation_request_version)
else:
# Some deployments use different schedulers that do not
# use Placement, so they will not have an
# allocation_request to claim with. For those cases,
# there is no concept of claiming, so just assume that
# the host is valid.
host_available = True
# There are no more available hosts. Raise a MaxRetriesExceeded
# exception in that case.
if not host_available:
reason = ("Exhausted all hosts available for retrying build "
"failures for instance %(instance_uuid)s." %
{"instance_uuid": self.instance.uuid})
raise exception.MaxRetriesExceeded(reason=reason)
scheduler_utils.populate_filter_properties(legacy_props, selection) scheduler_utils.populate_filter_properties(legacy_props, selection)
# context is not serializable # context is not serializable
@ -222,12 +290,16 @@ class MigrationTask(base.TaskBase):
# rpc fake driver. # rpc fake driver.
legacy_spec = jsonutils.loads(jsonutils.dumps(legacy_spec)) legacy_spec = jsonutils.loads(jsonutils.dumps(legacy_spec))
LOG.debug("Calling prep_resize with selected host: %s; "
"Selected node: %s; Alternates: %s", host, node,
self.host_list, instance=self.instance)
# RPC cast to the destination host to start the migration process. # RPC cast to the destination host to start the migration process.
self.compute_rpcapi.prep_resize( self.compute_rpcapi.prep_resize(
self.context, self.instance, legacy_spec['image'], self.context, self.instance, legacy_spec['image'],
self.flavor, host, migration, self.reservations, self.flavor, host, migration, self.reservations,
request_spec=legacy_spec, filter_properties=legacy_props, request_spec=legacy_spec, filter_properties=legacy_props,
node=node, clean_shutdown=self.clean_shutdown) node=node, clean_shutdown=self.clean_shutdown,
host_list=self.host_list)
def rollback(self): def rollback(self):
if self._migration: if self._migration:

View File

@ -31,7 +31,7 @@ LOG = logging.getLogger(__name__)
# NOTE(danms): This is the global service version counter # NOTE(danms): This is the global service version counter
SERVICE_VERSION = 27 SERVICE_VERSION = 28
# NOTE(danms): This is our SERVICE_VERSION history. The idea is that any # NOTE(danms): This is our SERVICE_VERSION history. The idea is that any
@ -125,6 +125,8 @@ SERVICE_VERSION_HISTORY = (
# Version 27: Compute RPC version 4.20; adds multiattach argument to # Version 27: Compute RPC version 4.20; adds multiattach argument to
# reserve_block_device_name(). # reserve_block_device_name().
{'compute_rpc': '4.20'}, {'compute_rpc': '4.20'},
# Version 28: Adds a 'host_list' parameter to prep_resize()
{'compute_rpc': '4.21'},
) )

View File

@ -32,6 +32,7 @@ from nova import db
from nova import exception from nova import exception
from nova import objects from nova import objects
from nova.objects import block_device as block_device_obj from nova.objects import block_device as block_device_obj
from nova.scheduler import weights
from nova import test from nova import test
from nova.tests import fixtures as nova_fixtures from nova.tests import fixtures as nova_fixtures
from nova.tests.functional.api import client from nova.tests.functional.api import client
@ -49,6 +50,17 @@ from nova import volume
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
class AltHostWeigher(weights.BaseHostWeigher):
"""Used in the alternate host tests to return a pre-determined list of
hosts.
"""
def _weigh_object(self, host_state, weight_properties):
"""Return a defined order of hosts."""
weights = {"selection": 999, "alt_host1": 888, "alt_host2": 777,
"alt_host3": 666, "host1": 0, "host2": 0}
return weights.get(host_state.host, 0)
class ServersTestBase(integrated_helpers._IntegratedTestBase): class ServersTestBase(integrated_helpers._IntegratedTestBase):
api_major_version = 'v2' api_major_version = 'v2'
_force_delete_parameter = 'forceDelete' _force_delete_parameter = 'forceDelete'
@ -1369,7 +1381,7 @@ class ProviderUsageBaseTestCase(test.TestCase,
nova.tests.unit.image.fake.stub_out_image_service(self) nova.tests.unit.image.fake.stub_out_image_service(self)
self.start_service('conductor') self.start_service('conductor')
self.start_service('scheduler') self.scheduler_service = self.start_service('scheduler')
self.addCleanup(nova.tests.unit.image.fake.FakeImageService_reset) self.addCleanup(nova.tests.unit.image.fake.FakeImageService_reset)
fake_network.set_stub_network_methods(self) fake_network.set_stub_network_methods(self)
@ -2699,6 +2711,114 @@ class ServerMovingTests(ProviderUsageBaseTestCase):
# allocation which just leaves us with the original flavor. # allocation which just leaves us with the original flavor.
self.assertFlavorMatchesAllocation(self.flavor1, source_usages) self.assertFlavorMatchesAllocation(self.flavor1, source_usages)
def _test_resize_reschedule_uses_host_lists(self, fails, num_alts=None):
"""Test that when a resize attempt fails, the retry comes from the
supplied host_list, and does not call the scheduler.
"""
server_req = self._build_minimal_create_server_request(
self.api, "some-server", flavor_id=self.flavor1["id"],
image_uuid="155d900f-4e14-4e4c-a73d-069cbf4541e6",
networks=[])
created_server = self.api.post_server({"server": server_req})
server = self._wait_for_state_change(self.api, created_server,
"ACTIVE")
inst_host = server["OS-EXT-SRV-ATTR:host"]
uuid_orig = self._get_provider_uuid_by_host(inst_host)
# We will need four new compute nodes to test the resize, representing
# the host selected by select_destinations(), along with 3 alternates.
self._start_compute(host="selection")
self._start_compute(host="alt_host1")
self._start_compute(host="alt_host2")
self._start_compute(host="alt_host3")
uuid_sel = self._get_provider_uuid_by_host("selection")
uuid_alt1 = self._get_provider_uuid_by_host("alt_host1")
uuid_alt2 = self._get_provider_uuid_by_host("alt_host2")
uuid_alt3 = self._get_provider_uuid_by_host("alt_host3")
hosts = [{"name": "selection", "uuid": uuid_sel},
{"name": "alt_host1", "uuid": uuid_alt1},
{"name": "alt_host2", "uuid": uuid_alt2},
{"name": "alt_host3", "uuid": uuid_alt3},
]
self.flags(weight_classes=[__name__ + '.AltHostWeigher'],
group='filter_scheduler')
self.scheduler_service.stop()
self.scheduler_service = self.start_service('scheduler')
def fake_prep_resize(*args, **kwargs):
if self.num_fails < fails:
self.num_fails += 1
raise Exception("fake_prep_resize")
actual_prep_resize(*args, **kwargs)
# Yes this isn't great in a functional test, but it's simple.
actual_prep_resize = compute_manager.ComputeManager._prep_resize
self.stub_out("nova.compute.manager.ComputeManager._prep_resize",
fake_prep_resize)
self.num_fails = 0
num_alts = 4 if num_alts is None else num_alts
# Make sure we have enough retries available for the number of
# requested fails.
attempts = min(fails + 2, num_alts)
self.flags(max_attempts=attempts, group='scheduler')
server_uuid = server["id"]
data = {"resize": {"flavorRef": self.flavor2["id"]}}
self.api.post_server_action(server_uuid, data)
if num_alts < fails:
# We will run out of alternates before populate_retry will
# raise a MaxRetriesExceeded exception, so the migration will
# fail and the server should be in status "ERROR"
server = self._wait_for_state_change(self.api, created_server,
"ERROR")
source_usages = self._get_provider_usages(uuid_orig)
# The usage should be unchanged from the original flavor
self.assertFlavorMatchesAllocation(self.flavor1, source_usages)
# There should be no usages on any of the hosts
target_uuids = (uuid_sel, uuid_alt1, uuid_alt2, uuid_alt3)
empty_usage = {"VCPU": 0, "MEMORY_MB": 0, "DISK_GB": 0}
for target_uuid in target_uuids:
usage = self._get_provider_usages(target_uuid)
self.assertEqual(empty_usage, usage)
else:
server = self._wait_for_state_change(self.api, created_server,
"VERIFY_RESIZE")
# Verify that the selected host failed, and was rescheduled to
# an alternate host.
new_server_host = server.get("OS-EXT-SRV-ATTR:host")
expected_host = hosts[fails]["name"]
self.assertEqual(expected_host, new_server_host)
uuid_dest = hosts[fails]["uuid"]
source_usages = self._get_provider_usages(uuid_orig)
dest_usages = self._get_provider_usages(uuid_dest)
# The usage should match the resized flavor
self.assertFlavorMatchesAllocation(self.flavor2, dest_usages)
# Verify that the other host have no allocations
target_uuids = (uuid_sel, uuid_alt1, uuid_alt2, uuid_alt3)
empty_usage = {"VCPU": 0, "MEMORY_MB": 0, "DISK_GB": 0}
for target_uuid in target_uuids:
if target_uuid == uuid_dest:
continue
usage = self._get_provider_usages(target_uuid)
self.assertEqual(empty_usage, usage)
# Verify that there is only one migration record for the instance.
ctxt = context.get_admin_context()
filters = {"instance_uuid": server["id"]}
migrations = objects.MigrationList.get_by_filters(ctxt, filters)
self.assertEqual(1, len(migrations.objects))
def test_resize_reschedule_uses_host_lists_1_fail(self):
self._test_resize_reschedule_uses_host_lists(fails=1)
def test_resize_reschedule_uses_host_lists_3_fails(self):
self._test_resize_reschedule_uses_host_lists(fails=3)
def test_resize_reschedule_uses_host_lists_not_enough_alts(self):
self._test_resize_reschedule_uses_host_lists(fails=3, num_alts=1)
class ServerLiveMigrateForceAndAbort(ProviderUsageBaseTestCase): class ServerLiveMigrateForceAndAbort(ProviderUsageBaseTestCase):
"""Test Server live migrations, which delete the migration or """Test Server live migrations, which delete the migration or

View File

@ -145,7 +145,7 @@ def unify_instance(instance):
class FakeComputeTaskAPI(object): class FakeComputeTaskAPI(object):
def resize_instance(self, context, instance, extra_instance_updates, def resize_instance(self, context, instance, extra_instance_updates,
scheduler_hint, flavor, reservations): scheduler_hint, flavor, reservationsi, host_list=None):
pass pass
@ -5526,7 +5526,8 @@ class ComputeTestCase(BaseTestCase,
instance=instance, instance=instance,
instance_type=new_instance_type_ref, instance_type=new_instance_type_ref,
image={}, reservations=[], request_spec={}, image={}, reservations=[], request_spec={},
filter_properties={}, node=None, clean_shutdown=True) filter_properties={}, node=None, clean_shutdown=True,
host_list=None)
# Memory usage should increase after the resize as well # Memory usage should increase after the resize as well
self.assertEqual(self.rt.compute_nodes[NODENAME].memory_mb_used, self.assertEqual(self.rt.compute_nodes[NODENAME].memory_mb_used,
@ -12314,11 +12315,11 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase):
instance_type=self.instance_type, instance_type=self.instance_type,
reservations=[], request_spec={}, reservations=[], request_spec={},
filter_properties={}, node=None, filter_properties={}, node=None,
clean_shutdown=True) clean_shutdown=True, host_list=None)
mock_mig.assert_called_once_with(mock.ANY, mock.ANY) mock_mig.assert_called_once_with(mock.ANY, mock.ANY)
mock_res.assert_called_once_with(mock.ANY, None, inst_obj, mock.ANY, mock_res.assert_called_once_with(mock.ANY, None, inst_obj, mock.ANY,
self.instance_type, {}, {}) self.instance_type, {}, {}, None)
@mock.patch.object(compute_manager.ComputeManager, "_reschedule") @mock.patch.object(compute_manager.ComputeManager, "_reschedule")
@mock.patch('nova.compute.utils.notify_about_instance_action') @mock.patch('nova.compute.utils.notify_about_instance_action')
@ -12337,12 +12338,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase):
exc_info = sys.exc_info() exc_info = sys.exc_info()
self.assertRaises(test.TestingException, self.assertRaises(test.TestingException,
self.compute._reschedule_resize_or_reraise, self.context, self.compute._reschedule_resize_or_reraise, self.context,
None, instance, exc_info, self.instance_type, {}, {}) None, instance, exc_info, self.instance_type, {}, {}, None)
mock_res.assert_called_once_with( mock_res.assert_called_once_with(
self.context, {}, {}, instance, self.context, {}, {}, instance,
self.compute.compute_task_api.resize_instance, method_args, self.compute.compute_task_api.resize_instance, method_args,
task_states.RESIZE_PREP, exc_info) task_states.RESIZE_PREP, exc_info, host_list=None)
mock_notify.assert_called_once_with( mock_notify.assert_called_once_with(
self.context, instance, 'fake-mini', action='resize', self.context, instance, 'fake-mini', action='resize',
phase='error', exception=mock_res.side_effect) phase='error', exception=mock_res.side_effect)
@ -12363,12 +12364,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase):
exc_info = sys.exc_info() exc_info = sys.exc_info()
self.assertRaises(test.TestingException, self.assertRaises(test.TestingException,
self.compute._reschedule_resize_or_reraise, self.context, self.compute._reschedule_resize_or_reraise, self.context,
None, instance, exc_info, self.instance_type, {}, {}) None, instance, exc_info, self.instance_type, {}, {}, None)
mock_res.assert_called_once_with( mock_res.assert_called_once_with(
self.context, {}, {}, instance, self.context, {}, {}, instance,
self.compute.compute_task_api.resize_instance, method_args, self.compute.compute_task_api.resize_instance, method_args,
task_states.RESIZE_PREP, exc_info) task_states.RESIZE_PREP, exc_info, host_list=None)
@mock.patch.object(compute_manager.ComputeManager, "_reschedule") @mock.patch.object(compute_manager.ComputeManager, "_reschedule")
@mock.patch.object(compute_manager.ComputeManager, "_log_original_error") @mock.patch.object(compute_manager.ComputeManager, "_log_original_error")
@ -12386,11 +12387,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase):
self.compute._reschedule_resize_or_reraise( self.compute._reschedule_resize_or_reraise(
self.context, None, instance, exc_info, self.context, None, instance, exc_info,
self.instance_type, {}, {}) self.instance_type, {}, {}, None)
mock_res.assert_called_once_with(self.context, {}, {}, mock_res.assert_called_once_with(self.context, {}, {},
instance, self.compute.compute_task_api.resize_instance, instance, self.compute.compute_task_api.resize_instance,
method_args, task_states.RESIZE_PREP, exc_info) method_args, task_states.RESIZE_PREP, exc_info,
host_list=None)
mock_log.assert_called_once_with(exc_info, instance.uuid) mock_log.assert_called_once_with(exc_info, instance.uuid)

View File

@ -495,7 +495,8 @@ class ComputeRpcAPITestCase(test.NoDBTestCase):
request_spec='fake_spec', request_spec='fake_spec',
filter_properties={'fakeprop': 'fakeval'}, filter_properties={'fakeprop': 'fakeval'},
migration='migration', migration='migration',
node='node', clean_shutdown=True, version='4.18') node='node', clean_shutdown=True, host_list=None,
version='4.21')
self.flags(compute='4.0', group='upgrade_levels') self.flags(compute='4.0', group='upgrade_levels')
expected_args = {'instance_type': self.fake_flavor} expected_args = {'instance_type': self.fake_flavor}
self._test_compute_api('prep_resize', 'cast', expected_args, self._test_compute_api('prep_resize', 'cast', expected_args,

View File

@ -55,7 +55,8 @@ class MigrationTaskTestCase(test.NoDBTestCase):
self.request_spec, self.reservations, self.request_spec, self.reservations,
self.clean_shutdown, self.clean_shutdown,
compute_rpcapi.ComputeAPI(), compute_rpcapi.ComputeAPI(),
scheduler_client.SchedulerClient()) scheduler_client.SchedulerClient(),
host_list=None)
@mock.patch('nova.objects.Service.get_minimum_version_multi') @mock.patch('nova.objects.Service.get_minimum_version_multi')
@mock.patch('nova.availability_zones.get_host_availability_zone') @mock.patch('nova.availability_zones.get_host_availability_zone')
@ -75,17 +76,18 @@ class MigrationTaskTestCase(test.NoDBTestCase):
sig_mock.assert_called_once_with(self.context, self.request_spec) sig_mock.assert_called_once_with(self.context, self.request_spec)
task.scheduler_client.select_destinations.assert_called_once_with( task.scheduler_client.select_destinations.assert_called_once_with(
self.context, self.request_spec, [self.instance.uuid], self.context, self.request_spec, [self.instance.uuid],
return_objects=True, return_alternates=False) return_objects=True, return_alternates=True)
selection = self.host_lists[0][0] selection = self.host_lists[0][0]
prep_resize_mock.assert_called_once_with( prep_resize_mock.assert_called_once_with(
self.context, self.instance, legacy_request_spec['image'], self.context, self.instance, legacy_request_spec['image'],
self.flavor, selection.service_host, None, self.reservations, self.flavor, selection.service_host, None, self.reservations,
request_spec=legacy_request_spec, request_spec=legacy_request_spec,
filter_properties=self.filter_properties, node=selection.nodename, filter_properties=self.filter_properties, node=selection.nodename,
clean_shutdown=self.clean_shutdown) clean_shutdown=self.clean_shutdown, host_list=[])
az_mock.assert_called_once_with(self.context, 'host1') az_mock.assert_called_once_with(self.context, 'host1')
self.assertIsNone(task._migration) self.assertIsNone(task._migration)
@mock.patch.object(objects.MigrationList, 'get_by_filters')
@mock.patch('nova.scheduler.client.report.SchedulerReportClient') @mock.patch('nova.scheduler.client.report.SchedulerReportClient')
@mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename') @mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename')
@mock.patch('nova.objects.Migration.save') @mock.patch('nova.objects.Migration.save')
@ -96,10 +98,11 @@ class MigrationTaskTestCase(test.NoDBTestCase):
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations') @mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize') @mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
def _test_execute(self, prep_resize_mock, sel_dest_mock, sig_mock, az_mock, def _test_execute(self, prep_resize_mock, sel_dest_mock, sig_mock, az_mock,
gmv_mock, cm_mock, sm_mock, cn_mock, rc_mock, gmv_mock, cm_mock, sm_mock, cn_mock, rc_mock, gbf_mock,
requested_destination=False): requested_destination=False):
sel_dest_mock.return_value = self.host_lists sel_dest_mock.return_value = self.host_lists
az_mock.return_value = 'myaz' az_mock.return_value = 'myaz'
gbf_mock.return_value = objects.MigrationList()
if requested_destination: if requested_destination:
self.request_spec.requested_destination = objects.Destination( self.request_spec.requested_destination = objects.Destination(
@ -130,14 +133,14 @@ class MigrationTaskTestCase(test.NoDBTestCase):
sig_mock.assert_called_once_with(self.context, self.request_spec) sig_mock.assert_called_once_with(self.context, self.request_spec)
task.scheduler_client.select_destinations.assert_called_once_with( task.scheduler_client.select_destinations.assert_called_once_with(
self.context, self.request_spec, [self.instance.uuid], self.context, self.request_spec, [self.instance.uuid],
return_objects=True, return_alternates=False) return_objects=True, return_alternates=True)
selection = self.host_lists[0][0] selection = self.host_lists[0][0]
prep_resize_mock.assert_called_once_with( prep_resize_mock.assert_called_once_with(
self.context, self.instance, legacy_request_spec['image'], self.context, self.instance, legacy_request_spec['image'],
self.flavor, selection.service_host, task._migration, self.flavor, selection.service_host, task._migration,
self.reservations, request_spec=legacy_request_spec, self.reservations, request_spec=legacy_request_spec,
filter_properties=self.filter_properties, node=selection.nodename, filter_properties=self.filter_properties, node=selection.nodename,
clean_shutdown=self.clean_shutdown) clean_shutdown=self.clean_shutdown, host_list=[])
az_mock.assert_called_once_with(self.context, 'host1') az_mock.assert_called_once_with(self.context, 'host1')
self.assertIsNotNone(task._migration) self.assertIsNotNone(task._migration)
@ -172,6 +175,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
self.flavor.id = 3 self.flavor.id = 3
self._test_execute() self._test_execute()
@mock.patch.object(objects.MigrationList, 'get_by_filters')
@mock.patch('nova.conductor.tasks.migrate.revert_allocation_for_migration') @mock.patch('nova.conductor.tasks.migrate.revert_allocation_for_migration')
@mock.patch('nova.scheduler.client.report.SchedulerReportClient') @mock.patch('nova.scheduler.client.report.SchedulerReportClient')
@mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename') @mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename')
@ -184,11 +188,12 @@ class MigrationTaskTestCase(test.NoDBTestCase):
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize') @mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
def test_execute_rollback(self, prep_resize_mock, sel_dest_mock, sig_mock, def test_execute_rollback(self, prep_resize_mock, sel_dest_mock, sig_mock,
az_mock, gmv_mock, cm_mock, sm_mock, cn_mock, az_mock, gmv_mock, cm_mock, sm_mock, cn_mock,
rc_mock, mock_ra): rc_mock, mock_ra, mock_gbf):
sel_dest_mock.return_value = self.host_lists sel_dest_mock.return_value = self.host_lists
az_mock.return_value = 'myaz' az_mock.return_value = 'myaz'
task = self._generate_task() task = self._generate_task()
gmv_mock.return_value = 23 gmv_mock.return_value = 23
mock_gbf.return_value = objects.MigrationList()
# We just need this hook point to set a uuid on the # We just need this hook point to set a uuid on the
# migration before we use it for teardown # migration before we use it for teardown

View File

@ -400,7 +400,7 @@ class _BaseTaskTestCase(object):
# converted into 'migrate_server' when doing RPC. # converted into 'migrate_server' when doing RPC.
self.conductor.resize_instance( self.conductor.resize_instance(
self.context, inst_obj, {}, scheduler_hint, flavor, [], self.context, inst_obj, {}, scheduler_hint, flavor, [],
clean_shutdown) clean_shutdown, host_list=None)
else: else:
self.conductor.migrate_server( self.conductor.migrate_server(
self.context, inst_obj, scheduler_hint, self.context, inst_obj, scheduler_hint,
@ -2197,7 +2197,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.conductor._cold_migrate, self.conductor._cold_migrate,
self.context, inst_obj, self.context, inst_obj,
flavor, {}, [resvs], flavor, {}, [resvs],
True, None) True, None, None)
metadata_mock.assert_called_with({}) metadata_mock.assert_called_with({})
sig_mock.assert_called_once_with(self.context, fake_spec) sig_mock.assert_called_once_with(self.context, fake_spec)
self.assertEqual(inst_obj.project_id, fake_spec.project_id) self.assertEqual(inst_obj.project_id, fake_spec.project_id)
@ -2249,7 +2249,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.conductor._cold_migrate, self.conductor._cold_migrate,
self.context, inst_obj, self.context, inst_obj,
flavor, {}, [resvs], flavor, {}, [resvs],
True, None) True, None, None)
metadata_mock.assert_called_with({}) metadata_mock.assert_called_with({})
sig_mock.assert_called_once_with(self.context, fake_spec) sig_mock.assert_called_once_with(self.context, fake_spec)
self.assertEqual(inst_obj.project_id, fake_spec.project_id) self.assertEqual(inst_obj.project_id, fake_spec.project_id)
@ -2284,7 +2284,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
nvh = self.assertRaises(exc.NoValidHost, nvh = self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate, self.context, self.conductor._cold_migrate, self.context,
inst_obj, flavor, {}, [resvs], inst_obj, flavor, {}, [resvs],
True, fake_spec) True, fake_spec, None)
self.assertIn('cold migrate', nvh.message) self.assertIn('cold migrate', nvh.message)
@mock.patch.object(utils, 'get_image_from_system_metadata') @mock.patch.object(utils, 'get_image_from_system_metadata')
@ -2322,7 +2322,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.assertRaises(exc.UnsupportedPolicyException, self.assertRaises(exc.UnsupportedPolicyException,
self.conductor._cold_migrate, self.context, self.conductor._cold_migrate, self.context,
inst_obj, flavor, {}, [resvs], True, None) inst_obj, flavor, {}, [resvs], True, None, None)
updates = {'vm_state': vm_states.STOPPED, 'task_state': None} updates = {'vm_state': vm_states.STOPPED, 'task_state': None}
set_vm_mock.assert_called_once_with(self.context, inst_obj.uuid, set_vm_mock.assert_called_once_with(self.context, inst_obj.uuid,
@ -2376,7 +2376,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.assertRaises(test.TestingException, self.assertRaises(test.TestingException,
self.conductor._cold_migrate, self.conductor._cold_migrate,
self.context, inst_obj, flavor, self.context, inst_obj, flavor,
{}, [resvs], True, None) {}, [resvs], True, None, None)
# Filter properties are populated during code execution # Filter properties are populated during code execution
legacy_filter_props = {'retry': {'num_attempts': 1, legacy_filter_props = {'retry': {'num_attempts': 1,
@ -2387,13 +2387,13 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
sig_mock.assert_called_once_with(self.context, fake_spec) sig_mock.assert_called_once_with(self.context, fake_spec)
self.assertEqual(inst_obj.project_id, fake_spec.project_id) self.assertEqual(inst_obj.project_id, fake_spec.project_id)
select_dest_mock.assert_called_once_with(self.context, fake_spec, select_dest_mock.assert_called_once_with(self.context, fake_spec,
[inst_obj.uuid], return_objects=True, return_alternates=False) [inst_obj.uuid], return_objects=True, return_alternates=True)
prep_resize_mock.assert_called_once_with( prep_resize_mock.assert_called_once_with(
self.context, inst_obj, legacy_request_spec['image'], self.context, inst_obj, legacy_request_spec['image'],
flavor, hosts[0]['host'], None, [resvs], flavor, hosts[0]['host'], None, [resvs],
request_spec=legacy_request_spec, request_spec=legacy_request_spec,
filter_properties=legacy_filter_props, filter_properties=legacy_filter_props,
node=hosts[0]['nodename'], clean_shutdown=True) node=hosts[0]['nodename'], clean_shutdown=True, host_list=[])
notify_mock.assert_called_once_with(self.context, inst_obj.uuid, notify_mock.assert_called_once_with(self.context, inst_obj.uuid,
'migrate_server', updates, 'migrate_server', updates,
exc_info, fake_spec) exc_info, fake_spec)
@ -2427,7 +2427,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
# the new one # the new one
self.assertNotEqual(flavor, fake_spec.flavor) self.assertNotEqual(flavor, fake_spec.flavor)
self.conductor._cold_migrate(self.context, inst_obj, flavor, {}, self.conductor._cold_migrate(self.context, inst_obj, flavor, {},
[resvs], True, fake_spec) [resvs], True, fake_spec, None)
# Now the RequestSpec should be updated... # Now the RequestSpec should be updated...
self.assertEqual(flavor, fake_spec.flavor) self.assertEqual(flavor, fake_spec.flavor)
@ -2464,7 +2464,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
nvh = self.assertRaises(exc.NoValidHost, nvh = self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate, self.context, self.conductor._cold_migrate, self.context,
inst_obj, flavor_new, {}, inst_obj, flavor_new, {},
[resvs], True, fake_spec) [resvs], True, fake_spec, None)
self.assertIn('resize', nvh.message) self.assertIn('resize', nvh.message)
@mock.patch('nova.objects.BuildRequest.get_by_instance_uuid') @mock.patch('nova.objects.BuildRequest.get_by_instance_uuid')