Make conductor ask scheduler to limit migrates to same cell

This makes changes to the host manager to collect information
about the cell that a particular HostState is in, and makes
FilterScheduler and CachingScheduler properly filter for a
requested cell when appropriate.

Related to blueprint cells-aware-api
Change-Id: Ic6dc46183676a91cf9037726316b677a29e2d342
This commit is contained in:
Dan Smith 2017-02-24 07:27:35 -08:00
parent 9dce214e06
commit 9370ccf055
12 changed files with 272 additions and 100 deletions

View File

@ -10,6 +10,7 @@
# License for the specific language governing permissions and limitations
# under the License.
from oslo_log import log as logging
from oslo_serialization import jsonutils
from nova import availability_zones
@ -17,6 +18,8 @@ from nova.conductor.tasks import base
from nova import objects
from nova.scheduler import utils as scheduler_utils
LOG = logging.getLogger(__name__)
class MigrationTask(base.TaskBase):
def __init__(self, context, instance, flavor,
@ -49,6 +52,23 @@ class MigrationTask(base.TaskBase):
# if we want to make sure that the next destination
# is not forced to be the original host
self.request_spec.reset_forced_destinations()
# NOTE(danms): Right now we only support migrate to the same
# cell as the current instance, so request that the scheduler
# limit thusly.
instance_mapping = objects.InstanceMapping.get_by_instance_uuid(
self.context, self.instance.uuid)
LOG.debug('Requesting cell %(cell)s while migrating',
{'cell': instance_mapping.cell_mapping.identity},
instance=self.instance)
if ('requested_destination' in self.request_spec and
self.request_spec.requested_destination):
self.request_spec.requested_destination.cell = (
instance_mapping.cell_mapping)
else:
self.request_spec.requested_destination = objects.Destination(
cell=instance_mapping.cell_mapping)
hosts = self.scheduler_client.select_destinations(
self.context, self.request_spec)
host_state = hosts[0]

View File

@ -13,6 +13,9 @@
# License for the specific language governing permissions and limitations
# under the License.
import collections
import itertools
from nova.scheduler import filter_scheduler
@ -70,8 +73,27 @@ class CachingScheduler(filter_scheduler.FilterScheduler):
# Rather than raise an error, we fetch the list of hosts.
self.all_host_states = self._get_up_hosts(context)
return self.all_host_states
if (spec_obj and 'requested_destination' in spec_obj and
spec_obj.requested_destination and
'cell' in spec_obj.requested_destination):
only_cell = spec_obj.requested_destination.cell
else:
only_cell = None
if only_cell:
return self.all_host_states.get(only_cell.uuid, [])
else:
return itertools.chain.from_iterable(
self.all_host_states.values())
def _get_up_hosts(self, context):
all_hosts_iterator = self.host_manager.get_all_host_states(context)
return list(all_hosts_iterator)
# NOTE(danms): This could be more efficient if host_manager returned
# a dict for us. However, it flattens the information for the more
# mainline FilterScheduler. Since CachingScheduler goes away soonish,
# and since we were already iterating the full host list on every
# refresh, just build the dict here for now.
hosts_by_cell = collections.defaultdict(list)
for host in all_hosts_iterator:
hosts_by_cell[host.cell_uuid].append(host)
return hosts_by_cell

View File

@ -60,6 +60,8 @@ class ChanceScheduler(driver.Scheduler):
num_instances = spec_obj.num_instances
# NOTE(timello): Returns a list of dicts with 'host', 'nodename' and
# 'limits' as keys for compatibility with filter_scheduler.
# TODO(danms): This needs to be extended to support multiple cells
# and limiting the destination scope to a single requested cell
dests = []
for i in range(num_instances):
host = self._schedule(context, CONF.compute_topic, spec_obj)

View File

@ -176,4 +176,5 @@ class FilterScheduler(driver.Scheduler):
return []
compute_uuids = [rp.uuid for rp in rps]
return self.host_manager.get_host_states_by_uuids(context,
compute_uuids)
compute_uuids,
spec_obj)

View File

@ -106,7 +106,7 @@ class HostState(object):
previously used and lock down access.
"""
def __init__(self, host, node):
def __init__(self, host, node, cell_uuid):
self.host = host
self.nodename = node
self._lock_name = (host, node)
@ -152,6 +152,9 @@ class HostState(object):
self.cpu_allocation_ratio = None
self.disk_allocation_ratio = None
# Host cell (v2) membership
self.cell_uuid = cell_uuid
self.updated = None
def update(self, compute=None, service=None, aggregates=None,
@ -330,8 +333,8 @@ class HostManager(object):
"""Base HostManager class."""
# Can be overridden in a subclass
def host_state_cls(self, host, node, **kwargs):
return HostState(host, node)
def host_state_cls(self, host, node, cell, **kwargs):
return HostState(host, node, cell)
def __init__(self):
self.host_state_map = {}
@ -543,7 +546,7 @@ class HostManager(object):
force_nodes = spec_obj.force_nodes or []
requested_node = spec_obj.requested_destination
if requested_node is not None:
if requested_node is not None and 'host' in requested_node:
# NOTE(sbauza): Reduce a potentially long set of hosts as much as
# possible to any requested destination nodes before passing the
# list to the filters
@ -577,24 +580,21 @@ class HostManager(object):
return self.weight_handler.get_weighed_objects(self.weighers,
hosts, spec_obj)
def _get_computes_all_cells(self, context, compute_uuids=None):
if not self.cells:
# NOTE(danms): global list of cells cached forever right now
self.cells = objects.CellMappingList.get_all(context)
LOG.debug('Found %(count)i cells: %(cells)s',
{'count': len(self.cells),
'cells': ', '.join([c.uuid for c in self.cells])})
compute_nodes = []
def _get_computes_for_cells(self, context, cells, compute_uuids=None):
"""Returns: a cell-uuid keyed dict of compute node lists."""
compute_nodes = collections.defaultdict(list)
services = {}
for cell in self.cells:
for cell in cells:
LOG.debug('Getting compute nodes and services for cell %(cell)s',
{'cell': cell.identity})
with context_module.target_cell(context, cell):
if compute_uuids is None:
compute_nodes.extend(objects.ComputeNodeList.get_all(
context))
compute_nodes[cell.uuid].extend(
objects.ComputeNodeList.get_all(
context))
else:
compute_nodes.extend(
compute_nodes[cell.uuid].extend(
objects.ComputeNodeList.get_all_by_uuids(
context, compute_uuids))
services.update(
@ -604,9 +604,31 @@ class HostManager(object):
include_disabled=True)})
return compute_nodes, services
def get_host_states_by_uuids(self, context, compute_uuids):
compute_nodes, services = self._get_computes_all_cells(context,
compute_uuids)
def _load_cells(self, context):
if not self.cells:
# NOTE(danms): global list of cells cached forever right now
self.cells = objects.CellMappingList.get_all(context)
LOG.debug('Found %(count)i cells: %(cells)s',
{'count': len(self.cells),
'cells': ', '.join([c.uuid for c in self.cells])})
def get_host_states_by_uuids(self, context, compute_uuids, spec_obj):
self._load_cells(context)
if (spec_obj and 'requested_destination' in spec_obj and
spec_obj.requested_destination and
'cell' in spec_obj.requested_destination):
only_cell = spec_obj.requested_destination.cell
else:
only_cell = None
if only_cell:
cells = [only_cell]
else:
cells = self.cells
compute_nodes, services = self._get_computes_for_cells(
context, cells, compute_uuids=compute_uuids)
return self._get_host_states(context, compute_nodes, services)
def get_all_host_states(self, context):
@ -614,7 +636,9 @@ class HostManager(object):
the HostManager knows about. Also, each of the consumable resources
in HostState are pre-populated and adjusted based on data in the db.
"""
compute_nodes, services = self._get_computes_all_cells(context)
self._load_cells(context)
compute_nodes, services = self._get_computes_for_cells(context,
self.cells)
return self._get_host_states(context, compute_nodes, services)
def _get_host_states(self, context, compute_nodes, services):
@ -624,30 +648,34 @@ class HostManager(object):
"""
# Get resource usage across the available compute nodes:
seen_nodes = set()
for compute in compute_nodes:
service = services.get(compute.host)
for cell_uuid, computes in compute_nodes.items():
for compute in computes:
service = services.get(compute.host)
if not service:
LOG.warning(_LW(
"No compute service record found for host %(host)s"),
{'host': compute.host})
continue
host = compute.host
node = compute.hypervisor_hostname
state_key = (host, node)
host_state = self.host_state_map.get(state_key)
if not host_state:
host_state = self.host_state_cls(host, node, compute=compute)
self.host_state_map[state_key] = host_state
# We force to update the aggregates info each time a new request
# comes in, because some changes on the aggregates could have been
# happening after setting this field for the first time
host_state.update(compute,
dict(service),
self._get_aggregates_info(host),
self._get_instance_info(context, compute))
if not service:
LOG.warning(_LW(
"No compute service record found for host %(host)s"),
{'host': compute.host})
continue
host = compute.host
node = compute.hypervisor_hostname
state_key = (host, node)
host_state = self.host_state_map.get(state_key)
if not host_state:
host_state = self.host_state_cls(host, node,
cell_uuid,
compute=compute)
self.host_state_map[state_key] = host_state
# We force to update the aggregates info each time a
# new request comes in, because some changes on the
# aggregates could have been happening after setting
# this field for the first time
host_state.update(compute,
dict(service),
self._get_aggregates_info(host),
self._get_instance_info(context, compute))
seen_nodes.add(state_key)
seen_nodes.add(state_key)
# remove compute nodes from host_state_map if they are not active
dead_nodes = set(self.host_state_map.keys()) - seen_nodes

View File

@ -97,13 +97,13 @@ class IronicHostManager(host_manager.HostManager):
return CONF.filter_scheduler.baremetal_enabled_filters
return super(IronicHostManager, self)._load_filters()
def host_state_cls(self, host, node, **kwargs):
def host_state_cls(self, host, node, cell, **kwargs):
"""Factory function/property to create a new HostState."""
compute = kwargs.get('compute')
if compute and self._is_ironic_compute(compute):
return IronicNodeState(host, node)
return IronicNodeState(host, node, cell)
else:
return host_manager.HostState(host, node)
return host_manager.HostState(host, node, cell)
def _init_instance_info(self, compute_nodes=None):
"""Ironic hosts should not pass instance info."""

View File

@ -1917,6 +1917,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
self.conductor._set_vm_state_and_notify(
self.context, 1, 'method', 'updates', 'ex', 'request_spec')
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid')
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@ -1927,7 +1928,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@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, spec_fc_mock):
metadata_mock, sig_mock, spec_fc_mock, im_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -1951,6 +1952,10 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
updates = {'vm_state': vm_states.ACTIVE,
'task_state': None}
im_mock.return_value = objects.InstanceMapping(
cell_mapping=objects.CellMapping.get_by_uuid(self.context,
uuids.cell1))
# Filter properties are populated during code execution
legacy_filter_props = {'retry': {'num_attempts': 1,
'hosts': []}}
@ -1970,6 +1975,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
exc_info, legacy_request_spec)
rollback_mock.assert_called_once_with()
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@ -1980,7 +1986,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@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, spec_fc_mock, sig_mock):
metadata_mock, spec_fc_mock, sig_mock, im_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -2004,6 +2010,10 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
legacy_filter_props = {'retry': {'num_attempts': 1,
'hosts': []}}
im_mock.return_value = objects.InstanceMapping(
cell_mapping=objects.CellMapping.get_by_uuid(self.context,
uuids.cell1))
metadata_mock.return_value = image
exc_info = exc.NoValidHost(reason="")
select_dest_mock.side_effect = exc_info
@ -2096,6 +2106,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
'migrate_server', updates,
exception, legacy_request_spec)
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@ -2108,7 +2119,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
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, spec_fc_mock,
sig_mock):
sig_mock, im_mock):
flavor = flavors.get_flavor_by_name('m1.tiny')
inst_obj = objects.Instance(
image_ref='fake-image_ref',
@ -2127,6 +2138,10 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
legacy_request_spec = fake_spec.to_legacy_request_spec_dict()
spec_fc_mock.return_value = fake_spec
im_mock.return_value = objects.InstanceMapping(
cell_mapping=objects.CellMapping.get_by_uuid(self.context,
uuids.cell1))
hosts = [dict(host='host1', nodename=None, limits={})]
metadata_mock.return_value = image
exc_info = test.TestingException('something happened')

View File

@ -156,7 +156,7 @@ def get_service_by_host(host):
class FakeHostState(host_manager.HostState):
def __init__(self, host, node, attribute_dict, instances=None):
super(FakeHostState, self).__init__(host, node)
super(FakeHostState, self).__init__(host, node, None)
if instances:
self.instances = {inst.uuid: inst for inst in instances}
else:

View File

@ -22,6 +22,7 @@ from nova import objects
from nova.scheduler import caching_scheduler
from nova.scheduler import host_manager
from nova.tests.unit.scheduler import test_scheduler
from nova.tests import uuidsentinel as uuids
ENABLE_PROFILER = False
@ -46,37 +47,40 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch.object(caching_scheduler.CachingScheduler,
"_get_up_hosts")
def test_get_all_host_states_returns_cached_value(self, mock_up_hosts):
self.driver.all_host_states = []
self.driver.all_host_states = {uuids.cell: []}
self.driver._get_all_host_states(self.context, None)
self.assertFalse(mock_up_hosts.called)
self.assertEqual([], self.driver.all_host_states)
self.assertEqual({uuids.cell: []}, self.driver.all_host_states)
@mock.patch.object(caching_scheduler.CachingScheduler,
"_get_up_hosts")
def test_get_all_host_states_loads_hosts(self, mock_up_hosts):
mock_up_hosts.return_value = ["asdf"]
host_state = self._get_fake_host_state()
mock_up_hosts.return_value = {uuids.cell: [host_state]}
result = self.driver._get_all_host_states(self.context, None)
self.assertTrue(mock_up_hosts.called)
self.assertEqual(["asdf"], self.driver.all_host_states)
self.assertEqual(["asdf"], result)
self.assertEqual({uuids.cell: [host_state]},
self.driver.all_host_states)
self.assertEqual([host_state], list(result))
def test_get_up_hosts(self):
with mock.patch.object(self.driver.host_manager,
"get_all_host_states") as mock_get_hosts:
mock_get_hosts.return_value = ["asdf"]
host_state = self._get_fake_host_state()
mock_get_hosts.return_value = [host_state]
result = self.driver._get_up_hosts(self.context)
self.assertTrue(mock_get_hosts.called)
self.assertEqual(mock_get_hosts.return_value, result)
self.assertEqual({uuids.cell: [host_state]}, result)
def test_select_destination_raises_with_no_hosts(self):
spec_obj = self._get_fake_request_spec()
self.driver.all_host_states = []
self.driver.all_host_states = {uuids.cell: []}
self.assertRaises(exception.NoValidHost,
self.driver.select_destinations,
@ -88,7 +92,7 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase):
def test_select_destination_works(self, mock_get_extra):
spec_obj = self._get_fake_request_spec()
fake_host = self._get_fake_host_state()
self.driver.all_host_states = [fake_host]
self.driver.all_host_states = {uuids.cell: [fake_host]}
result = self._test_select_destinations(spec_obj)
@ -134,7 +138,8 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase):
def _get_fake_host_state(self, index=0):
host_state = host_manager.HostState(
'host_%s' % index,
'node_%s' % index)
'node_%s' % index,
uuids.cell)
host_state.free_ram_mb = 50000
host_state.total_usable_ram_mb = 50000
host_state.free_disk_mb = 4096
@ -164,7 +169,7 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase):
for x in range(hosts):
host_state = self._get_fake_host_state(x)
host_states.append(host_state)
self.driver.all_host_states = host_states
self.driver.all_host_states = {uuids.cell: host_states}
def run_test():
a = timeutils.utcnow()
@ -204,6 +209,22 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase):
# But this is here so you can do simply performance testing easily.
self.assertLess(per_request_ms, 1000)
def test_request_single_cell(self):
spec_obj = self._get_fake_request_spec()
spec_obj.requested_destination = objects.Destination(
cell=objects.CellMapping(uuid=uuids.cell2))
host_states_cell1 = [self._get_fake_host_state(i)
for i in range(1, 5)]
host_states_cell2 = [self._get_fake_host_state(i)
for i in range(5, 10)]
self.driver.all_host_states = {
uuids.cell1: host_states_cell1,
uuids.cell2: host_states_cell2,
}
d = self.driver.select_destinations(self.context, spec_obj)
self.assertIn(d[0]['host'], [hs.host for hs in host_states_cell2])
if __name__ == '__main__':
# A handy tool to help profile the schedulers performance

View File

@ -109,7 +109,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
retry = {'hosts': [], 'num_attempts': 1}
filter_properties = {'retry': retry}
host_state = host_manager.HostState('host', 'node')
host_state = host_manager.HostState('host', 'node', uuids.cell)
host_state.limits['vcpu'] = 5
scheduler_utils.populate_filter_properties(filter_properties,
host_state)

View File

@ -17,6 +17,7 @@ Tests For HostManager
"""
import collections
import contextlib
import datetime
import mock
@ -65,10 +66,11 @@ class HostManagerTestCase(test.NoDBTestCase):
self.flags(enabled_filters=['FakeFilterClass1'],
group='filter_scheduler')
self.host_manager = host_manager.HostManager()
cell = uuids.cell
self.fake_hosts = [host_manager.HostState('fake_host%s' % x,
'fake-node') for x in range(1, 5)]
'fake-node', cell) for x in range(1, 5)]
self.fake_hosts += [host_manager.HostState('fake_multihost',
'fake-node%s' % x) for x in range(1, 5)]
'fake-node%s' % x, cell) for x in range(1, 5)]
self.useFixture(fixtures.SpawnIsSynchronousFixture())
@ -656,7 +658,7 @@ class HostManagerTestCase(test.NoDBTestCase):
cn1 = objects.ComputeNode(host='host1')
hm._instance_info = {'host1': {'instances': {uuids.instance: inst1},
'updated': True}}
host_state = host_manager.HostState('host1', cn1)
host_state = host_manager.HostState('host1', cn1, uuids.cell)
self.assertFalse(host_state.instances)
mock_get_by_host.return_value = None
host_state.update(
@ -679,7 +681,7 @@ class HostManagerTestCase(test.NoDBTestCase):
cn1 = objects.ComputeNode(host='host1')
hm._instance_info = {'host1': {'instances': {uuids.instance: inst1},
'updated': False}}
host_state = host_manager.HostState('host1', cn1)
host_state = host_manager.HostState('host1', cn1, uuids.cell)
self.assertFalse(host_state.instances)
mock_get_by_host.return_value = objects.InstanceList(objects=[inst1])
host_state.update(
@ -857,8 +859,8 @@ class HostManagerTestCase(test.NoDBTestCase):
@mock.patch('nova.objects.CellMappingList.get_all')
@mock.patch('nova.objects.ComputeNodeList.get_all')
@mock.patch('nova.objects.ServiceList.get_by_binary')
def test_get_computes_all_cells(self, mock_sl, mock_cn, mock_cm):
mock_cm.return_value = [
def test_get_computes_for_cells(self, mock_sl, mock_cn, mock_cm):
cells = [
objects.CellMapping(uuid=uuids.cell1,
db_connection='none://1',
transport_url='none://'),
@ -866,6 +868,7 @@ class HostManagerTestCase(test.NoDBTestCase):
db_connection='none://2',
transport_url='none://'),
]
mock_cm.return_value = cells
mock_sl.side_effect = [
[objects.ServiceList(host='foo')],
[objects.ServiceList(host='bar')],
@ -875,16 +878,18 @@ class HostManagerTestCase(test.NoDBTestCase):
[objects.ComputeNode(host='bar')],
]
context = nova_context.RequestContext('fake', 'fake')
cns, srv = self.host_manager._get_computes_all_cells(context)
self.assertEqual(['bar', 'foo'],
sorted([cn.host for cn in cns]))
cns, srv = self.host_manager._get_computes_for_cells(context, cells)
self.assertEqual({uuids.cell1: ['foo'],
uuids.cell2: ['bar']},
{cell: [cn.host for cn in computes]
for cell, computes in cns.items()})
self.assertEqual(['bar', 'foo'], sorted(list(srv.keys())))
@mock.patch('nova.objects.CellMappingList.get_all')
@mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids')
@mock.patch('nova.objects.ServiceList.get_by_binary')
def test_get_computes_all_cells_uuid(self, mock_sl, mock_cn, mock_cm):
mock_cm.return_value = [
def test_get_computes_for_cells_uuid(self, mock_sl, mock_cn, mock_cm):
cells = [
objects.CellMapping(uuid=uuids.cell1,
db_connection='none://1',
transport_url='none://'),
@ -892,6 +897,7 @@ class HostManagerTestCase(test.NoDBTestCase):
db_connection='none://2',
transport_url='none://'),
]
mock_cm.return_value = cells
mock_sl.side_effect = [
[objects.ServiceList(host='foo')],
[objects.ServiceList(host='bar')],
@ -901,11 +907,56 @@ class HostManagerTestCase(test.NoDBTestCase):
[objects.ComputeNode(host='bar')],
]
context = nova_context.RequestContext('fake', 'fake')
cns, srv = self.host_manager._get_computes_all_cells(context, [])
self.assertEqual(['bar', 'foo'],
sorted([cn.host for cn in cns]))
cns, srv = self.host_manager._get_computes_for_cells(context, cells,
[])
self.assertEqual({uuids.cell1: ['foo'],
uuids.cell2: ['bar']},
{cell: [cn.host for cn in computes]
for cell, computes in cns.items()})
self.assertEqual(['bar', 'foo'], sorted(list(srv.keys())))
@mock.patch('nova.context.target_cell')
@mock.patch('nova.objects.CellMappingList.get_all')
@mock.patch('nova.objects.ComputeNodeList.get_all')
@mock.patch('nova.objects.ServiceList.get_by_binary')
def test_get_computes_for_cells_limit_to_cell(self, mock_sl,
mock_cn, mock_cm,
mock_target):
host_manager.LOG.debug = host_manager.LOG.error
cells = [
objects.CellMapping(uuid=uuids.cell1,
database_connection='none://1',
transport_url='none://'),
objects.CellMapping(uuid=uuids.cell2,
database_connection='none://2',
transport_url='none://'),
]
mock_sl.return_value = [objects.ServiceList(host='foo')]
mock_cn.return_value = [objects.ComputeNode(host='foo')]
mock_cm.return_value = cells
@contextlib.contextmanager
def fake_set_target(context, cell):
yield
mock_target.side_effect = fake_set_target
context = nova_context.RequestContext('fake', 'fake')
cns, srv = self.host_manager._get_computes_for_cells(
context, cells=cells[1:])
self.assertEqual({uuids.cell2: ['foo']},
{cell: [cn.host for cn in computes]
for cell, computes in cns.items()})
self.assertEqual(['foo'], list(srv.keys()))
# NOTE(danms): We have two cells, but we should only have
# targeted one if we honored the only-cell destination requirement,
# and only looked up services and compute nodes in one
mock_target.assert_called_once_with(context, cells[1])
mock_cn.assert_called_once_with(context)
mock_sl.assert_called_once_with(context, 'nova-compute',
include_disabled=True)
class HostManagerChangedNodesTestCase(test.NoDBTestCase):
"""Test case for HostManager class."""
@ -916,10 +967,10 @@ class HostManagerChangedNodesTestCase(test.NoDBTestCase):
super(HostManagerChangedNodesTestCase, self).setUp()
self.host_manager = host_manager.HostManager()
self.fake_hosts = [
host_manager.HostState('host1', 'node1'),
host_manager.HostState('host2', 'node2'),
host_manager.HostState('host3', 'node3'),
host_manager.HostState('host4', 'node4')
host_manager.HostState('host1', 'node1', uuids.cell),
host_manager.HostState('host2', 'node2', uuids.cell),
host_manager.HostState('host3', 'node3', uuids.cell),
host_manager.HostState('host4', 'node4', uuids.cell)
]
@mock.patch('nova.objects.ServiceList.get_by_binary')
@ -1020,7 +1071,7 @@ class HostStateTestCase(test.NoDBTestCase):
cpu_allocation_ratio=16.0, ram_allocation_ratio=1.5,
disk_allocation_ratio=1.0)
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
host.update(compute=compute)
sync_mock.assert_called_once_with(("fakehost", "fakenode"))
@ -1063,7 +1114,7 @@ class HostStateTestCase(test.NoDBTestCase):
cpu_allocation_ratio=16.0, ram_allocation_ratio=1.5,
disk_allocation_ratio=1.0)
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
host.update(compute=compute)
self.assertEqual([], host.pci_stats.pools)
self.assertEqual(hyper_ver_int, host.hypervisor_version)
@ -1096,7 +1147,7 @@ class HostStateTestCase(test.NoDBTestCase):
cpu_allocation_ratio=16.0, ram_allocation_ratio=1.5,
disk_allocation_ratio=1.0)
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
host.update(compute=compute)
self.assertEqual(5, host.num_instances)
@ -1131,7 +1182,7 @@ class HostStateTestCase(test.NoDBTestCase):
vcpus=0),
numa_topology=fake_numa_topology,
pci_requests=objects.InstancePCIRequests(requests=[]))
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
self.assertIsNone(host.updated)
host.consume_from_request(spec_obj)
@ -1186,7 +1237,7 @@ class HostStateTestCase(test.NoDBTestCase):
ephemeral_gb=0,
memory_mb=512,
vcpus=1))
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
self.assertIsNone(host.updated)
host.pci_stats = pci_stats.PciDeviceStats(
[objects.PciDevicePool(vendor_id='8086',
@ -1219,7 +1270,7 @@ class HostStateTestCase(test.NoDBTestCase):
ephemeral_gb=0,
memory_mb=1024,
vcpus=1))
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
self.assertIsNone(host.updated)
fake_updated = mock.sentinel.fake_updated
host.updated = fake_updated
@ -1256,7 +1307,7 @@ class HostStateTestCase(test.NoDBTestCase):
stats=None, pci_device_pools=None,
cpu_allocation_ratio=16.0, ram_allocation_ratio=1.5,
disk_allocation_ratio=1.0)
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
host.update(compute=compute)
self.assertEqual(len(host.metrics), 2)
@ -1273,7 +1324,7 @@ class HostStateTestCase(test.NoDBTestCase):
compute = objects.ComputeNode(free_ram_mb=100,
uuid=uuids.compute_node_uuid)
host = host_manager.HostState("fakehost", "fakenode")
host = host_manager.HostState("fakehost", "fakenode", uuids.cell)
host._update_from_compute_node(compute)
# Because compute record not ready, the update of free ram
# will not happen and the value will still be 0

View File

@ -61,9 +61,11 @@ class IronicHostManagerTestCase(test.NoDBTestCase):
def test_state_public_api_signatures(self):
self.assertPublicAPISignatures(
host_manager.HostState("dummy",
"dummy"),
"dummy",
uuids.cell),
ironic_host_manager.IronicNodeState("dummy",
"dummy")
"dummy",
uuids.cell)
)
@mock.patch('nova.objects.ServiceList.get_by_binary')
@ -190,6 +192,7 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
init_mock.return_value = None
compute = objects.ComputeNode(**{'hypervisor_type': 'ironic'})
host_state = self.host_manager.host_state_cls('fake-host', 'fake-node',
uuids.cell,
compute=compute)
self.assertIs(ironic_host_manager.IronicNodeState, type(host_state))
@ -198,13 +201,15 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
init_mock.return_value = None
compute = objects.ComputeNode(**{'cpu_info': 'other cpu'})
host_state = self.host_manager.host_state_cls('fake-host', 'fake-node',
uuids.cell,
compute=compute)
self.assertIs(host_manager.HostState, type(host_state))
@mock.patch.object(host_manager.HostState, '__init__')
def test_create_host_state_null_compute(self, init_mock):
init_mock.return_value = None
host_state = self.host_manager.host_state_cls('fake-host', 'fake-node')
host_state = self.host_manager.host_state_cls('fake-host', 'fake-node',
uuids.cell)
self.assertIs(host_manager.HostState, type(host_state))
@mock.patch('nova.objects.ServiceList.get_by_binary')
@ -253,7 +258,8 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
self.assertEqual(len(host_states_map), 0)
def test_update_from_compute_node(self):
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode")
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode",
uuids.cell)
host.update(compute=self.compute_node)
self.assertEqual(1024, host.free_ram_mb)
@ -270,13 +276,15 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
"""Tests that we ignore a compute node that does not have its
free_disk_gb field set yet from the compute resource tracker.
"""
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode")
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode",
uuids.cell)
self.compute_node.free_disk_gb = None
host.update(compute=self.compute_node)
self.assertEqual(0, host.free_disk_mb)
def test_consume_identical_instance_from_compute(self):
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode")
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode",
uuids.cell)
host.update(compute=self.compute_node)
self.assertIsNone(host.updated)
@ -292,7 +300,8 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
self.assertIsNotNone(host.updated)
def test_consume_larger_instance_from_compute(self):
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode")
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode",
uuids.cell)
host.update(compute=self.compute_node)
self.assertIsNone(host.updated)
@ -307,7 +316,8 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
self.assertIsNotNone(host.updated)
def test_consume_smaller_instance_from_compute(self):
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode")
host = ironic_host_manager.IronicNodeState("fakehost", "fakenode",
uuids.cell)
host.update(compute=self.compute_node)
self.assertIsNone(host.updated)
@ -338,10 +348,12 @@ class IronicHostManagerTestFilters(test.NoDBTestCase):
self.flags(baremetal_enabled_filters=['FakeFilterClass2'],
group='filter_scheduler')
self.host_manager = ironic_host_manager.IronicHostManager()
cell = uuids.cell
self.fake_hosts = [ironic_host_manager.IronicNodeState(
'fake_host%s' % x, 'fake-node') for x in range(1, 5)]
'fake_host%s' % x, 'fake-node', cell) for x in range(1, 5)]
self.fake_hosts += [ironic_host_manager.IronicNodeState(
'fake_multihost', 'fake-node%s' % x) for x in range(1, 5)]
'fake_multihost', 'fake-node%s' % x, cell)
for x in range(1, 5)]
def test_enabled_filters(self):
enabled_filters = self.host_manager.enabled_filters