diff options
-rw-r--r-- | nova/cmd/scheduler.py | 1 | ||||
-rw-r--r-- | nova/conf/compute.py | 23 | ||||
-rw-r--r-- | nova/scheduler/caching_scheduler.py | 2 | ||||
-rw-r--r-- | nova/scheduler/client/report.py | 38 | ||||
-rw-r--r-- | nova/scheduler/filter_scheduler.py | 64 | ||||
-rw-r--r-- | nova/scheduler/host_manager.py | 15 | ||||
-rw-r--r-- | nova/tests/functional/libvirt/test_numa_servers.py | 2 | ||||
-rw-r--r-- | nova/tests/functional/libvirt/test_pci_sriov_servers.py | 3 | ||||
-rw-r--r-- | nova/tests/functional/test_server_group.py | 6 | ||||
-rw-r--r-- | nova/tests/unit/scheduler/client/test_report.py | 44 | ||||
-rw-r--r-- | nova/tests/unit/scheduler/fakes.py | 20 | ||||
-rw-r--r-- | nova/tests/unit/scheduler/test_caching_scheduler.py | 4 | ||||
-rw-r--r-- | nova/tests/unit/scheduler/test_filter_scheduler.py | 94 | ||||
-rw-r--r-- | releasenotes/notes/resource_providers_scheduler_db_filters-16b2ed3da00c51dd.yaml | 18 |
14 files changed, 311 insertions, 23 deletions
diff --git a/nova/cmd/scheduler.py b/nova/cmd/scheduler.py index f8cc35281a..ce164f627f 100644 --- a/nova/cmd/scheduler.py +++ b/nova/cmd/scheduler.py @@ -36,6 +36,7 @@ def main(): logging.setup(CONF, "nova") utils.monkey_patch() objects.register_all() + objects.Service.enable_min_version_cache() gmr.TextGuruMeditation.setup_autorun(version) diff --git a/nova/conf/compute.py b/nova/conf/compute.py index f9a80706e9..481a26b318 100644 --- a/nova/conf/compute.py +++ b/nova/conf/compute.py @@ -402,8 +402,12 @@ allocation_ratio_opts = [ default=0.0, min=0.0, help=""" -This option helps you specify virtual CPU to physical CPU allocation -ratio which affects all CPU filters. +This option helps you specify virtual CPU to physical CPU allocation ratio. + +From Ocata (15.0.0) this is used to influence the hosts selected by +the Placement API. Note that when Placement is used, the CoreFilter +is redundant, because the Placement API will have already filtered +out hosts that would have failed the CoreFilter. This configuration specifies ratio for CoreFilter which can be set per compute node. For AggregateCoreFilter, it will fall back to this @@ -423,7 +427,12 @@ Possible values: min=0.0, help=""" This option helps you specify virtual RAM to physical RAM -allocation ratio which affects all RAM filters. +allocation ratio. + +From Ocata (15.0.0) this is used to influence the hosts selected by +the Placement API. Note that when Placement is used, the RamFilter +is redundant, because the Placement API will have already filtered +out hosts that would have failed the RamFilter. This configuration specifies ratio for RamFilter which can be set per compute node. For AggregateRamFilter, it will fall back to this @@ -443,8 +452,12 @@ Possible values: min=0.0, help=""" This option helps you specify virtual disk to physical disk -allocation ratio used by the disk_filter.py script to determine if -a host has sufficient disk space to fit a requested instance. +allocation ratio. + +From Ocata (15.0.0) this is used to influence the hosts selected by +the Placement API. Note that when Placement is used, the DiskFilter +is redundant, because the Placement API will have already filtered +out hosts that would have failed the DiskFilter. A ratio greater than 1.0 will result in over-subscription of the available physical disk, which can be useful for more diff --git a/nova/scheduler/caching_scheduler.py b/nova/scheduler/caching_scheduler.py index 98033e23ac..fce16e6bfd 100644 --- a/nova/scheduler/caching_scheduler.py +++ b/nova/scheduler/caching_scheduler.py @@ -62,7 +62,7 @@ class CachingScheduler(filter_scheduler.FilterScheduler): # fetch the list of hosts. self.all_host_states = self._get_up_hosts(elevated) - def _get_all_host_states(self, context): + def _get_all_host_states(self, context, spec_obj): """Called from the filter scheduler, in a template pattern.""" if self.all_host_states is None: # NOTE(johngarbutt) We only get here when we a scheduler request diff --git a/nova/scheduler/client/report.py b/nova/scheduler/client/report.py index a290248ebb..8162b497ef 100644 --- a/nova/scheduler/client/report.py +++ b/nova/scheduler/client/report.py @@ -15,6 +15,7 @@ import functools import re +from six.moves.urllib import parse import time from keystoneauth1 import exceptions as ks_exc @@ -208,6 +209,43 @@ class SchedulerReportClient(object): url, endpoint_filter=self.ks_filter, raise_exc=False) + # TODO(sbauza): Change that poor interface into passing a rich versioned + # object that would provide the ResourceProvider requirements. + @safe_connect + def get_filtered_resource_providers(self, filters): + """Returns a list of ResourceProviders matching the requirements + expressed by the filters argument, which can include a dict named + 'resources' where amounts are keyed by resource class names. + + eg. filters = {'resources': {'VCPU': 1}} + """ + resources = filters.pop("resources", None) + if resources: + resource_query = ",".join(sorted("%s:%s" % (rc, amount) + for (rc, amount) in resources.items())) + filters['resources'] = resource_query + resp = self.get("/resource_providers?%s" % parse.urlencode(filters), + version='1.4') + if resp.status_code == 200: + data = resp.json() + raw_rps = data.get('resource_providers', []) + rps = [objects.ResourceProvider(uuid=rp['uuid'], + name=rp['name'], + generation=rp['generation'], + ) for rp in raw_rps] + return objects.ResourceProviderList(objects=rps) + else: + msg = _LE("Failed to retrieve filtered list of resource providers " + "from placement API for filters %(filters)s. " + "Got %(status_code)d: %(err_text)s.") + args = { + 'filters': filters, + 'status_code': resp.status_code, + 'err_text': resp.text, + } + LOG.error(msg, args) + return None + @safe_connect def _get_provider_aggregates(self, rp_uuid): """Queries the placement API for a resource provider's aggregates. diff --git a/nova/scheduler/filter_scheduler.py b/nova/scheduler/filter_scheduler.py index b128e81ca0..658a7169ab 100644 --- a/nova/scheduler/filter_scheduler.py +++ b/nova/scheduler/filter_scheduler.py @@ -27,7 +27,10 @@ from six.moves import range import nova.conf from nova import exception from nova.i18n import _ +from nova import objects +from nova.objects import fields from nova import rpc +from nova.scheduler import client as scheduler_client from nova.scheduler import driver @@ -40,6 +43,10 @@ class FilterScheduler(driver.Scheduler): def __init__(self, *args, **kwargs): super(FilterScheduler, self).__init__(*args, **kwargs) self.notifier = rpc.get_notifier('scheduler') + # TODO(sbauza): It seems weird that we load a scheduler client for + # the FilterScheduler but it will be the PlacementClient later on once + # we split the needed methods into a separate library. + self.scheduler_client = scheduler_client.SchedulerClient() def select_destinations(self, context, spec_obj): """Selects a filtered set of hosts and nodes.""" @@ -93,7 +100,7 @@ class FilterScheduler(driver.Scheduler): # Note: remember, we are using an iterator here. So only # traverse this list once. This can bite you if the hosts # are being scanned in a filter or weighing function. - hosts = self._get_all_host_states(elevated) + hosts = self._get_all_host_states(elevated, spec_obj) selected_hosts = [] num_instances = spec_obj.num_instances @@ -129,6 +136,57 @@ class FilterScheduler(driver.Scheduler): spec_obj.instance_group.obj_reset_changes(['hosts']) return selected_hosts - def _get_all_host_states(self, context): + def _get_resources_per_request_spec(self, spec_obj): + resources = {} + + resources[fields.ResourceClass.VCPU] = spec_obj.vcpus + resources[fields.ResourceClass.MEMORY_MB] = spec_obj.memory_mb + + requested_disk_mb = (1024 * (spec_obj.root_gb + + spec_obj.ephemeral_gb) + + spec_obj.swap) + # NOTE(sbauza): Disk request is expressed in MB but we count + # resources in GB. Since there could be a remainder of the division + # by 1024, we need to ceil the result to the next bigger Gb so we + # can be sure there would be enough disk space in the destination + # to sustain the request. + # FIXME(sbauza): All of that could be using math.ceil() but since + # we support both py2 and py3, let's fake it until we only support + # py3. + requested_disk_gb = requested_disk_mb // 1024 + if requested_disk_mb % 1024 != 0: + # Let's ask for a bit more space since we count in GB + requested_disk_gb += 1 + # NOTE(sbauza): Some flavors provide zero size for disk values, we need + # to avoid asking for disk usage. + if requested_disk_gb != 0: + resources[fields.ResourceClass.DISK_GB] = requested_disk_gb + + return resources + + def _get_all_host_states(self, context, spec_obj): """Template method, so a subclass can implement caching.""" - return self.host_manager.get_all_host_states(context) + # NOTE(sbauza): Since Newton compute nodes require a configuration + # change to request the Placement API, and given it goes against + # our rolling upgrade process, we define a graceful period for allowing + # clouds that are not fully upgraded to Ocata to still be able to + # have instances being scheduled on old nodes. + service_version = objects.Service.get_minimum_version( + context, 'nova-compute') + # TODO(sbauza): Remove that version check in Pike so we fully call + # the placement API anyway. + if service_version < 16: + LOG.debug("Skipping call to placement, as upgrade in progress.") + return self.host_manager.get_all_host_states(context) + filters = {'resources': self._get_resources_per_request_spec(spec_obj)} + reportclient = self.scheduler_client.reportclient + rps = reportclient.get_filtered_resource_providers(filters) + # NOTE(sbauza): In case the Placement service is not running yet or + # when returning an exception, we wouldn't get any ResourceProviders. + # If so, let's return an empty list so _schedule would raise a + # NoValidHosts. + if not rps: + return [] + compute_uuids = [rp.uuid for rp in rps] + return self.host_manager.get_host_states_by_uuids(context, + compute_uuids) diff --git a/nova/scheduler/host_manager.py b/nova/scheduler/host_manager.py index 0abd233af1..366f6f5d16 100644 --- a/nova/scheduler/host_manager.py +++ b/nova/scheduler/host_manager.py @@ -576,17 +576,28 @@ class HostManager(object): return self.weight_handler.get_weighed_objects(self.weighers, hosts, spec_obj) + def get_host_states_by_uuids(self, context, compute_uuids): + compute_nodes = objects.ComputeNodeList.get_all_by_uuids(context, + compute_uuids) + return self._get_host_states(context, compute_nodes) + def get_all_host_states(self, context): """Returns a list of HostStates that represents all the hosts 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 = objects.ComputeNodeList.get_all(context) + return self._get_host_states(context, compute_nodes) + + def _get_host_states(self, context, compute_nodes): + """Returns a tuple of HostStates given a list of computes. + Also updates the HostStates internal mapping for the HostManager. + """ service_refs = {service.host: service for service in objects.ServiceList.get_by_binary( context, 'nova-compute', include_disabled=True)} # Get resource usage across the available compute nodes: - compute_nodes = objects.ComputeNodeList.get_all(context) seen_nodes = set() for compute in compute_nodes: service = service_refs.get(compute.host) @@ -621,7 +632,7 @@ class HostManager(object): "from scheduler"), {'host': host, 'node': node}) del self.host_state_map[state_key] - return six.itervalues(self.host_state_map) + return (self.host_state_map[host] for host in seen_nodes) def _get_aggregates_info(self, host): return [self.aggs_by_id[agg_id] for agg_id in diff --git a/nova/tests/functional/libvirt/test_numa_servers.py b/nova/tests/functional/libvirt/test_numa_servers.py index 3f1ca43744..a6066eb9d5 100644 --- a/nova/tests/functional/libvirt/test_numa_servers.py +++ b/nova/tests/functional/libvirt/test_numa_servers.py @@ -21,6 +21,7 @@ from oslo_log import log as logging from nova import context as nova_context from nova import objects from nova import test +from nova.tests import fixtures as nova_fixtures from nova.tests.functional.test_servers import ServersTestBase from nova.tests.unit import fake_network from nova.tests.unit.virt.libvirt import fake_imagebackend @@ -75,6 +76,7 @@ class NUMAServersTest(ServersTestBase): 'nova.virt.libvirt.guest.libvirt', fakelibvirt)) self.useFixture(fakelibvirt.FakeLibvirtFixture()) + self.useFixture(nova_fixtures.PlacementFixture()) def _setup_compute_service(self): pass diff --git a/nova/tests/functional/libvirt/test_pci_sriov_servers.py b/nova/tests/functional/libvirt/test_pci_sriov_servers.py index 04cb23502c..1dfadb7d37 100644 --- a/nova/tests/functional/libvirt/test_pci_sriov_servers.py +++ b/nova/tests/functional/libvirt/test_pci_sriov_servers.py @@ -20,6 +20,7 @@ from oslo_log import log as logging from nova.objects import fields from nova import test +from nova.tests import fixtures as nova_fixtures from nova.tests.functional.test_servers import ServersTestBase from nova.tests.unit import fake_network from nova.tests.unit.virt.libvirt import fake_libvirt_utils @@ -89,6 +90,8 @@ class SRIOVServersTest(ServersTestBase): fakelibvirt)) self.useFixture(fakelibvirt.FakeLibvirtFixture()) + self.useFixture(nova_fixtures.PlacementFixture()) + self.compute_started = False def _setup_compute_service(self): diff --git a/nova/tests/functional/test_server_group.py b/nova/tests/functional/test_server_group.py index d250f53ace..cc7bec29f7 100644 --- a/nova/tests/functional/test_server_group.py +++ b/nova/tests/functional/test_server_group.py @@ -74,6 +74,7 @@ class ServerGroupTestBase(test.TestCase, self.useFixture(policy_fixture.RealPolicyFixture()) self.useFixture(nova_fixtures.NeutronFixture(self)) + self.useFixture(nova_fixtures.PlacementFixture()) api_fixture = self.useFixture(nova_fixtures.OSAPIFixture( api_version='v2.1')) @@ -141,6 +142,10 @@ class ServerGroupTestV21(ServerGroupTestBase): self.compute = self.start_service('compute') # NOTE(gibi): start a second compute host to be able to test affinity + # NOTE(sbauza): Make sure the FakeDriver returns a different nodename + # for the second compute node. + fake.set_nodes(['host2']) + self.addCleanup(fake.restore_nodes) self.compute2 = self.start_service('compute', host='host2') fake_network.set_stub_network_methods(self) @@ -361,6 +366,7 @@ class ServerGroupTestV21(ServerGroupTestBase): def test_migrate_with_anti_affinity(self): # Start additional host to test migration with anti-affinity + fake.set_nodes(['host3']) self.start_service('compute', host='host3') created_group = self.api.post_server_groups(self.anti_affinity) diff --git a/nova/tests/unit/scheduler/client/test_report.py b/nova/tests/unit/scheduler/client/test_report.py index 88d2431759..590ce5bfc6 100644 --- a/nova/tests/unit/scheduler/client/test_report.py +++ b/nova/tests/unit/scheduler/client/test_report.py @@ -13,6 +13,7 @@ from keystoneauth1 import exceptions as ks_exc import mock import six +from six.moves.urllib import parse import nova.conf from nova import context @@ -268,6 +269,49 @@ class TestProviderOperations(SchedulerReportClientTestCase): mock.sentinel.name, ) + def test_get_filtered_resource_providers(self): + uuid = uuids.compute_node + resp_mock = mock.Mock(status_code=200) + json_data = { + 'resource_providers': [ + {'uuid': uuid, + 'name': uuid, + 'generation': 42} + ], + } + filters = {'resources': {'VCPU': 1, 'MEMORY_MB': 1024}} + resp_mock.json.return_value = json_data + self.ks_sess_mock.get.return_value = resp_mock + + result = self.client.get_filtered_resource_providers(filters) + + expected_provider = objects.ResourceProvider( + uuid=uuid, + name=uuid, + generation=42, + ) + expected_url = '/resource_providers?%s' % parse.urlencode( + {'resources': 'MEMORY_MB:1024,VCPU:1'}) + self.ks_sess_mock.get.assert_called_once_with( + expected_url, endpoint_filter=mock.ANY, raise_exc=False, + headers={'OpenStack-API-Version': 'placement 1.4'}) + self.assertTrue(obj_base.obj_equal_prims(expected_provider, + result[0])) + + def test_get_filtered_resource_providers_not_found(self): + # Ensure _get_resource_provider() just returns None when the placement + # API doesn't find a resource provider matching a UUID + resp_mock = mock.Mock(status_code=404) + self.ks_sess_mock.get.return_value = resp_mock + + result = self.client.get_filtered_resource_providers({'foo': 'bar'}) + + expected_url = '/resource_providers?foo=bar' + self.ks_sess_mock.get.assert_called_once_with( + expected_url, endpoint_filter=mock.ANY, raise_exc=False, + headers={'OpenStack-API-Version': 'placement 1.4'}) + self.assertIsNone(result) + def test_get_resource_provider_found(self): # Ensure _get_resource_provider() returns a ResourceProvider object if # it finds a resource provider record from the placement API diff --git a/nova/tests/unit/scheduler/fakes.py b/nova/tests/unit/scheduler/fakes.py index bbf1761022..e29dd2bc3c 100644 --- a/nova/tests/unit/scheduler/fakes.py +++ b/nova/tests/unit/scheduler/fakes.py @@ -22,6 +22,7 @@ import six from nova import objects from nova.scheduler import driver from nova.scheduler import host_manager +from nova.tests import uuidsentinel NUMA_TOPOLOGY = objects.NUMATopology( cells=[ @@ -122,6 +123,25 @@ COMPUTE_NODES = [ host='fake', hypervisor_hostname='fake-hyp'), ] +RESOURCE_PROVIDERS = [ + objects.ResourceProvider( + uuid=uuidsentinel.rp1, + name='host1', + generation=1), + objects.ResourceProvider( + uuid=uuidsentinel.rp2, + name='host2', + generation=1), + objects.ResourceProvider( + uuid=uuidsentinel.rp3, + name='host3', + generation=1), + objects.ResourceProvider( + uuid=uuidsentinel.rp4, + name='host4', + generation=1), +] + SERVICES = [ objects.Service(host='host1', disabled=False), objects.Service(host='host2', disabled=True), diff --git a/nova/tests/unit/scheduler/test_caching_scheduler.py b/nova/tests/unit/scheduler/test_caching_scheduler.py index 9fbb01f172..fd28885ac2 100644 --- a/nova/tests/unit/scheduler/test_caching_scheduler.py +++ b/nova/tests/unit/scheduler/test_caching_scheduler.py @@ -48,7 +48,7 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase): def test_get_all_host_states_returns_cached_value(self, mock_up_hosts): self.driver.all_host_states = [] - self.driver._get_all_host_states(self.context) + self.driver._get_all_host_states(self.context, None) self.assertFalse(mock_up_hosts.called) self.assertEqual([], self.driver.all_host_states) @@ -58,7 +58,7 @@ class CachingSchedulerTestCase(test_scheduler.SchedulerTestCase): def test_get_all_host_states_loads_hosts(self, mock_up_hosts): mock_up_hosts.return_value = ["asdf"] - result = self.driver._get_all_host_states(self.context) + result = self.driver._get_all_host_states(self.context, None) self.assertTrue(mock_up_hosts.called) self.assertEqual(["asdf"], self.driver.all_host_states) diff --git a/nova/tests/unit/scheduler/test_filter_scheduler.py b/nova/tests/unit/scheduler/test_filter_scheduler.py index 677d75627a..d5820b5ea2 100644 --- a/nova/tests/unit/scheduler/test_filter_scheduler.py +++ b/nova/tests/unit/scheduler/test_filter_scheduler.py @@ -42,12 +42,18 @@ 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', + @mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids', return_value=fakes.COMPUTE_NODES) + @mock.patch('nova.scheduler.client.report.SchedulerReportClient.' + 'get_filtered_resource_providers', + return_value=fakes.RESOURCE_PROVIDERS) + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=objects.service.SERVICE_VERSION) @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, + def test_schedule_happy_day(self, mock_get_extra, mock_get_mv, + mock_get_rps, mock_get_all, mock_by_host, mock_get_by_binary): """Make sure there's nothing glaringly wrong with _schedule() by doing a happy day pass through. @@ -68,6 +74,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): flavor=objects.Flavor(memory_mb=512, root_gb=512, ephemeral_gb=0, + swap=0, vcpus=1), project_id=1, os_type='Linux', @@ -115,12 +122,18 @@ 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', + @mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids', return_value=fakes.COMPUTE_NODES) + @mock.patch('nova.scheduler.client.report.SchedulerReportClient.' + 'get_filtered_resource_providers', + return_value=fakes.RESOURCE_PROVIDERS) + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=objects.service.SERVICE_VERSION) @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, + def test_schedule_host_pool(self, mock_get_extra, mock_get_mv, + mock_get_rps, mock_get_all, mock_by_host, mock_get_by_binary): """Make sure the host_subset_size property works properly.""" @@ -134,6 +147,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): flavor=objects.Flavor(root_gb=512, memory_mb=512, ephemeral_gb=0, + swap=0, vcpus=1), pci_requests=None, numa_topology=None, @@ -150,12 +164,18 @@ 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', + @mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids', return_value=fakes.COMPUTE_NODES) + @mock.patch('nova.scheduler.client.report.SchedulerReportClient.' + 'get_filtered_resource_providers', + return_value=fakes.RESOURCE_PROVIDERS) + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=objects.service.SERVICE_VERSION) @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, + def test_schedule_large_host_pool(self, mock_get_extra, mock_get_mv, + mock_get_rps, mock_get_all, mock_by_host, mock_get_by_binary): """Hosts should still be chosen if pool size is larger than number of filtered hosts. @@ -171,6 +191,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): flavor=objects.Flavor(root_gb=512, memory_mb=512, ephemeral_gb=0, + swap=0, vcpus=1), pci_requests=None, numa_topology=None, @@ -187,12 +208,18 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): @mock.patch('nova.scheduler.host_manager.HostManager._get_instance_info') @mock.patch('nova.objects.ServiceList.get_by_binary', return_value=fakes.SERVICES) - @mock.patch('nova.objects.ComputeNodeList.get_all', + @mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids', return_value=fakes.COMPUTE_NODES) + @mock.patch('nova.scheduler.client.report.SchedulerReportClient.' + 'get_filtered_resource_providers', + return_value=fakes.RESOURCE_PROVIDERS) + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=objects.service.SERVICE_VERSION) @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_cn_get_all, + def test_schedule_chooses_best_host(self, mock_get_extra, mock_get_mv, + mock_get_rps, mock_cn_get_all, mock_get_by_binary, mock_get_inst_info): """If host_subset_size is 1, the largest host with greatest weight @@ -219,6 +246,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): flavor=objects.Flavor(root_gb=512, memory_mb=512, ephemeral_gb=0, + swap=0, vcpus=1), pci_requests=None, numa_topology=None, @@ -234,15 +262,33 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): self.assertEqual(50, hosts[0].weight) + # TODO(sbauza): Remove that unit test in Pike + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=15) + def test_get_all_host_states_with_newton_computes(self, mock_get_mv): + """If at least one compute node is older than Ocata, then we should + not call the placement API. + """ + with mock.patch.object(self.driver.host_manager, + 'get_all_host_states') as mock_get_hosts: + self.driver._get_all_host_states(self.context, mock.sentinel.spec) + mock_get_hosts.assert_called_once_with(self.context) + @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', + @mock.patch('nova.objects.ComputeNodeList.get_all_by_uuids', return_value=fakes.COMPUTE_NODES) + @mock.patch('nova.scheduler.client.report.SchedulerReportClient.' + 'get_filtered_resource_providers', + return_value=fakes.RESOURCE_PROVIDERS) + @mock.patch('nova.objects.Service.get_minimum_version', + return_value=objects.service.SERVICE_VERSION) @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, + def test_select_destinations(self, mock_get_extra, mock_get_mv, + mock_get_rps, mock_get_all, mock_by_host, mock_get_by_binary): """select_destinations is basically a wrapper around _schedule(). @@ -269,6 +315,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): flavor=objects.Flavor(memory_mb=512, root_gb=512, ephemeral_gb=0, + swap=0, vcpus=1), project_id=1, os_type='Linux', @@ -286,6 +333,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): (host, node) = (dests[0]['host'], dests[0]['nodename']) self.assertEqual(host, selected_hosts[0]) self.assertEqual(node, selected_nodes[0]) + mock_get_mv.assert_called_once_with(mock.ANY, 'nova-compute') @mock.patch.object(filter_scheduler.FilterScheduler, '_schedule') def test_select_destinations_notifications(self, mock_schedule): @@ -332,3 +380,29 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase): # Make sure that the consumed hosts have chance to be reverted. for host in consumed_hosts: self.assertIsNone(host.obj.updated) + + def _test_get_resources_per_request_spec(self, flavor, expected): + fake_spec = objects.RequestSpec(flavor=flavor) + resources = self.driver._get_resources_per_request_spec(fake_spec) + self.assertEqual(expected, resources) + + def test_get_resources_per_request_spec(self): + flavor = objects.Flavor(vcpus=1, + memory_mb=1024, + root_gb=10, + ephemeral_gb=5, + swap=0) + expected_resources = {'VCPU': 1, + 'MEMORY_MB': 1024, + 'DISK_GB': 15} + self._test_get_resources_per_request_spec(flavor, expected_resources) + + def test_get_resources_per_request_spec_with_no_disk(self): + flavor = objects.Flavor(vcpus=1, + memory_mb=1024, + root_gb=0, + ephemeral_gb=0, + swap=0) + expected_resources = {'VCPU': 1, + 'MEMORY_MB': 1024} + self._test_get_resources_per_request_spec(flavor, expected_resources) diff --git a/releasenotes/notes/resource_providers_scheduler_db_filters-16b2ed3da00c51dd.yaml b/releasenotes/notes/resource_providers_scheduler_db_filters-16b2ed3da00c51dd.yaml new file mode 100644 index 0000000000..afda0446a3 --- /dev/null +++ b/releasenotes/notes/resource_providers_scheduler_db_filters-16b2ed3da00c51dd.yaml @@ -0,0 +1,18 @@ +--- +features: + - nova-scheduler process is now calling the placement API in order to get + a list of valid destinations before calling the filters. That works only + if all your compute nodes are fully upgraded to Ocata. If some + nodes are not upgraded, the scheduler will still lookup from the DB + instead which is less performant. +upgrade: + - Since the Placement service is now mandatory in Ocata, you need to deploy + it and amend your compute node configuration with correct placement + instructions before restarting nova-compute or the compute node will + refuse to start. + - If by Newton (14.0.0), you don't use any of the CoreFilter, RamFilter or + DiskFilter, then please modify all your compute node's configuration by + amending either ``cpu_allocation_ratio`` (if you don't use CoreFilter) or + ``ram_allocation_ratio`` (if you don't use RamFilter) or + ``disk_allocation_ratio`` (if you don't use DiskFilter) by putting a 9999.0 + value for the ratio before upgrading the nova-scheduler to Ocata. |