Implement instance update logic in Scheduler

This commit adds the logic in the HostManager class for processing calls
that include updated instance information about hosts to the scheduler.
Subsequent commits will add the RPC calls.

DocImpact
We add a new CONF option 'scheduler_tracks_instance_changes'. This
option is a boolean option that defaults to True. This option controls
whether or not the scheduler is responsible for tracking changes to
instances on compute nodes. If True, compute nodes notify the scheduler
when instances are updated, created, and deleted. This allows the
scheduler to keep its in-memory representation of usage and instance
location accurate. However, the current implementation of instance
tracking functionality in the scheduler *may* be too resource-intensive,
and thus this option enables deployers that do not either use scheduler
filters that rely on instance information or are concerned about
performance of the scheduler to disable this functionality.

Partially-Implements: blueprint isolate-scheduler-db
(https://blueprints.launchpad.net/nova/+spec/isolate-scheduler-db)

Change-Id: Ibfcd3b77dbde94db14a9e4027dbaceecb76182cd
This commit is contained in:
EdLeafe
2015-03-16 18:36:16 +00:00
parent d0877b1e75
commit 82cc056fb7
6 changed files with 496 additions and 52 deletions

View File

@@ -18,6 +18,7 @@ Manage hosts in the current zone.
"""
import collections
import time
import UserDict
import iso8601
@@ -28,13 +29,14 @@ from oslo_utils import timeutils
from nova.compute import task_states
from nova.compute import vm_states
from nova import context as ctxt_mod
from nova import context as context_module
from nova import exception
from nova.i18n import _, _LI, _LW
from nova import objects
from nova.pci import stats as pci_stats
from nova.scheduler import filters
from nova.scheduler import weights
from nova import utils
from nova.virt import hardware
host_manager_opts = [
@@ -60,12 +62,17 @@ host_manager_opts = [
cfg.ListOpt('scheduler_weight_classes',
default=['nova.scheduler.weights.all_weighers'],
help='Which weight class names to use for weighing hosts'),
]
cfg.BoolOpt('scheduler_tracks_instance_changes',
default=True,
help='Determines if the Scheduler tracks changes to instances '
'to help with its filtering decisions.'),
]
CONF = cfg.CONF
CONF.register_opts(host_manager_opts)
LOG = logging.getLogger(__name__)
HOST_INSTANCE_SEMAPHORE = "host_instance"
class ReadOnlyDict(UserDict.IterableUserDict):
@@ -142,6 +149,9 @@ class HostState(object):
# List of aggregates the host belongs to
self.aggregates = []
# Instances on this host
self.instances = {}
self.updated = None
if compute:
self.update_from_compute_node(compute)
@@ -312,9 +322,14 @@ class HostManager(object):
# to those aggregates
self.host_aggregates_map = collections.defaultdict(set)
self._init_aggregates()
self.tracks_instance_changes = CONF.scheduler_tracks_instance_changes
# Dict of instances and status, keyed by host
self._instance_info = {}
if self.tracks_instance_changes:
self._init_instance_info()
def _init_aggregates(self):
elevated = ctxt_mod.get_admin_context()
elevated = context_module.get_admin_context()
aggs = objects.AggregateList.get_all(elevated)
for agg in aggs:
self.aggs_by_id[agg.id] = agg
@@ -349,6 +364,51 @@ class HostManager(object):
if aggregate.id in self.host_aggregates_map[host]:
self.host_aggregates_map[host].remove(aggregate.id)
def _init_instance_info(self):
"""Creates the initial view of instances for all hosts.
As this initial population of instance information may take some time,
we don't wish to block the scheduler's startup while this completes.
The async method allows us to simply mock out the _init_instance_info()
method in tests.
"""
def _async_init_instance_info():
context = context_module.get_admin_context()
LOG.debug("START:_async_init_instance_info")
self._instance_info = {}
compute_nodes = objects.ComputeNodeList.get_all(context).objects
LOG.debug("Total number of compute nodes: %s", len(compute_nodes))
# Break the queries into batches of 10 to reduce the total number
# of calls to the DB.
batch_size = 10
start_node = 0
end_node = batch_size
while start_node <= len(compute_nodes):
curr_nodes = compute_nodes[start_node:end_node]
start_node += batch_size
end_node += batch_size
filters = {"host": [curr_node.host
for curr_node in curr_nodes]}
result = objects.InstanceList.get_by_filters(context,
filters)
instances = result.objects
LOG.debug("Adding %s instances for hosts %s-%s",
len(instances), start_node, end_node)
for instance in instances:
host = instance.host
if host not in self._instance_info:
self._instance_info[host] = {"instances": {},
"updated": False}
inst_dict = self._instance_info[host]
inst_dict["instances"][instance.uuid] = instance
# Call sleep() to cooperatively yield
time.sleep(0)
LOG.debug("END:_async_init_instance_info")
# Run this async so that we don't block the scheduler start-up
utils.spawn_n(_async_init_instance_info)
def _choose_host_filters(self, filter_cls_names):
"""Since the caller may specify which filters to use we need
to have an authoritative list of what is permissible. This
@@ -491,6 +551,7 @@ class HostManager(object):
self.host_aggregates_map[
host_state.host]]
host_state.update_service(dict(service.iteritems()))
self._add_instance_info(context, compute, host_state)
seen_nodes.add(state_key)
# remove compute nodes from host_state_map if they are not active
@@ -502,3 +563,106 @@ class HostManager(object):
del self.host_state_map[state_key]
return self.host_state_map.itervalues()
def _add_instance_info(self, context, compute, host_state):
"""Adds the host instance info to the host_state object.
Some older compute nodes may not be sending instance change updates to
the Scheduler; other sites may disable this feature for performance
reasons. In either of these cases, there will either be no information
for the host, or the 'updated' value for that host dict will be False.
In those cases, we need to grab the current InstanceList instead of
relying on the version in _instance_info.
"""
host_name = compute.host
host_info = self._instance_info.get(host_name)
if host_info and host_info.get("updated"):
inst_dict = host_info["instances"]
else:
# Host is running old version, or updates aren't flowing.
inst_list = objects.InstanceList.get_by_host(context, host_name)
inst_dict = {instance.uuid: instance
for instance in inst_list.objects}
host_state.instances = inst_dict
def _recreate_instance_info(self, context, host_name):
"""Get the InstanceList for the specified host, and store it in the
_instance_info dict.
"""
instances = objects.InstanceList.get_by_host(context, host_name)
inst_dict = {instance.uuid: instance for instance in instances}
host_info = self._instance_info[host_name] = {}
host_info["instances"] = inst_dict
host_info["updated"] = False
@utils.synchronized(HOST_INSTANCE_SEMAPHORE)
def update_instance_info(self, context, host_name, instance_info):
"""Receives an InstanceList object from a compute node.
This method receives information from a compute node when it starts up,
or when its instances have changed, and updates its view of hosts and
instances with it.
"""
host_info = self._instance_info.get(host_name)
if host_info:
inst_dict = host_info.get("instances")
for instance in instance_info.objects:
# Overwrite the entry (if any) with the new info.
inst_dict[instance.uuid] = instance
host_info["updated"] = True
else:
instances = instance_info.objects
if len(instances) > 1:
# This is a host sending its full instance list, so use it.
host_info = self._instance_info[host_name] = {}
host_info["instances"] = {instance.uuid: instance
for instance in instances}
host_info["updated"] = True
else:
self._recreate_instance_info(context, host_name)
LOG.info(_LI("Received an update from an unknown host '%s'. "
"Re-created its InstanceList."), host_name)
@utils.synchronized(HOST_INSTANCE_SEMAPHORE)
def delete_instance_info(self, context, host_name, instance_uuid):
"""Receives the UUID from a compute node when one of its instances is
terminated.
The instance in the local view of the host's instances is removed.
"""
host_info = self._instance_info.get(host_name)
if host_info:
inst_dict = host_info["instances"]
# Remove the existing Instance object, if any
inst_dict.pop(instance_uuid, None)
host_info["updated"] = True
else:
self._recreate_instance_info(context, host_name)
LOG.info(_LI("Received a delete update from an unknown host '%s'. "
"Re-created its InstanceList."), host_name)
@utils.synchronized(HOST_INSTANCE_SEMAPHORE)
def sync_instance_info(self, context, host_name, instance_uuids):
"""Receives the uuids of the instances on a host.
This method is periodically called by the compute nodes, which send a
list of all the UUID values for the instances on that node. This is
used by the scheduler's HostManager to detect when its view of the
compute node's instances is out of sync.
"""
host_info = self._instance_info.get(host_name)
if host_info:
local_set = set(host_info["instances"].keys())
compute_set = set(instance_uuids)
if not local_set == compute_set:
self._recreate_instance_info(context, host_name)
LOG.info(_LI("The instance sync for host '%s' did not match. "
"Re-created its InstanceList."), host_name)
return
host_info["updated"] = True
LOG.info(_LI("Successfully synced instances from host '%s'."),
host_name)
else:
self._recreate_instance_info(context, host_name)
LOG.info(_LI("Received a sync request from an unknown host '%s'. "
"Re-created its InstanceList."), host_name)

View File

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

View File

@@ -38,13 +38,14 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.objects.ServiceList.get_by_binary',
return_value=fakes.SERVICES)
@mock.patch('nova.objects.InstanceList.get_by_host')
@mock.patch('nova.objects.ComputeNodeList.get_all',
return_value=fakes.COMPUTE_NODES)
@mock.patch('nova.db.instance_extra_get_by_instance_uuid',
return_value={'numa_topology': None,
'pci_requests': None})
def test_schedule_happy_day(self, mock_get_extra, mock_get_all,
mock_get_by_binary):
mock_by_host, mock_get_by_binary):
"""Make sure there's nothing glaringly wrong with _schedule()
by doing a happy day pass through.
"""
@@ -116,13 +117,14 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.objects.ServiceList.get_by_binary',
return_value=fakes.SERVICES)
@mock.patch('nova.objects.InstanceList.get_by_host')
@mock.patch('nova.objects.ComputeNodeList.get_all',
return_value=fakes.COMPUTE_NODES)
@mock.patch('nova.db.instance_extra_get_by_instance_uuid',
return_value={'numa_topology': None,
'pci_requests': None})
def test_schedule_host_pool(self, mock_get_extra, mock_get_all,
mock_get_by_binary):
mock_by_host, mock_get_by_binary):
"""Make sure the scheduler_host_subset_size property works properly."""
self.flags(scheduler_host_subset_size=2)
@@ -149,13 +151,14 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.objects.ServiceList.get_by_binary',
return_value=fakes.SERVICES)
@mock.patch('nova.objects.InstanceList.get_by_host')
@mock.patch('nova.objects.ComputeNodeList.get_all',
return_value=fakes.COMPUTE_NODES)
@mock.patch('nova.db.instance_extra_get_by_instance_uuid',
return_value={'numa_topology': None,
'pci_requests': None})
def test_schedule_large_host_pool(self, mock_get_extra, mock_get_all,
mock_get_by_binary):
mock_by_host, mock_get_by_binary):
"""Hosts should still be chosen if pool size
is larger than number of filtered hosts.
"""
@@ -181,6 +184,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
# one host should be chose
self.assertEqual(len(hosts), 1)
@mock.patch('nova.scheduler.host_manager.HostManager._add_instance_info')
@mock.patch('nova.objects.ServiceList.get_by_binary',
return_value=fakes.SERVICES)
@mock.patch('nova.objects.ComputeNodeList.get_all',
@@ -188,8 +192,9 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.db.instance_extra_get_by_instance_uuid',
return_value={'numa_topology': None,
'pci_requests': None})
def test_schedule_chooses_best_host(self, mock_get_extra, mock_get_all,
mock_get_by_binary):
def test_schedule_chooses_best_host(self, mock_get_extra, mock_cn_get_all,
mock_get_by_binary,
mock_add_inst_info):
"""If scheduler_host_subset_size is 1, the largest host with greatest
weight should be returned.
"""
@@ -223,7 +228,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
filter_properties = {}
self.mox.ReplayAll()
hosts = self.driver._schedule(self.context, request_spec,
filter_properties=filter_properties)
filter_properties=filter_properties)
# one host should be chosen
self.assertEqual(1, len(hosts))
@@ -232,13 +237,14 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.objects.ServiceList.get_by_binary',
return_value=fakes.SERVICES)
@mock.patch('nova.objects.InstanceList.get_by_host')
@mock.patch('nova.objects.ComputeNodeList.get_all',
return_value=fakes.COMPUTE_NODES)
@mock.patch('nova.db.instance_extra_get_by_instance_uuid',
return_value={'numa_topology': None,
'pci_requests': None})
def test_select_destinations(self, mock_get_extra, mock_get_all,
mock_get_by_binary):
mock_by_host, mock_get_by_binary):
"""select_destinations is basically a wrapper around _schedule().
Similar to the _schedule tests, this just does a happy path test to

View File

@@ -19,9 +19,11 @@ Tests For HostManager
import collections
import mock
from oslo_config import cfg
from oslo_serialization import jsonutils
import six
import nova
from nova.compute import task_states
from nova.compute import vm_states
from nova import exception
@@ -30,10 +32,15 @@ from nova.objects import base as obj_base
from nova.scheduler import filters
from nova.scheduler import host_manager
from nova import test
from nova.tests.unit import fake_instance
from nova.tests.unit import matchers
from nova.tests.unit.scheduler import fakes
from nova import utils
CONF = cfg.CONF
CONF.import_opt('scheduler_tracks_instance_changes',
'nova.scheduler.host_manager')
class FakeFilterClass1(filters.BaseHostFilter):
def host_passes(self, host_state, filter_properties):
@@ -48,41 +55,84 @@ class FakeFilterClass2(filters.BaseHostFilter):
class HostManagerTestCase(test.NoDBTestCase):
"""Test case for HostManager class."""
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(HostManagerTestCase, self).setUp()
self.flags(scheduler_available_filters=['%s.%s' % (__name__, cls) for
cls in ['FakeFilterClass1',
'FakeFilterClass2']])
self.flags(scheduler_default_filters=['FakeFilterClass1'])
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.host_manager = host_manager.HostManager()
self.host_manager = host_manager.HostManager()
self.fake_hosts = [host_manager.HostState('fake_host%s' % x,
'fake-node') for x in xrange(1, 5)]
self.fake_hosts += [host_manager.HostState('fake_multihost',
'fake-node%s' % x) for x in xrange(1, 5)]
@mock.patch.object(nova.objects.InstanceList, 'get_by_filters')
@mock.patch.object(nova.objects.ComputeNodeList, 'get_all')
@mock.patch('nova.utils.spawn_n')
def test_init_instance_info_batches(self, mock_spawn, mock_get_all,
mock_get_by_filters):
mock_spawn.side_effect = lambda f, *a, **k: f(*a, **k)
cn_list = objects.ComputeNodeList()
for num in range(22):
host_name = 'host_%s' % num
cn_list.objects.append(objects.ComputeNode(host=host_name))
mock_get_all.return_value = cn_list
self.host_manager._init_instance_info()
self.assertEqual(mock_get_by_filters.call_count, 3)
@mock.patch.object(nova.objects.InstanceList, 'get_by_filters')
@mock.patch.object(nova.objects.ComputeNodeList, 'get_all')
@mock.patch('nova.utils.spawn_n')
def test_init_instance_info(self, mock_spawn, mock_get_all,
mock_get_by_filters):
mock_spawn.side_effect = lambda f, *a, **k: f(*a, **k)
cn1 = objects.ComputeNode(host='host1')
cn2 = objects.ComputeNode(host='host2')
inst1 = objects.Instance(host='host1', uuid='uuid1')
inst2 = objects.Instance(host='host1', uuid='uuid2')
inst3 = objects.Instance(host='host2', uuid='uuid3')
mock_get_all.return_value = objects.ComputeNodeList(objects=[cn1, cn2])
mock_get_by_filters.return_value = objects.InstanceList(
objects=[inst1, inst2, inst3])
hm = self.host_manager
hm._instance_info = {}
hm._init_instance_info()
self.assertEqual(len(hm._instance_info), 2)
fake_info = hm._instance_info['host1']
self.assertIn('uuid1', fake_info['instances'])
self.assertIn('uuid2', fake_info['instances'])
self.assertNotIn('uuid3', fake_info['instances'])
def test_default_filters(self):
default_filters = self.host_manager.default_filters
self.assertEqual(1, len(default_filters))
self.assertIsInstance(default_filters[0], FakeFilterClass1)
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(objects.AggregateList, 'get_all')
def test_init_aggregates_no_aggs(self, agg_get_all):
def test_init_aggregates_no_aggs(self, agg_get_all, mock_init_info):
agg_get_all.return_value = []
self.host_manager = host_manager.HostManager()
self.assertEqual({}, self.host_manager.aggs_by_id)
self.assertEqual({}, self.host_manager.host_aggregates_map)
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(objects.AggregateList, 'get_all')
def test_init_aggregates_one_agg_no_hosts(self, agg_get_all):
def test_init_aggregates_one_agg_no_hosts(self, agg_get_all,
mock_init_info):
fake_agg = objects.Aggregate(id=1, hosts=[])
agg_get_all.return_value = [fake_agg]
self.host_manager = host_manager.HostManager()
self.assertEqual({1: fake_agg}, self.host_manager.aggs_by_id)
self.assertEqual({}, self.host_manager.host_aggregates_map)
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(objects.AggregateList, 'get_all')
def test_init_aggregates_one_agg_with_hosts(self, agg_get_all):
def test_init_aggregates_one_agg_with_hosts(self, agg_get_all,
mock_init_info):
fake_agg = objects.Aggregate(id=1, hosts=['fake-host'])
agg_get_all.return_value = [fake_agg]
self.host_manager = host_manager.HostManager()
@@ -305,10 +355,10 @@ class HostManagerTestCase(test.NoDBTestCase):
fake_properties)
self._verify_result(info, result, False)
def test_get_all_host_states(self):
@mock.patch.object(nova.objects.InstanceList, 'get_by_host')
def test_get_all_host_states(self, mock_get_by_host):
mock_get_by_host.return_value = objects.InstanceList()
context = 'fake_context'
self.mox.StubOutWithMock(objects.ServiceList, 'get_by_binary')
self.mox.StubOutWithMock(objects.ComputeNodeList, 'get_all')
self.mox.StubOutWithMock(host_manager.LOG, 'warning')
@@ -365,30 +415,35 @@ class HostManagerTestCase(test.NoDBTestCase):
self.assertEqual(host_states_map[('host4', 'node4')].free_disk_mb,
8388608)
@mock.patch.object(nova.objects.InstanceList, 'get_by_host')
@mock.patch.object(host_manager.HostState, 'update_from_compute_node')
@mock.patch.object(objects.ComputeNodeList, 'get_all')
@mock.patch.object(objects.ServiceList, 'get_by_binary')
def test_get_all_host_states_with_no_aggs(self, svc_get_by_binary,
cn_get_all, update_from_cn):
cn_get_all, update_from_cn,
mock_get_by_host):
svc_get_by_binary.return_value = [objects.Service(host='fake')]
cn_get_all.return_value = [
objects.ComputeNode(host='fake', hypervisor_hostname='fake')]
mock_get_by_host.return_value = objects.InstanceList()
self.host_manager.host_aggregates_map = collections.defaultdict(set)
self.host_manager.get_all_host_states('fake-context')
host_state = self.host_manager.host_state_map[('fake', 'fake')]
self.assertEqual([], host_state.aggregates)
@mock.patch.object(nova.objects.InstanceList, 'get_by_host')
@mock.patch.object(host_manager.HostState, 'update_from_compute_node')
@mock.patch.object(objects.ComputeNodeList, 'get_all')
@mock.patch.object(objects.ServiceList, 'get_by_binary')
def test_get_all_host_states_with_matching_aggs(self, svc_get_by_binary,
cn_get_all,
update_from_cn):
update_from_cn,
mock_get_by_host):
svc_get_by_binary.return_value = [objects.Service(host='fake')]
cn_get_all.return_value = [
objects.ComputeNode(host='fake', hypervisor_hostname='fake')]
mock_get_by_host.return_value = objects.InstanceList()
fake_agg = objects.Aggregate(id=1)
self.host_manager.host_aggregates_map = collections.defaultdict(
set, {'fake': set([1])})
@@ -398,18 +453,21 @@ class HostManagerTestCase(test.NoDBTestCase):
host_state = self.host_manager.host_state_map[('fake', 'fake')]
self.assertEqual([fake_agg], host_state.aggregates)
@mock.patch.object(nova.objects.InstanceList, 'get_by_host')
@mock.patch.object(host_manager.HostState, 'update_from_compute_node')
@mock.patch.object(objects.ComputeNodeList, 'get_all')
@mock.patch.object(objects.ServiceList, 'get_by_binary')
def test_get_all_host_states_with_not_matching_aggs(self,
svc_get_by_binary,
cn_get_all,
update_from_cn):
update_from_cn,
mock_get_by_host):
svc_get_by_binary.return_value = [objects.Service(host='fake'),
objects.Service(host='other')]
cn_get_all.return_value = [
objects.ComputeNode(host='fake', hypervisor_hostname='fake'),
objects.ComputeNode(host='other', hypervisor_hostname='other')]
mock_get_by_host.return_value = objects.InstanceList()
fake_agg = objects.Aggregate(id=1)
self.host_manager.host_aggregates_map = collections.defaultdict(
set, {'other': set([1])})
@@ -419,14 +477,205 @@ class HostManagerTestCase(test.NoDBTestCase):
host_state = self.host_manager.host_state_map[('fake', 'fake')]
self.assertEqual([], host_state.aggregates)
@mock.patch('nova.objects.ServiceList.get_by_binary')
@mock.patch('nova.objects.ComputeNodeList.get_all')
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_get_all_host_states_updated(self, mock_get_by_host,
mock_get_all_comp,
mock_get_svc_by_binary):
mock_get_all_comp.return_value = fakes.COMPUTE_NODES
mock_get_svc_by_binary.return_value = fakes.SERVICES
context = 'fake_context'
hm = self.host_manager
inst1 = objects.Instance(uuid='uuid1')
cn1 = objects.ComputeNode(host='host1')
hm._instance_info = {'host1': {'instances': {'uuid1': inst1},
'updated': True}}
host_state = host_manager.HostState('host1', cn1)
self.assertFalse(host_state.instances)
mock_get_by_host.return_value = None
hm._add_instance_info(context, cn1, host_state)
self.assertFalse(mock_get_by_host.called)
self.assertTrue(host_state.instances)
self.assertEqual(host_state.instances['uuid1'], inst1)
@mock.patch('nova.objects.ServiceList.get_by_binary')
@mock.patch('nova.objects.ComputeNodeList.get_all')
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_get_all_host_states_not_updated(self, mock_get_by_host,
mock_get_all_comp,
mock_get_svc_by_binary):
mock_get_all_comp.return_value = fakes.COMPUTE_NODES
mock_get_svc_by_binary.return_value = fakes.SERVICES
context = 'fake_context'
hm = self.host_manager
inst1 = objects.Instance(uuid='uuid1')
cn1 = objects.ComputeNode(host='host1')
hm._instance_info = {'host1': {'instances': {'uuid1': inst1},
'updated': False}}
host_state = host_manager.HostState('host1', cn1)
self.assertFalse(host_state.instances)
mock_get_by_host.return_value = objects.InstanceList(objects=[inst1])
hm._add_instance_info(context, cn1, host_state)
mock_get_by_host.assert_called_once_with(context, cn1.host)
self.assertTrue(host_state.instances)
self.assertEqual(host_state.instances['uuid1'], inst1)
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_recreate_instance_info(self, mock_get_by_host):
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
new_inst_list = objects.InstanceList(objects=[inst1, inst2])
mock_get_by_host.return_value = new_inst_list
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': True,
}}
self.host_manager._recreate_instance_info('fake_context', host_name)
new_info = self.host_manager._instance_info[host_name]
self.assertEqual(len(new_info['instances']), len(new_inst_list))
self.assertFalse(new_info['updated'])
def test_update_instance_info(self):
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
inst3 = fake_instance.fake_instance_obj('fake_context', uuid='ccc',
host=host_name)
inst4 = fake_instance.fake_instance_obj('fake_context', uuid='ddd',
host=host_name)
update = objects.InstanceList(objects=[inst3, inst4])
self.host_manager.update_instance_info('fake_context', host_name,
update)
new_info = self.host_manager._instance_info[host_name]
self.assertEqual(len(new_info['instances']), 4)
self.assertTrue(new_info['updated'])
def test_update_instance_info_unknown_host(self):
self.host_manager._recreate_instance_info = mock.MagicMock()
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
bad_host = 'bad_host'
inst3 = fake_instance.fake_instance_obj('fake_context', uuid='ccc',
host=bad_host)
inst_list3 = objects.InstanceList(objects=[inst3])
self.host_manager.update_instance_info('fake_context', bad_host,
inst_list3)
new_info = self.host_manager._instance_info[host_name]
self.host_manager._recreate_instance_info.assert_called_once_with(
'fake_context', bad_host)
self.assertEqual(len(new_info['instances']), len(orig_inst_dict))
self.assertFalse(new_info['updated'])
def test_delete_instance_info(self):
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
self.host_manager.delete_instance_info('fake_context', host_name,
inst1.uuid)
new_info = self.host_manager._instance_info[host_name]
self.assertEqual(len(new_info['instances']), 1)
self.assertTrue(new_info['updated'])
def test_delete_instance_info_unknown_host(self):
self.host_manager._recreate_instance_info = mock.MagicMock()
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
bad_host = 'bad_host'
self.host_manager.delete_instance_info('fake_context', bad_host, 'aaa')
new_info = self.host_manager._instance_info[host_name]
self.host_manager._recreate_instance_info.assert_called_once_with(
'fake_context', bad_host)
self.assertEqual(len(new_info['instances']), len(orig_inst_dict))
self.assertFalse(new_info['updated'])
def test_sync_instance_info(self):
self.host_manager._recreate_instance_info = mock.MagicMock()
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
self.host_manager.sync_instance_info('fake_context', host_name,
['bbb', 'aaa'])
new_info = self.host_manager._instance_info[host_name]
self.assertFalse(self.host_manager._recreate_instance_info.called)
self.assertTrue(new_info['updated'])
def test_sync_instance_info_fail(self):
self.host_manager._recreate_instance_info = mock.MagicMock()
host_name = 'fake_host'
inst1 = fake_instance.fake_instance_obj('fake_context', uuid='aaa',
host=host_name)
inst2 = fake_instance.fake_instance_obj('fake_context', uuid='bbb',
host=host_name)
orig_inst_dict = {inst1.uuid: inst1, inst2.uuid: inst2}
self.host_manager._instance_info = {
host_name: {
'instances': orig_inst_dict,
'updated': False,
}}
self.host_manager.sync_instance_info('fake_context', host_name,
['bbb', 'aaa', 'new'])
new_info = self.host_manager._instance_info[host_name]
self.host_manager._recreate_instance_info.assert_called_once_with(
'fake_context', host_name)
self.assertFalse(new_info['updated'])
class HostManagerChangedNodesTestCase(test.NoDBTestCase):
"""Test case for HostManager class."""
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(HostManagerChangedNodesTestCase, self).setUp()
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.host_manager = host_manager.HostManager()
self.host_manager = host_manager.HostManager()
self.fake_hosts = [
host_manager.HostState('host1', 'node1'),
host_manager.HostState('host2', 'node2'),
@@ -434,7 +683,9 @@ class HostManagerChangedNodesTestCase(test.NoDBTestCase):
host_manager.HostState('host4', 'node4')
]
def test_get_all_host_states(self):
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_get_all_host_states(self, mock_get_by_host):
mock_get_by_host.return_value = objects.InstanceList()
context = 'fake_context'
self.mox.StubOutWithMock(objects.ServiceList, 'get_by_binary')
@@ -448,7 +699,9 @@ class HostManagerChangedNodesTestCase(test.NoDBTestCase):
host_states_map = self.host_manager.host_state_map
self.assertEqual(len(host_states_map), 4)
def test_get_all_host_states_after_delete_one(self):
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_get_all_host_states_after_delete_one(self, mock_get_by_host):
mock_get_by_host.return_value = objects.InstanceList()
context = 'fake_context'
self.mox.StubOutWithMock(objects.ServiceList, 'get_by_binary')
@@ -470,7 +723,9 @@ class HostManagerChangedNodesTestCase(test.NoDBTestCase):
host_states_map = self.host_manager.host_state_map
self.assertEqual(len(host_states_map), 3)
def test_get_all_host_states_after_delete_all(self):
@mock.patch('nova.objects.InstanceList.get_by_host')
def test_get_all_host_states_after_delete_all(self, mock_get_by_host):
mock_get_by_host.return_value = objects.InstanceList()
context = 'fake_context'
self.mox.StubOutWithMock(objects.ServiceList, 'get_by_binary')

View File

@@ -19,6 +19,7 @@ Tests For IronicHostManager
import mock
import nova
from nova import exception
from nova import objects
from nova.objects import base as obj_base
@@ -42,13 +43,16 @@ class FakeFilterClass2(filters.BaseHostFilter):
class IronicHostManagerTestCase(test.NoDBTestCase):
"""Test case for IronicHostManager class."""
def setUp(self):
super(IronicHostManagerTestCase, self).setUp()
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.host_manager = ironic_host_manager.IronicHostManager()
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def test_manager_public_api_signatures(self, mock_init_aggs):
def setUp(self, mock_init_agg, mock_init_inst):
super(IronicHostManagerTestCase, self).setUp()
self.host_manager = ironic_host_manager.IronicHostManager()
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def test_manager_public_api_signatures(self, mock_init_aggs,
mock_init_inst):
self.assertPublicAPISignatures(host_manager.HostManager(),
self.host_manager)
@@ -72,7 +76,8 @@ class IronicHostManagerTestCase(test.NoDBTestCase):
ironic_fakes.COMPUTE_NODES)
self.mox.ReplayAll()
self.host_manager.get_all_host_states(context)
with mock.patch.object(nova.objects.InstanceList, 'get_by_host'):
self.host_manager.get_all_host_states(context)
host_states_map = self.host_manager.host_state_map
self.assertEqual(len(host_states_map), 4)
@@ -95,10 +100,11 @@ class IronicHostManagerTestCase(test.NoDBTestCase):
class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
"""Test case for IronicHostManager class."""
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(IronicHostManagerChangedNodesTestCase, self).setUp()
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.host_manager = ironic_host_manager.IronicHostManager()
self.host_manager = ironic_host_manager.IronicHostManager()
ironic_driver = "nova.virt.ironic.driver.IronicDriver"
supported_instances = [
objects.HVSpec.from_list(["i386", "baremetal", "baremetal"])]
@@ -149,8 +155,9 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
objects.ComputeNodeList.get_all(context).AndReturn(running_nodes)
self.mox.ReplayAll()
self.host_manager.get_all_host_states(context)
self.host_manager.get_all_host_states(context)
with mock.patch.object(nova.objects.InstanceList, 'get_by_host'):
self.host_manager.get_all_host_states(context)
self.host_manager.get_all_host_states(context)
host_states_map = self.host_manager.host_state_map
self.assertEqual(3, len(host_states_map))
@@ -170,8 +177,9 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
objects.ComputeNodeList.get_all(context).AndReturn([])
self.mox.ReplayAll()
self.host_manager.get_all_host_states(context)
self.host_manager.get_all_host_states(context)
with mock.patch.object(nova.objects.InstanceList, 'get_by_host'):
self.host_manager.get_all_host_states(context)
self.host_manager.get_all_host_states(context)
host_states_map = self.host_manager.host_state_map
self.assertEqual(0, len(host_states_map))
@@ -226,14 +234,15 @@ class IronicHostManagerChangedNodesTestCase(test.NoDBTestCase):
class IronicHostManagerTestFilters(test.NoDBTestCase):
"""Test filters work for IronicHostManager."""
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(IronicHostManagerTestFilters, self).setUp()
self.flags(scheduler_available_filters=['%s.%s' % (__name__, cls) for
cls in ['FakeFilterClass1',
'FakeFilterClass2']])
self.flags(scheduler_default_filters=['FakeFilterClass1'])
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.host_manager = ironic_host_manager.IronicHostManager()
self.host_manager = ironic_host_manager.IronicHostManager()
self.fake_hosts = [ironic_host_manager.IronicNodeState(
'fake_host%s' % x, 'fake-node') for x in range(1, 5)]
self.fake_hosts += [ironic_host_manager.IronicNodeState(

View File

@@ -36,7 +36,9 @@ class SchedulerManagerTestCase(test.NoDBTestCase):
driver_cls = driver.Scheduler
driver_cls_name = 'nova.scheduler.driver.Scheduler'
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(SchedulerManagerTestCase, self).setUp()
self.flags(scheduler_driver=self.driver_cls_name)
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
@@ -74,7 +76,10 @@ class SchedulerManagerTestCase(test.NoDBTestCase):
class SchedulerV3PassthroughTestCase(test.TestCase):
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(SchedulerV3PassthroughTestCase, self).setUp()
self.manager = manager.SchedulerManager()
self.proxy = manager._SchedulerManagerV3Proxy(self.manager)
@@ -92,10 +97,11 @@ class SchedulerTestCase(test.NoDBTestCase):
# So we can subclass this test and re-use tests if we need.
driver_cls = driver.Scheduler
def setUp(self):
@mock.patch.object(host_manager.HostManager, '_init_instance_info')
@mock.patch.object(host_manager.HostManager, '_init_aggregates')
def setUp(self, mock_init_agg, mock_init_inst):
super(SchedulerTestCase, self).setUp()
with mock.patch.object(host_manager.HostManager, '_init_aggregates'):
self.driver = self.driver_cls()
self.driver = self.driver_cls()
self.context = context.RequestContext('fake_user', 'fake_project')
self.topic = 'fake_topic'
self.servicegroup_api = servicegroup.API()