Create class hierarchy for tasks in conductor

Conductor has 2 approaches to hande rpc api calls:
- in place (code located in manager itself)
- in tasks (for each operation creates new task class
where all logic is located)
Second approach allows to implement single responsibility
to each method.

In this patch, a class hierarchy for tasks is introduced.
It allows share same public interface between tasks.
Cold-migration/resize logic is also  moved to its
new task class. This change simplifies unit tests,
by making them cleaner.

Closes-Bug: #1468696

Change-Id: I820c4a8ccc58e140f24597a52f2e01444cd6d134
This commit is contained in:
Timofey Durakov 2015-06-24 13:05:06 +03:00
parent a74f07a923
commit 83787211b3
9 changed files with 452 additions and 268 deletions

View File

@ -34,6 +34,7 @@ from nova.compute import task_states
from nova.compute import utils as compute_utils
from nova.compute import vm_states
from nova.conductor.tasks import live_migrate
from nova.conductor.tasks import migrate
from nova.db import base
from nova import exception
from nova.i18n import _, _LE, _LW
@ -47,6 +48,7 @@ from nova import quota
from nova import rpc
from nova.scheduler import client as scheduler_client
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
from nova import utils
LOG = logging.getLogger(__name__)
@ -493,6 +495,7 @@ class ComputeTaskManager(base.Base):
super(ComputeTaskManager, self).__init__()
self.compute_rpcapi = compute_rpcapi.ComputeAPI()
self.image_api = image.API()
self.servicegroup_api = servicegroup.API()
self.scheduler_client = scheduler_client.SchedulerClient()
self.notifier = rpc.get_notifier('compute', CONF.host)
@ -545,17 +548,11 @@ class ComputeTaskManager(base.Base):
request_spec = scheduler_utils.build_request_spec(
context, image, [instance], instance_type=flavor)
quotas = objects.Quotas.from_reservations(context,
reservations,
instance=instance)
task = self._build_cold_migrate_task(context, instance, flavor,
filter_properties, request_spec,
reservations, clean_shutdown)
try:
scheduler_utils.setup_instance_group(context, request_spec,
filter_properties)
scheduler_utils.populate_retry(filter_properties, instance.uuid)
hosts = self.scheduler_client.select_destinations(
context, request_spec, filter_properties)
host_state = hosts[0]
task.execute()
except exception.NoValidHost as ex:
vm_state = instance.vm_state
if not vm_state:
@ -564,7 +561,6 @@ class ComputeTaskManager(base.Base):
self._set_vm_state_and_notify(context, instance.uuid,
'migrate_server',
updates, ex, request_spec)
quotas.rollback()
# if the flavor IDs match, it's migrate; otherwise resize
if flavor['id'] == instance.instance_type_id:
@ -581,21 +577,6 @@ class ComputeTaskManager(base.Base):
self._set_vm_state_and_notify(context, instance.uuid,
'migrate_server',
updates, ex, request_spec)
quotas.rollback()
try:
scheduler_utils.populate_filter_properties(filter_properties,
host_state)
# context is not serializable
filter_properties.pop('context', None)
(host, node) = (host_state['host'], host_state['nodename'])
self.compute_rpcapi.prep_resize(
context, image, instance,
flavor, host,
reservations, request_spec=request_spec,
filter_properties=filter_properties, node=node,
clean_shutdown=clean_shutdown)
except Exception as ex:
with excutils.save_and_reraise_exception():
updates = {'vm_state': instance.vm_state,
@ -603,7 +584,6 @@ class ComputeTaskManager(base.Base):
self._set_vm_state_and_notify(context, instance.uuid,
'migrate_server',
updates, ex, request_spec)
quotas.rollback()
def _set_vm_state_and_notify(self, context, instance_uuid, method, updates,
ex, request_spec):
@ -642,10 +622,11 @@ class ComputeTaskManager(base.Base):
migration.new_instance_type_id = instance.instance_type_id
migration.create()
task = self._build_live_migrate_task(context, instance, destination,
block_migration, disk_over_commit,
migration)
try:
live_migrate.execute(context, instance, destination,
block_migration, disk_over_commit,
migration)
task.execute()
except (exception.NoValidHost,
exception.ComputeServiceUnavailable,
exception.InvalidHypervisorType,
@ -674,6 +655,24 @@ class ComputeTaskManager(base.Base):
migration.save()
raise exception.MigrationError(reason=six.text_type(ex))
def _build_live_migrate_task(self, context, instance, destination,
block_migration, disk_over_commit, migration):
return live_migrate.LiveMigrationTask(context, instance,
destination, block_migration,
disk_over_commit, migration,
self.compute_rpcapi,
self.servicegroup_api,
self.scheduler_client)
def _build_cold_migrate_task(self, context, instance, flavor,
filter_properties, request_spec, reservations,
clean_shutdown):
return migrate.MigrationTask(context, instance, flavor,
filter_properties, request_spec,
reservations, clean_shutdown,
self.compute_rpcapi,
self.scheduler_client)
def build_instances(self, context, instances, image, filter_properties,
admin_password, injected_files, requested_networks,
security_groups, block_device_mapping=None, legacy_bdm=True):

View File

@ -0,0 +1,56 @@
# 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.
import abc
import functools
from oslo_utils import excutils
import six
def rollback_wrapper(original):
@functools.wraps(original)
def wrap(self):
try:
return original(self)
except Exception:
with excutils.save_and_reraise_exception():
self.rollback()
return wrap
@six.add_metaclass(abc.ABCMeta)
class TaskBase(object):
def __init__(self, context, instance):
self.context = context
self.instance = instance
@rollback_wrapper
def execute(self):
"""Run task's logic, written in _execute() method
"""
return self._execute()
@abc.abstractmethod
def _execute(self):
"""Descendants should place task's logic here, while resource
initialization should be performed over __init__
"""
pass
def rollback(self):
"""Rollback failed task
Descendants should implement this method to allow task user to
rollback status to state before execute method was call
"""
pass

View File

@ -14,14 +14,11 @@ from oslo_config import cfg
from oslo_log import log as logging
from nova.compute import power_state
from nova.compute import rpcapi as compute_rpcapi
from nova.conductor.tasks import base
from nova import exception
from nova.i18n import _
from nova import image
from nova import objects
from nova.scheduler import client as scheduler_client
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
from nova import utils
LOG = logging.getLogger(__name__)
@ -36,23 +33,23 @@ CONF = cfg.CONF
CONF.register_opt(migrate_opt)
class LiveMigrationTask(object):
class LiveMigrationTask(base.TaskBase):
def __init__(self, context, instance, destination,
block_migration, disk_over_commit, migration):
self.context = context
self.instance = instance
block_migration, disk_over_commit, migration, compute_rpcapi,
servicegroup_api, scheduler_client):
super(LiveMigrationTask, self).__init__(context, instance)
self.destination = destination
self.block_migration = block_migration
self.disk_over_commit = disk_over_commit
self.migration = migration
self.source = instance.host
self.migrate_data = None
self.compute_rpcapi = compute_rpcapi.ComputeAPI()
self.servicegroup_api = servicegroup.API()
self.scheduler_client = scheduler_client.SchedulerClient()
self.image_api = image.API()
def execute(self):
self.compute_rpcapi = compute_rpcapi
self.servicegroup_api = servicegroup_api
self.scheduler_client = scheduler_client
def _execute(self):
self._check_instance_is_active()
self._check_host_is_up(self.source)
@ -79,7 +76,7 @@ class LiveMigrationTask(object):
# calls, since this class currently makes no state changes,
# except to call the compute method, that has no matching
# rollback call right now.
raise NotImplementedError()
pass
def _check_instance_is_active(self):
if self.instance.power_state not in (power_state.RUNNING,
@ -185,14 +182,3 @@ class LiveMigrationTask(object):
% {'max_retries': retries,
'instance_uuid': self.instance.uuid})
raise exception.MaxRetriesExceeded(reason=msg)
def execute(context, instance, destination,
block_migration, disk_over_commit, migration):
task = LiveMigrationTask(context, instance,
destination,
block_migration,
disk_over_commit,
migration)
# TODO(johngarbutt) create a superclass that contains a safe_execute call
return task.execute()

View File

@ -0,0 +1,60 @@
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from nova.conductor.tasks import base
from nova import objects
from nova.scheduler import utils as scheduler_utils
class MigrationTask(base.TaskBase):
def __init__(self, context, instance, flavor, filter_properties,
request_spec, reservations, clean_shutdown, compute_rpcapi,
scheduler_client):
super(MigrationTask, self).__init__(context, instance)
self.clean_shutdown = clean_shutdown
self.request_spec = request_spec
self.reservations = reservations
self.filter_properties = filter_properties
self.flavor = flavor
self.quotas = None
self.compute_rpcapi = compute_rpcapi
self.scheduler_client = scheduler_client
def _execute(self):
image = self.request_spec.get('image')
self.quotas = objects.Quotas.from_reservations(self.context,
self.reservations,
instance=self.instance)
scheduler_utils.setup_instance_group(self.context, self.request_spec,
self.filter_properties)
scheduler_utils.populate_retry(self.filter_properties,
self.instance.uuid)
hosts = self.scheduler_client.select_destinations(
self.context, self.request_spec, self.filter_properties)
host_state = hosts[0]
scheduler_utils.populate_filter_properties(self.filter_properties,
host_state)
# context is not serializable
self.filter_properties.pop('context', None)
(host, node) = (host_state['host'], host_state['nodename'])
self.compute_rpcapi.prep_resize(
self.context, image, self.instance, self.flavor, host,
self.reservations, request_spec=self.request_spec,
filter_properties=self.filter_properties, node=node,
clean_shutdown=self.clean_shutdown)
def rollback(self):
if self.quotas:
self.quotas.rollback()

View File

@ -350,8 +350,9 @@ class CellsConductorAPIRPCRedirect(test.NoDBTestCase):
@mock.patch.object(objects.Instance, 'save')
@mock.patch.object(flavors, 'extract_flavor')
@mock.patch.object(compute_api.API, '_check_auto_disk_config')
def test_resize_instance(self, _check, _extract, _save, _upsize, _reserve,
_cells, _record):
@mock.patch.object(objects.BlockDeviceMappingList, 'get_by_instance_uuid')
def test_resize_instance(self, _bdms, _check, _extract, _save, _upsize,
_reserve, _cells, _record):
flavor = objects.Flavor(**test_flavor.fake_flavor)
_extract.return_value = flavor
orig_system_metadata = {}

View File

@ -0,0 +1,53 @@
# 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.
import mock
from nova.conductor.tasks import base
from nova import test
class FakeTask(base.TaskBase):
def __init__(self, context, instance, fail=False):
super(FakeTask, self).__init__(context, instance)
self.fail = fail
def _execute(self):
if self.fail:
raise Exception
else:
pass
class TaskBaseTestCase(test.NoDBTestCase):
def setUp(self):
super(TaskBaseTestCase, self).setUp()
self.task = FakeTask(mock.MagicMock(), mock.MagicMock())
@mock.patch.object(FakeTask, 'rollback')
def test_wrapper_exception(self, fake_rollback):
self.task.fail = True
try:
self.task.execute()
except Exception:
pass
fake_rollback.assert_called_once_with()
@mock.patch.object(FakeTask, 'rollback')
def test_wrapper_no_exception(self, fake_rollback):
try:
self.task.execute()
except Exception:
pass
self.assertFalse(fake_rollback.called)

View File

@ -14,10 +14,13 @@ import mock
from mox3 import mox
from nova.compute import power_state
from nova.compute import rpcapi as compute_rpcapi
from nova.conductor.tasks import live_migrate
from nova import exception
from nova import objects
from nova.scheduler import client as scheduler_client
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
from nova import test
from nova.tests.unit import fake_instance
from nova import utils
@ -48,7 +51,8 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
def _generate_task(self):
self.task = live_migrate.LiveMigrationTask(self.context,
self.instance, self.destination, self.block_migration,
self.disk_over_commit, self.migration)
self.disk_over_commit, self.migration, compute_rpcapi.ComputeAPI(),
servicegroup.API(), scheduler_client.SchedulerClient())
def test_execute_with_destination(self):
self.mox.StubOutWithMock(self.task, '_check_host_is_up')
@ -406,6 +410,3 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
self.mox.ReplayAll()
self.assertRaises(exception.NoValidHost, self.task._find_destination)
def test_not_implemented_rollback(self):
self.assertRaises(NotImplementedError, self.task.rollback)

View File

@ -0,0 +1,87 @@
# 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.
import mock
from nova.compute import rpcapi as compute_rpcapi
from nova.conductor.tasks import migrate
from nova import objects
from nova.objects import base as obj_base
from nova.scheduler import client as scheduler_client
from nova.scheduler import utils as scheduler_utils
from nova import test
from nova.tests.unit.conductor.test_conductor import FakeContext
from nova.tests.unit import fake_flavor
from nova.tests.unit import fake_instance
class MigrationTaskTestCase(test.NoDBTestCase):
def setUp(self):
super(MigrationTaskTestCase, self).setUp()
self.user_id = 'fake'
self.project_id = 'fake'
self.context = FakeContext(self.user_id, self.project_id)
inst = fake_instance.fake_db_instance(image_ref='image_ref')
self.instance = objects.Instance._from_db_object(
self.context, objects.Instance(), inst, [])
self.instance.system_metadata = {'image_hw_disk_bus': 'scsi'}
self.flavor = fake_flavor.fake_flavor_obj(self.context)
self.flavor.extra_specs = {'extra_specs': 'fake'}
self.request_spec = {'instance_type':
obj_base.obj_to_primitive(self.flavor),
'instance_properties': {},
'image': 'image'}
self.hosts = [dict(host='host1', nodename=None, limits={})]
self.filter_properties = {'limits': {}, 'retry': {'num_attempts': 1,
'hosts': [['host1', None]]}}
self.reservations = []
self.clean_shutdown = True
def _generate_task(self):
return migrate.MigrationTask(self.context, self.instance, self.flavor,
self.filter_properties, self.request_spec,
self.reservations, self.clean_shutdown,
compute_rpcapi.ComputeAPI(),
scheduler_client.SchedulerClient())
@mock.patch.object(scheduler_utils, 'build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
@mock.patch.object(objects.Quotas, 'from_reservations')
def test_execute(self, quotas_mock, prep_resize_mock,
sel_dest_mock, sig_mock, brs_mock):
brs_mock.return_value = self.request_spec
sel_dest_mock.return_value = self.hosts
task = self._generate_task()
task.execute()
quotas_mock.assert_called_once_with(self.context, self.reservations,
instance=self.instance)
sig_mock.assert_called_once_with(self.context, self.request_spec,
self.filter_properties)
task.scheduler_client.select_destinations.assert_called_once_with(
self.context, self.request_spec, self.filter_properties)
prep_resize_mock.assert_called_once_with(
self.context, 'image', self.instance, self.flavor,
self.hosts[0]['host'], self.reservations,
request_spec=self.request_spec,
filter_properties=self.filter_properties,
node=self.hosts[0]['nodename'], clean_shutdown=self.clean_shutdown)
self.assertFalse(quotas_mock.return_value.rollback.called)
def test_rollback(self):
task = self._generate_task()
task.quotas = mock.MagicMock()
task.rollback()
task.quotas.rollback.assert_called_once_with()

View File

@ -27,6 +27,7 @@ import six
from nova.api.ec2 import ec2utils
from nova.compute import arch
from nova.compute import flavors
from nova.compute import rpcapi as compute_rpcapi
from nova.compute import task_states
from nova.compute import utils as compute_utils
from nova.compute import vm_states
@ -35,6 +36,7 @@ from nova.conductor import api as conductor_api
from nova.conductor import manager as conductor_manager
from nova.conductor import rpcapi as conductor_rpcapi
from nova.conductor.tasks import live_migrate
from nova.conductor.tasks import migrate
from nova import context
from nova import db
from nova.db.sqlalchemy import models
@ -45,9 +47,8 @@ from nova import objects
from nova.objects import base as obj_base
from nova.objects import block_device as block_device_obj
from nova.objects import fields
from nova.objects import quotas as quotas_obj
from nova import quota
from nova import rpc
from nova.scheduler import client as scheduler_client
from nova.scheduler import utils as scheduler_utils
from nova import test
from nova.tests.unit import cast_as_call
@ -942,14 +943,11 @@ class _BaseTaskTestCase(object):
self.context, objects.Instance(), inst, [])
migration = migobj()
self.mox.StubOutWithMock(live_migrate, 'execute')
live_migrate.execute(self.context,
mox.IsA(objects.Instance),
'destination',
'block_migration',
'disk_over_commit',
migration)
self.mox.StubOutWithMock(live_migrate.LiveMigrationTask, 'execute')
task = self.conductor_manager._build_live_migrate_task(
self.context, inst_obj, 'destination', 'block_migration',
'disk_over_commit', migration)
task.execute()
self.mox.ReplayAll()
if isinstance(self.conductor, (conductor_api.ComputeTaskAPI,
@ -970,17 +968,16 @@ class _BaseTaskTestCase(object):
def _test_cold_migrate(self, clean_shutdown=True):
self.mox.StubOutWithMock(utils, 'get_image_from_system_metadata')
self.mox.StubOutWithMock(scheduler_utils, 'build_request_spec')
self.mox.StubOutWithMock(scheduler_utils, 'setup_instance_group')
self.mox.StubOutWithMock(
self.conductor_manager.compute_rpcapi, 'prep_resize')
self.mox.StubOutWithMock(self.conductor_manager.scheduler_client,
'select_destinations')
self.mox.StubOutWithMock(migrate.MigrationTask, 'execute')
inst = fake_instance.fake_db_instance(image_ref='image_ref')
inst_obj = objects.Instance._from_db_object(
self.context, objects.Instance(), inst, [])
inst_obj.system_metadata = {'image_hw_disk_bus': 'scsi'}
flavor = flavors.get_default_flavor()
flavor.extra_specs = {'extra_specs': 'fake'}
filter_properties = {'limits': {},
'retry': {'num_attempts': 1,
'hosts': [['host1', None]]}}
request_spec = {'instance_type': obj_base.obj_to_primitive(flavor),
'instance_properties': {}}
utils.get_image_from_system_metadata(
@ -990,24 +987,10 @@ class _BaseTaskTestCase(object):
self.context, 'image',
[mox.IsA(objects.Instance)],
instance_type=mox.IsA(objects.Flavor)).AndReturn(request_spec)
scheduler_utils.setup_instance_group(self.context, request_spec, {})
hosts = [dict(host='host1', nodename=None, limits={})]
self.conductor_manager.scheduler_client.select_destinations(
self.context, request_spec,
{'retry': {'num_attempts': 1, 'hosts': []}}).AndReturn(hosts)
filter_properties = {'limits': {},
'retry': {'num_attempts': 1,
'hosts': [['host1', None]]}}
self.conductor_manager.compute_rpcapi.prep_resize(
self.context, 'image', mox.IsA(objects.Instance),
mox.IsA(objects.Flavor), 'host1', [], request_spec=request_spec,
filter_properties=filter_properties, node=None,
clean_shutdown=clean_shutdown)
task = self.conductor_manager._build_cold_migrate_task(
self.context, inst_obj, flavor, filter_properties,
request_spec, [], clean_shutdown=clean_shutdown)
task.execute()
self.mox.ReplayAll()
scheduler_hint = {'filter_properties': {}}
@ -1538,15 +1521,18 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
vm_state=vm_states.ACTIVE)
inst_obj = objects.Instance._from_db_object(
self.context, objects.Instance(), instance, [])
self.mox.StubOutWithMock(live_migrate, 'execute')
self.mox.StubOutWithMock(live_migrate.LiveMigrationTask, 'execute')
self.mox.StubOutWithMock(scheduler_utils,
'set_vm_state_and_notify')
migration = migobj()
live_migrate.execute(self.context, mox.IsA(objects.Instance),
'destination', 'block_migration',
'disk_over_commit', migration).AndRaise(ex)
task = self.conductor._build_live_migrate_task(self.context, inst_obj,
'destination',
'block_migration',
'disk_over_commit',
migration)
task.execute().AndRaise(ex)
scheduler_utils.set_vm_state_and_notify(self.context,
inst_obj.uuid,
@ -1572,15 +1558,16 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
vm_state=vm_states.ACTIVE)
inst_obj = objects.Instance._from_db_object(
self.context, objects.Instance(), instance, [])
self.mox.StubOutWithMock(live_migrate, 'execute')
self.mox.StubOutWithMock(live_migrate.LiveMigrationTask, 'execute')
self.mox.StubOutWithMock(scheduler_utils,
'set_vm_state_and_notify')
ex = exc.InvalidCPUInfo(reason="invalid cpu info.")
live_migrate.execute(self.context, mox.IsA(objects.Instance),
'destination', 'block_migration',
'disk_over_commit',
mox.IsA(objects.Migration)).AndRaise(ex)
task = self.conductor._build_live_migrate_task(
self.context, inst_obj, 'destination', 'block_migration',
'disk_over_commit', mox.IsA(objects.Migration))
task.execute().AndRaise(ex)
scheduler_utils.set_vm_state_and_notify(self.context,
inst_obj.uuid,
@ -1617,7 +1604,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self._test_migrate_server_deals_with_expected_exceptions(ex)
@mock.patch.object(scheduler_utils, 'set_vm_state_and_notify')
@mock.patch.object(live_migrate, 'execute')
@mock.patch.object(live_migrate.LiveMigrationTask, 'execute')
def test_migrate_server_deals_with_unexpected_exceptions(self,
mock_live_migrate, mock_set_state):
expected_ex = IOError('fake error')
@ -1653,7 +1640,17 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.conductor._set_vm_state_and_notify(
self.context, 1, 'method', 'updates', 'ex', 'request_spec')
def test_cold_migrate_no_valid_host_back_in_active_state(self):
@mock.patch.object(scheduler_utils, 'build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(objects.Quotas, 'from_reservations')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
def test_cold_migrate_no_valid_host_back_in_active_state(
self, rollback_mock, notify_mock, select_dest_mock, quotas_mock,
metadata_mock, sig_mock, brs_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -1667,56 +1664,41 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
filter_props = dict(context=None)
resvs = 'fake-resvs'
image = 'fake-image'
self.mox.StubOutWithMock(utils, 'get_image_from_system_metadata')
self.mox.StubOutWithMock(scheduler_utils, 'build_request_spec')
self.mox.StubOutWithMock(scheduler_utils, 'setup_instance_group')
self.mox.StubOutWithMock(self.conductor.scheduler_client,
'select_destinations')
self.mox.StubOutWithMock(self.conductor,
'_set_vm_state_and_notify')
self.mox.StubOutWithMock(quota.QUOTAS, 'rollback')
utils.get_image_from_system_metadata(
inst_obj.system_metadata).AndReturn(image)
scheduler_utils.build_request_spec(
self.context, image, [inst_obj],
instance_type=flavor).AndReturn(request_spec)
scheduler_utils.setup_instance_group(self.context, request_spec,
filter_props)
metadata_mock.return_value = image
brs_mock.return_value = request_spec
exc_info = exc.NoValidHost(reason="")
self.conductor.scheduler_client.select_destinations(
self.context, request_spec,
filter_props).AndRaise(exc_info)
select_dest_mock.side_effect = exc_info
updates = {'vm_state': vm_states.ACTIVE,
'task_state': None}
self.conductor._set_vm_state_and_notify(self.context,
inst_obj.uuid,
'migrate_server',
updates, exc_info,
request_spec)
# NOTE(mriedem): Validate that the quota rollback is using
# the correct project_id and user_id.
project_id, user_id = quotas_obj.ids_from_instance(self.context,
inst_obj)
quota.QUOTAS.rollback(self.context, [resvs], project_id=project_id,
user_id=user_id)
self.mox.ReplayAll()
self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate,
self.context, inst_obj,
flavor, filter_props, [resvs],
clean_shutdown=True)
metadata_mock.assert_called_with({})
brs_mock.assert_called_once_with(self.context, image,
[inst_obj],
instance_type=flavor)
quotas_mock.assert_called_once_with(self.context, [resvs],
instance=inst_obj)
sig_mock.assert_called_once_with(self.context, request_spec,
filter_props)
notify_mock.assert_called_once_with(self.context, inst_obj.uuid,
'migrate_server', updates,
exc_info, request_spec)
rollback_mock.assert_called_once_with()
def test_cold_migrate_no_valid_host_back_in_stopped_state(self):
@mock.patch.object(scheduler_utils, 'build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(objects.Quotas, 'from_reservations')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
def test_cold_migrate_no_valid_host_back_in_stopped_state(
self, rollback_mock, notify_mock, select_dest_mock, quotas_mock,
metadata_mock, sig_mock, brs_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -1725,58 +1707,36 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
system_metadata={},
uuid='fake',
user_id='fake')
image = 'fake-image'
request_spec = dict(instance_type=dict(extra_specs=dict()),
instance_properties=dict())
instance_properties=dict(),
image=image)
filter_props = dict(context=None)
resvs = 'fake-resvs'
image = 'fake-image'
self.mox.StubOutWithMock(utils, 'get_image_from_system_metadata')
self.mox.StubOutWithMock(scheduler_utils, 'build_request_spec')
self.mox.StubOutWithMock(scheduler_utils, 'setup_instance_group')
self.mox.StubOutWithMock(self.conductor.scheduler_client,
'select_destinations')
self.mox.StubOutWithMock(self.conductor,
'_set_vm_state_and_notify')
self.mox.StubOutWithMock(quota.QUOTAS, 'rollback')
utils.get_image_from_system_metadata(
inst_obj.system_metadata).AndReturn(image)
scheduler_utils.build_request_spec(
self.context, image, [inst_obj],
instance_type=flavor).AndReturn(request_spec)
scheduler_utils.setup_instance_group(self.context, request_spec,
filter_props)
metadata_mock.return_value = image
brs_mock.return_value = request_spec
exc_info = exc.NoValidHost(reason="")
self.conductor.scheduler_client.select_destinations(
self.context, request_spec,
filter_props).AndRaise(exc_info)
select_dest_mock.side_effect = exc_info
updates = {'vm_state': vm_states.STOPPED,
'task_state': None}
self.conductor._set_vm_state_and_notify(self.context,
inst_obj.uuid,
'migrate_server',
updates, exc_info,
request_spec)
# NOTE(mriedem): Validate that the quota rollback is using
# the correct project_id and user_id.
project_id, user_id = quotas_obj.ids_from_instance(self.context,
inst_obj)
quota.QUOTAS.rollback(self.context, [resvs], project_id=project_id,
user_id=user_id)
self.mox.ReplayAll()
self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate, self.context,
inst_obj, flavor, filter_props, [resvs],
clean_shutdown=True)
self.conductor._cold_migrate,
self.context, inst_obj,
flavor, filter_props, [resvs],
clean_shutdown=True)
metadata_mock.assert_called_with({})
brs_mock.assert_called_once_with(self.context, image,
[inst_obj],
instance_type=flavor)
quotas_mock.assert_called_once_with(self.context, [resvs],
instance=inst_obj)
sig_mock.assert_called_once_with(self.context, request_spec,
filter_props)
notify_mock.assert_called_once_with(self.context, inst_obj.uuid,
'migrate_server', updates,
exc_info, request_spec)
rollback_mock.assert_called_once_with()
def test_cold_migrate_no_valid_host_error_msg(self):
flavor = flavors.get_flavor_by_name('m1.tiny')
@ -1798,13 +1758,13 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
return_value=image),
mock.patch.object(scheduler_utils, 'build_request_spec',
return_value=request_spec),
mock.patch.object(scheduler_utils, 'setup_instance_group',
return_value=False),
mock.patch.object(self.conductor, '_set_vm_state_and_notify'),
mock.patch.object(self.conductor.scheduler_client,
'select_destinations',
side_effect=exc.NoValidHost(reason=""))
) as (image_mock, brs_mock, sig_mock, set_vm_mock, select_dest_mock):
mock.patch.object(migrate.MigrationTask,
'execute',
side_effect=exc.NoValidHost(reason="")),
mock.patch.object(migrate.MigrationTask, 'rollback')
) as (image_mock, brs_mock, set_vm_mock, task_execute_mock,
task_rollback_mock):
nvh = self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate, self.context,
inst_obj, flavor, filter_props, [resvs],
@ -1813,12 +1773,14 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch('nova.scheduler.utils.build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(migrate.MigrationTask, 'execute')
@mock.patch.object(migrate.MigrationTask, 'rollback')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
def test_cold_migrate_no_valid_host_in_group(self,
set_vm_mock,
sig_mock,
task_rollback_mock,
task_exec_mock,
brs_mock,
image_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
@ -1838,7 +1800,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
image_mock.return_value = image
brs_mock.return_value = request_spec
sig_mock.side_effect = exception
task_exec_mock.side_effect = exception
self.assertRaises(exc.UnsupportedPolicyException,
self.conductor._cold_migrate, self.context,
@ -1850,7 +1812,18 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
'migrate_server', updates,
exception, request_spec)
def test_cold_migrate_exception_host_in_error_state_and_raise(self):
@mock.patch.object(scheduler_utils, 'build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(objects.Quotas, 'from_reservations')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
def test_cold_migrate_exception_host_in_error_state_and_raise(
self, prep_resize_mock, rollback_mock, notify_mock,
select_dest_mock, quotas_mock, metadata_mock, sig_mock, brs_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -1859,81 +1832,49 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
system_metadata={},
uuid='fake',
user_id='fake')
image = 'fake-image'
request_spec = dict(instance_type=dict(),
instance_properties=dict())
instance_properties=dict(),
image=image)
filter_props = dict(context=None)
resvs = 'fake-resvs'
image = 'fake-image'
hosts = [dict(host='host1', nodename=None, limits={})]
self.mox.StubOutWithMock(utils, 'get_image_from_system_metadata')
self.mox.StubOutWithMock(scheduler_utils, 'build_request_spec')
self.mox.StubOutWithMock(scheduler_utils, 'setup_instance_group')
self.mox.StubOutWithMock(self.conductor.scheduler_client,
'select_destinations')
self.mox.StubOutWithMock(scheduler_utils,
'populate_filter_properties')
self.mox.StubOutWithMock(self.conductor.compute_rpcapi,
'prep_resize')
self.mox.StubOutWithMock(self.conductor,
'_set_vm_state_and_notify')
self.mox.StubOutWithMock(quota.QUOTAS, 'rollback')
utils.get_image_from_system_metadata(
inst_obj.system_metadata).AndReturn(image)
scheduler_utils.build_request_spec(
self.context, image, [inst_obj],
instance_type='flavor').AndReturn(request_spec)
scheduler_utils.setup_instance_group(self.context, request_spec,
filter_props)
expected_filter_props = {'retry': {'num_attempts': 1,
'hosts': []},
'context': None}
self.conductor.scheduler_client.select_destinations(
self.context, request_spec,
expected_filter_props).AndReturn(hosts)
scheduler_utils.populate_filter_properties(filter_props,
hosts[0])
metadata_mock.return_value = image
brs_mock.return_value = request_spec
exc_info = test.TestingException('something happened')
expected_filter_props = {'retry': {'num_attempts': 1,
'hosts': []}}
self.conductor.compute_rpcapi.prep_resize(
self.context, image, inst_obj,
'flavor', hosts[0]['host'], [resvs],
request_spec=request_spec,
filter_properties=expected_filter_props,
node=hosts[0]['nodename'],
clean_shutdown=True).AndRaise(exc_info)
select_dest_mock.return_value = hosts
updates = {'vm_state': vm_states.STOPPED,
'task_state': None}
self.conductor._set_vm_state_and_notify(self.context,
inst_obj.uuid,
'migrate_server',
updates, exc_info,
request_spec)
# NOTE(mriedem): Validate that the quota rollback is using
# the correct project_id and user_id.
project_id, user_id = quotas_obj.ids_from_instance(self.context,
inst_obj)
quota.QUOTAS.rollback(self.context, [resvs], project_id=project_id,
user_id=user_id)
self.mox.ReplayAll()
prep_resize_mock.side_effect = exc_info
self.assertRaises(test.TestingException,
self.conductor._cold_migrate,
self.context, inst_obj, 'flavor',
self.context, inst_obj, flavor,
filter_props, [resvs],
clean_shutdown=True)
metadata_mock.assert_called_with({})
brs_mock.assert_called_once_with(self.context, image,
[inst_obj],
instance_type=flavor)
quotas_mock.assert_called_once_with(self.context, [resvs],
instance=inst_obj)
sig_mock.assert_called_once_with(self.context, request_spec,
filter_props)
select_dest_mock.assert_called_once_with(
self.context, request_spec, filter_props)
prep_resize_mock.assert_called_once_with(
self.context, image, inst_obj, flavor,
hosts[0]['host'], [resvs],
request_spec=request_spec,
filter_properties=filter_props,
node=hosts[0]['nodename'], clean_shutdown=True)
notify_mock.assert_called_once_with(self.context, inst_obj.uuid,
'migrate_server', updates,
exc_info, request_spec)
rollback_mock.assert_called_once_with()
def test_resize_no_valid_host_error_msg(self):
flavor = flavors.get_flavor_by_name('m1.tiny')
flavor_new = flavors.get_flavor_by_name('m1.small')
@ -1956,13 +1897,13 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
return_value=image),
mock.patch.object(scheduler_utils, 'build_request_spec',
return_value=request_spec),
mock.patch.object(scheduler_utils, 'setup_instance_group',
return_value=False),
mock.patch.object(self.conductor, '_set_vm_state_and_notify'),
mock.patch.object(self.conductor.scheduler_client,
'select_destinations',
side_effect=exc.NoValidHost(reason=""))
) as (image_mock, brs_mock, sig_mock, vm_st_mock, select_dest_mock):
mock.patch.object(migrate.MigrationTask,
'execute',
side_effect=exc.NoValidHost(reason="")),
mock.patch.object(migrate.MigrationTask, 'rollback')
) as (image_mock, brs_mock, vm_st_mock, task_execute_mock,
task_rb_mock):
nvh = self.assertRaises(exc.NoValidHost,
self.conductor._cold_migrate, self.context,
inst_obj, flavor_new, filter_props,