diff --git a/nova/compute/manager.py b/nova/compute/manager.py index 7ba5fb52a32f..584f77359f46 100644 --- a/nova/compute/manager.py +++ b/nova/compute/manager.py @@ -485,7 +485,7 @@ class ComputeVirtAPI(virtapi.VirtAPI): class ComputeManager(manager.Manager): """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 # 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, instance, reschedule_method, method_args, task_state, - exc_info=None): + exc_info=None, host_list=None): """Attempt to re-schedule a compute operation.""" instance_uuid = instance.uuid @@ -1358,7 +1358,7 @@ class ComputeManager(manager.Manager): retry['exc'] = traceback.format_exception_only(exc_info[0], exc_info[1]) - reschedule_method(context, *method_args) + reschedule_method(context, *method_args, host_list=host_list) return True @periodic_task.periodic_task @@ -4072,7 +4072,7 @@ class ComputeManager(manager.Manager): @wrap_instance_fault def prep_resize(self, context, image, instance, instance_type, 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. Possibly changes the VCPU, RAM and disk size in the process. @@ -4123,7 +4123,7 @@ class ComputeManager(manager.Manager): exc_info = sys.exc_info() self._reschedule_resize_or_reraise(context, image, instance, exc_info, instance_type, request_spec, - filter_properties) + filter_properties, host_list) finally: extra_usage_info = dict( new_instance_type=instance_type.name, @@ -4134,7 +4134,7 @@ class ComputeManager(manager.Manager): extra_usage_info=extra_usage_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 error out the instance. """ @@ -4154,7 +4154,7 @@ class ComputeManager(manager.Manager): rescheduled = self._reschedule(context, request_spec, 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: rescheduled = False LOG.exception("Error trying to reschedule", diff --git a/nova/compute/rpcapi.py b/nova/compute/rpcapi.py index f61237f1605c..7f62e5b341ba 100644 --- a/nova/compute/rpcapi.py +++ b/nova/compute/rpcapi.py @@ -338,6 +338,8 @@ class ComputeAPI(object): representing potential alternate hosts for retries within a cell. * 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 = { @@ -768,7 +770,7 @@ class ComputeAPI(object): def prep_resize(self, ctxt, instance, image, instance_type, host, migration, reservations=None, request_spec=None, filter_properties=None, node=None, - clean_shutdown=True): + clean_shutdown=True, host_list=None): image_p = jsonutils.to_primitive(image) msg_args = {'instance': instance, 'instance_type': instance_type, @@ -778,9 +780,13 @@ class ComputeAPI(object): 'filter_properties': filter_properties, 'node': node, 'migration': migration, - 'clean_shutdown': clean_shutdown} + 'clean_shutdown': clean_shutdown, + 'host_list': host_list} 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): version = '4.1' del msg_args['migration'] diff --git a/nova/conductor/api.py b/nova/conductor/api.py index 35317d9d85a8..91f8811cbf84 100644 --- a/nova/conductor/api.py +++ b/nova/conductor/api.py @@ -86,7 +86,8 @@ class ComputeTaskAPI(object): def resize_instance(self, context, instance, extra_instance_updates, 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 # needed for compatibility with the cells_rpcapi version of this # method. @@ -94,7 +95,7 @@ class ComputeTaskAPI(object): context, instance, scheduler_hint, live=False, rebuild=False, flavor=flavor, block_migration=None, disk_over_commit=None, 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, block_migration, disk_over_commit, diff --git a/nova/conductor/manager.py b/nova/conductor/manager.py index b5236a974634..e54f63843f7f 100644 --- a/nova/conductor/manager.py +++ b/nova/conductor/manager.py @@ -219,7 +219,7 @@ class ComputeTaskManager(base.Base): 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): super(ComputeTaskManager, self).__init__() @@ -258,7 +258,7 @@ class ComputeTaskManager(base.Base): @wrap_instance_event(prefix='conductor') def migrate_server(self, context, instance, scheduler_hint, live, rebuild, 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): # NOTE(danms): Until v2 of the RPC API, we need to tolerate # old-world instance objects here @@ -281,12 +281,13 @@ class ComputeTaskManager(base.Base): instance_uuid): self._cold_migrate(context, instance, flavor, scheduler_hint['filter_properties'], - reservations, clean_shutdown, request_spec) + reservations, clean_shutdown, request_spec, + host_list) else: raise NotImplementedError() 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( instance.system_metadata) @@ -307,8 +308,7 @@ class ComputeTaskManager(base.Base): request_spec.flavor = flavor task = self._build_cold_migrate_task(context, instance, flavor, - request_spec, - reservations, clean_shutdown) + request_spec, reservations, clean_shutdown, host_list) try: task.execute() except exception.NoValidHost as ex: @@ -461,14 +461,13 @@ class ComputeTaskManager(base.Base): self.scheduler_client, request_spec) - def _build_cold_migrate_task(self, context, instance, flavor, - request_spec, reservations, - clean_shutdown): + def _build_cold_migrate_task(self, context, instance, flavor, request_spec, + reservations, clean_shutdown, host_list): return migrate.MigrationTask(context, instance, flavor, request_spec, reservations, clean_shutdown, self.compute_rpcapi, - self.scheduler_client) + self.scheduler_client, host_list) def _destroy_build_request(self, context, instance): # The BuildRequest needs to be stored until the instance is mapped to diff --git a/nova/conductor/rpcapi.py b/nova/conductor/rpcapi.py index 85ab5f66f0c5..5b58c279dc0d 100644 --- a/nova/conductor/rpcapi.py +++ b/nova/conductor/rpcapi.py @@ -278,6 +278,8 @@ class ComputeTaskAPI(object): 1.19 - build_instances() now gets a 'host_lists' parameter that represents potential alternate hosts for retries within a cell for each instance. + 1.20 - migrate_server() now gets a 'host_list' parameter that represents + potential alternate hosts for retries within a cell. """ def __init__(self): @@ -303,7 +305,8 @@ class ComputeTaskAPI(object): # RPC API. def migrate_server(self, context, instance, scheduler_hint, live, rebuild, 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, 'live': live, 'rebuild': rebuild, 'flavor': flavor, 'block_migration': block_migration, @@ -311,8 +314,12 @@ class ComputeTaskAPI(object): 'reservations': reservations, 'clean_shutdown': clean_shutdown, '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): del kw['request_spec'] version = '1.11' diff --git a/nova/conductor/tasks/migrate.py b/nova/conductor/tasks/migrate.py index b2cf00f5580c..504cd31e597a 100644 --- a/nova/conductor/tasks/migrate.py +++ b/nova/conductor/tasks/migrate.py @@ -110,7 +110,7 @@ def should_do_migration_allocation(context): class MigrationTask(base.TaskBase): def __init__(self, context, instance, flavor, request_spec, reservations, clean_shutdown, compute_rpcapi, - scheduler_client): + scheduler_client, host_list): super(MigrationTask, self).__init__(context, instance) self.clean_shutdown = clean_shutdown self.request_spec = request_spec @@ -119,6 +119,8 @@ class MigrationTask(base.TaskBase): self.compute_rpcapi = compute_rpcapi 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 # 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). return None - 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 = (self.instance.flavor.id != self.flavor.id - and 'resize' or 'migration') - migration.create() + # If this is a rescheduled migration, don't create a new record. + migration_type = ("resize" if self.instance.flavor.id != self.flavor.id + else "migration") + filters = {"instance_uuid": self.instance.uuid, + "migration_type": migration_type, + "status": "pre-migrating"} + migrations = objects.MigrationList.get_by_filters(self.context, + filters).objects + if migrations: + 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 @@ -194,17 +206,73 @@ class MigrationTask(base.TaskBase): self.request_spec.requested_destination = objects.Destination( 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() + self.request_spec.ensure_project_id(self.instance) - # For now, don't request alternates. A later patch in the series will - # modify migration to use alternates instead of calling the scheduler - # again. - selection_lists = self.scheduler_client.select_destinations( - self.context, self.request_spec, [self.instance.uuid], - return_objects=True, return_alternates=False) - # We only need the first item in the first list, as there is only one - # instance, and we don't care about any alternates. - selection = selection_lists[0][0] + # On an initial call to migrate, 'self.host_list' will be None, so we + # have to call the scheduler to get a list of acceptable hosts to + # migrate to. That list will consist of a selected host, along with + # zero or more alternates. On a reschedule, though, the alternates will + # be passed to this object and stored in 'self.host_list', so we can + # pop the first alternate from the list to use for the destination, and + # pass the remaining alternates to the compute. + if self.host_list is None: + 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) # context is not serializable @@ -222,12 +290,16 @@ class MigrationTask(base.TaskBase): # rpc fake driver. 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. self.compute_rpcapi.prep_resize( self.context, self.instance, legacy_spec['image'], self.flavor, host, migration, self.reservations, 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): if self._migration: diff --git a/nova/objects/service.py b/nova/objects/service.py index 996676db136e..b3587ed0999e 100644 --- a/nova/objects/service.py +++ b/nova/objects/service.py @@ -31,7 +31,7 @@ LOG = logging.getLogger(__name__) # 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 @@ -125,6 +125,8 @@ SERVICE_VERSION_HISTORY = ( # Version 27: Compute RPC version 4.20; adds multiattach argument to # reserve_block_device_name(). {'compute_rpc': '4.20'}, + # Version 28: Adds a 'host_list' parameter to prep_resize() + {'compute_rpc': '4.21'}, ) diff --git a/nova/tests/functional/test_servers.py b/nova/tests/functional/test_servers.py index a22db98aba41..1c93487d3cbb 100644 --- a/nova/tests/functional/test_servers.py +++ b/nova/tests/functional/test_servers.py @@ -32,6 +32,7 @@ from nova import db from nova import exception from nova import objects from nova.objects import block_device as block_device_obj +from nova.scheduler import weights from nova import test from nova.tests import fixtures as nova_fixtures from nova.tests.functional.api import client @@ -49,6 +50,17 @@ from nova import volume 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): api_major_version = 'v2' _force_delete_parameter = 'forceDelete' @@ -1369,7 +1381,7 @@ class ProviderUsageBaseTestCase(test.TestCase, nova.tests.unit.image.fake.stub_out_image_service(self) self.start_service('conductor') - self.start_service('scheduler') + self.scheduler_service = self.start_service('scheduler') self.addCleanup(nova.tests.unit.image.fake.FakeImageService_reset) fake_network.set_stub_network_methods(self) @@ -2699,6 +2711,114 @@ class ServerMovingTests(ProviderUsageBaseTestCase): # allocation which just leaves us with the original flavor. 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): """Test Server live migrations, which delete the migration or diff --git a/nova/tests/unit/compute/test_compute.py b/nova/tests/unit/compute/test_compute.py index 386da6e63941..dc6fa2b55a6c 100644 --- a/nova/tests/unit/compute/test_compute.py +++ b/nova/tests/unit/compute/test_compute.py @@ -145,7 +145,7 @@ def unify_instance(instance): class FakeComputeTaskAPI(object): def resize_instance(self, context, instance, extra_instance_updates, - scheduler_hint, flavor, reservations): + scheduler_hint, flavor, reservationsi, host_list=None): pass @@ -5526,7 +5526,8 @@ class ComputeTestCase(BaseTestCase, instance=instance, instance_type=new_instance_type_ref, 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 self.assertEqual(self.rt.compute_nodes[NODENAME].memory_mb_used, @@ -12314,11 +12315,11 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase): instance_type=self.instance_type, reservations=[], request_spec={}, filter_properties={}, node=None, - clean_shutdown=True) + clean_shutdown=True, host_list=None) mock_mig.assert_called_once_with(mock.ANY, 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('nova.compute.utils.notify_about_instance_action') @@ -12337,12 +12338,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase): exc_info = sys.exc_info() self.assertRaises(test.TestingException, 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( self.context, {}, {}, instance, 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( self.context, instance, 'fake-mini', action='resize', phase='error', exception=mock_res.side_effect) @@ -12363,12 +12364,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase): exc_info = sys.exc_info() self.assertRaises(test.TestingException, 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( self.context, {}, {}, instance, 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, "_log_original_error") @@ -12386,11 +12387,12 @@ class ComputeRescheduleResizeOrReraiseTestCase(BaseTestCase): self.compute._reschedule_resize_or_reraise( self.context, None, instance, exc_info, - self.instance_type, {}, {}) + self.instance_type, {}, {}, None) mock_res.assert_called_once_with(self.context, {}, {}, 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) diff --git a/nova/tests/unit/compute/test_rpcapi.py b/nova/tests/unit/compute/test_rpcapi.py index e7ea95c92b06..cb72b22d59bb 100644 --- a/nova/tests/unit/compute/test_rpcapi.py +++ b/nova/tests/unit/compute/test_rpcapi.py @@ -495,7 +495,8 @@ class ComputeRpcAPITestCase(test.NoDBTestCase): request_spec='fake_spec', filter_properties={'fakeprop': 'fakeval'}, 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') expected_args = {'instance_type': self.fake_flavor} self._test_compute_api('prep_resize', 'cast', expected_args, diff --git a/nova/tests/unit/conductor/tasks/test_migrate.py b/nova/tests/unit/conductor/tasks/test_migrate.py index 7514ca0b0331..4c9be65442db 100644 --- a/nova/tests/unit/conductor/tasks/test_migrate.py +++ b/nova/tests/unit/conductor/tasks/test_migrate.py @@ -55,7 +55,8 @@ class MigrationTaskTestCase(test.NoDBTestCase): self.request_spec, self.reservations, self.clean_shutdown, compute_rpcapi.ComputeAPI(), - scheduler_client.SchedulerClient()) + scheduler_client.SchedulerClient(), + host_list=None) @mock.patch('nova.objects.Service.get_minimum_version_multi') @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) task.scheduler_client.select_destinations.assert_called_once_with( 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] prep_resize_mock.assert_called_once_with( self.context, self.instance, legacy_request_spec['image'], self.flavor, selection.service_host, None, self.reservations, request_spec=legacy_request_spec, 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') self.assertIsNone(task._migration) + @mock.patch.object(objects.MigrationList, 'get_by_filters') @mock.patch('nova.scheduler.client.report.SchedulerReportClient') @mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename') @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(compute_rpcapi.ComputeAPI, 'prep_resize') 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): sel_dest_mock.return_value = self.host_lists az_mock.return_value = 'myaz' + gbf_mock.return_value = objects.MigrationList() if requested_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) task.scheduler_client.select_destinations.assert_called_once_with( 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] prep_resize_mock.assert_called_once_with( self.context, self.instance, legacy_request_spec['image'], self.flavor, selection.service_host, task._migration, self.reservations, request_spec=legacy_request_spec, 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') self.assertIsNotNone(task._migration) @@ -172,6 +175,7 @@ class MigrationTaskTestCase(test.NoDBTestCase): self.flavor.id = 3 self._test_execute() + @mock.patch.object(objects.MigrationList, 'get_by_filters') @mock.patch('nova.conductor.tasks.migrate.revert_allocation_for_migration') @mock.patch('nova.scheduler.client.report.SchedulerReportClient') @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') def test_execute_rollback(self, prep_resize_mock, sel_dest_mock, sig_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 az_mock.return_value = 'myaz' task = self._generate_task() gmv_mock.return_value = 23 + mock_gbf.return_value = objects.MigrationList() # We just need this hook point to set a uuid on the # migration before we use it for teardown diff --git a/nova/tests/unit/conductor/test_conductor.py b/nova/tests/unit/conductor/test_conductor.py index 4c9d35eaa70a..dc44b52abcc2 100644 --- a/nova/tests/unit/conductor/test_conductor.py +++ b/nova/tests/unit/conductor/test_conductor.py @@ -400,7 +400,7 @@ class _BaseTaskTestCase(object): # converted into 'migrate_server' when doing RPC. self.conductor.resize_instance( self.context, inst_obj, {}, scheduler_hint, flavor, [], - clean_shutdown) + clean_shutdown, host_list=None) else: self.conductor.migrate_server( self.context, inst_obj, scheduler_hint, @@ -2197,7 +2197,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase): self.conductor._cold_migrate, self.context, inst_obj, flavor, {}, [resvs], - True, None) + True, None, None) metadata_mock.assert_called_with({}) sig_mock.assert_called_once_with(self.context, fake_spec) 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.context, inst_obj, flavor, {}, [resvs], - True, None) + True, None, None) metadata_mock.assert_called_with({}) sig_mock.assert_called_once_with(self.context, fake_spec) 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, self.conductor._cold_migrate, self.context, inst_obj, flavor, {}, [resvs], - True, fake_spec) + True, fake_spec, None) self.assertIn('cold migrate', nvh.message) @mock.patch.object(utils, 'get_image_from_system_metadata') @@ -2322,7 +2322,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase): self.assertRaises(exc.UnsupportedPolicyException, 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} 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.conductor._cold_migrate, self.context, inst_obj, flavor, - {}, [resvs], True, None) + {}, [resvs], True, None, None) # Filter properties are populated during code execution 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) self.assertEqual(inst_obj.project_id, fake_spec.project_id) 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( self.context, inst_obj, legacy_request_spec['image'], flavor, hosts[0]['host'], None, [resvs], request_spec=legacy_request_spec, 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, 'migrate_server', updates, exc_info, fake_spec) @@ -2427,7 +2427,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase): # the new one self.assertNotEqual(flavor, fake_spec.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... self.assertEqual(flavor, fake_spec.flavor) @@ -2464,7 +2464,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase): nvh = self.assertRaises(exc.NoValidHost, self.conductor._cold_migrate, self.context, inst_obj, flavor_new, {}, - [resvs], True, fake_spec) + [resvs], True, fake_spec, None) self.assertIn('resize', nvh.message) @mock.patch('nova.objects.BuildRequest.get_by_instance_uuid')