Rip out the SchedulerClient

Since all remaining SchedulerClient methods were direct passthroughs to
the SchedulerQueryClient, and nothing special is being done anymore to
instantiate that client, the SchedulerClient is no longer necessary. All
references to it are replaced with direct references to
SchedulerQueryClient.

Change-Id: I57dd199a7c5e762d97a600307aa13a7aeb62d2b2
This commit is contained in:
Eric Fried 2018-11-09 17:16:35 -06:00
parent 86e94a11f7
commit 4c34ab574e
14 changed files with 84 additions and 215 deletions

View File

@ -76,8 +76,8 @@ from nova.policies import servers as servers_policies
import nova.policy
from nova import profiler
from nova import rpc
from nova.scheduler.client import query as queryclient
from nova.scheduler.client import report as reportclient
from nova.scheduler.client import query
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
from nova import utils
@ -2123,7 +2123,7 @@ class API(base.Base):
@property
def placementclient(self):
if self._placementclient is None:
self._placementclient = reportclient.SchedulerReportClient()
self._placementclient = report.SchedulerReportClient()
return self._placementclient
def _local_delete(self, context, instance, bdms, delete_type, cb):
@ -5242,14 +5242,14 @@ class AggregateAPI(base.Base):
"""Sub-set of the Compute Manager API for managing host aggregates."""
def __init__(self, **kwargs):
self.compute_rpcapi = compute_rpcapi.ComputeAPI()
self.scheduler_client = queryclient.SchedulerQueryClient()
self.query_client = query.SchedulerQueryClient()
self._placement_client = None # Lazy-load on first access.
super(AggregateAPI, self).__init__(**kwargs)
@property
def placement_client(self):
if self._placement_client is None:
self._placement_client = reportclient.SchedulerReportClient()
self._placement_client = report.SchedulerReportClient()
return self._placement_client
@wrap_exception()
@ -5261,7 +5261,7 @@ class AggregateAPI(base.Base):
if availability_zone:
aggregate.metadata = {'availability_zone': availability_zone}
aggregate.create()
self.scheduler_client.update_aggregates(context, [aggregate])
self.query_client.update_aggregates(context, [aggregate])
return aggregate
def get_aggregate(self, context, aggregate_id):
@ -5288,7 +5288,7 @@ class AggregateAPI(base.Base):
if values:
aggregate.update_metadata(values)
aggregate.updated_at = timeutils.utcnow()
self.scheduler_client.update_aggregates(context, [aggregate])
self.query_client.update_aggregates(context, [aggregate])
# If updated values include availability_zones, then the cache
# which stored availability_zones and host need to be reset
if values.get('availability_zone'):
@ -5302,7 +5302,7 @@ class AggregateAPI(base.Base):
self.is_safe_to_update_az(context, metadata, aggregate=aggregate,
action_name=AGGREGATE_ACTION_UPDATE_META)
aggregate.update_metadata(metadata)
self.scheduler_client.update_aggregates(context, [aggregate])
self.query_client.update_aggregates(context, [aggregate])
# If updated metadata include availability_zones, then the cache
# which stored availability_zones and host need to be reset
if metadata and metadata.get('availability_zone'):
@ -5330,7 +5330,7 @@ class AggregateAPI(base.Base):
raise exception.InvalidAggregateActionDelete(
aggregate_id=aggregate_id, reason=msg)
aggregate.destroy()
self.scheduler_client.delete_aggregate(context, aggregate)
self.query_client.delete_aggregate(context, aggregate)
compute_utils.notify_about_aggregate_update(context,
"delete.end",
aggregate_payload)
@ -5439,7 +5439,7 @@ class AggregateAPI(base.Base):
hosts=[host_name], aggregate=aggregate)
aggregate.add_host(host_name)
self.scheduler_client.update_aggregates(context, [aggregate])
self.query_client.update_aggregates(context, [aggregate])
try:
self.placement_client.aggregate_add_host(
context, aggregate.uuid, host_name)
@ -5505,7 +5505,7 @@ class AggregateAPI(base.Base):
phase=fields_obj.NotificationPhase.START)
aggregate.delete_host(host_name)
self.scheduler_client.update_aggregates(context, [aggregate])
self.query_client.update_aggregates(context, [aggregate])
try:
self.placement_client.aggregate_remove_host(
context, aggregate.uuid, host_name)

View File

@ -87,7 +87,7 @@ from nova.objects import migrate_data as migrate_data_obj
from nova.pci import whitelist
from nova import rpc
from nova import safe_utils
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova import utils
from nova.virt import block_device as driver_block_device
from nova.virt import configdrive
@ -506,7 +506,7 @@ class ComputeManager(manager.Manager):
self.is_neutron_security_groups = (
openstack_driver.is_neutron_security_groups())
self.cells_rpcapi = cells_rpcapi.CellsAPI()
self.scheduler_client = scheduler_client.SchedulerClient()
self.query_client = query.SchedulerQueryClient()
self._reportclient = None
self._resource_tracker = None
self.instance_events = InstanceEvents()
@ -1726,16 +1726,16 @@ class ComputeManager(manager.Manager):
if isinstance(instance, obj_instance.Instance):
instance = objects.InstanceList(objects=[instance])
context = context.elevated()
self.scheduler_client.update_instance_info(context, self.host,
instance)
self.query_client.update_instance_info(context, self.host,
instance)
def _delete_scheduler_instance_info(self, context, instance_uuid):
"""Sends the uuid of the deleted Instance to the Scheduler client."""
if not self.send_instance_updates:
return
context = context.elevated()
self.scheduler_client.delete_instance_info(context, self.host,
instance_uuid)
self.query_client.delete_instance_info(context, self.host,
instance_uuid)
@periodic_task.periodic_task(spacing=CONF.scheduler_instance_sync_interval)
def _sync_scheduler_instance_info(self, context):
@ -1746,7 +1746,7 @@ class ComputeManager(manager.Manager):
expected_attrs=[],
use_slave=True)
uuids = [instance.uuid for instance in instances]
self.scheduler_client.sync_instance_info(context, self.host, uuids)
self.query_client.sync_instance_info(context, self.host, uuids)
def _notify_about_instance_usage(self, context, instance, event_suffix,
network_info=None, extra_usage_info=None,

View File

@ -42,7 +42,7 @@ from nova.pci import manager as pci_manager
from nova.pci import request as pci_request
from nova import rc_fields as fields
from nova import rpc
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova.scheduler.client import report
from nova import utils
from nova.virt import hardware
@ -147,7 +147,7 @@ class ResourceTracker(object):
monitor_handler = monitors.MonitorHandler(self)
self.monitors = monitor_handler.monitors
self.old_resources = collections.defaultdict(objects.ComputeNode)
self.scheduler_client = scheduler_client.SchedulerClient()
self.query_client = query.SchedulerQueryClient()
self.reportclient = report.SchedulerReportClient()
self.ram_allocation_ratio = CONF.ram_allocation_ratio
self.cpu_allocation_ratio = CONF.cpu_allocation_ratio
@ -919,8 +919,8 @@ class ResourceTracker(object):
# object of compute_node; instead the inventory data for these
# resource is reported by driver's get_inventory(). So even there
# is no resource change for compute_node as above, we need proceed
# to get inventory and use scheduler_client interfaces to update
# inventory to placement. It's scheduler_client's responsibility to
# to get inventory and use report client interfaces to update
# inventory to placement. It's report client's responsibility to
# ensure the update request to placement only happens when inventory
# is changed.
nodename = compute_node.hypervisor_hostname

View File

@ -50,7 +50,7 @@ from nova.objects import base as nova_object
from nova.objects import fields
from nova import profiler
from nova import rpc
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
@ -233,7 +233,7 @@ class ComputeTaskManager(base.Base):
self.image_api = image.API()
self.network_api = network.API()
self.servicegroup_api = servicegroup.API()
self.scheduler_client = scheduler_client.SchedulerClient()
self.query_client = query.SchedulerQueryClient()
self.report_client = report.SchedulerReportClient()
self.notifier = rpc.get_notifier('compute', CONF.host)
# Help us to record host in EventReporter
@ -465,7 +465,7 @@ class ComputeTaskManager(base.Base):
disk_over_commit, migration,
self.compute_rpcapi,
self.servicegroup_api,
self.scheduler_client,
self.query_client,
self.report_client,
request_spec)
@ -474,7 +474,7 @@ class ComputeTaskManager(base.Base):
return migrate.MigrationTask(context, instance, flavor,
request_spec, clean_shutdown,
self.compute_rpcapi,
self.scheduler_client, self.report_client,
self.query_client, self.report_client,
host_list)
def _destroy_build_request(self, context, instance):
@ -747,9 +747,9 @@ class ComputeTaskManager(base.Base):
instance_uuids=None, return_alternates=False):
scheduler_utils.setup_instance_group(context, request_spec)
with timeutils.StopWatch() as timer:
host_lists = self.scheduler_client.select_destinations(context,
request_spec, instance_uuids, return_objects=True,
return_alternates=return_alternates)
host_lists = self.query_client.select_destinations(
context, request_spec, instance_uuids, return_objects=True,
return_alternates=return_alternates)
LOG.debug('Took %0.2f seconds to select destinations for %s '
'instance(s).', timer.elapsed(), len(instance_uuids))
return host_lists

View File

@ -46,7 +46,7 @@ def supports_extended_port_binding(context, host):
class LiveMigrationTask(base.TaskBase):
def __init__(self, context, instance, destination,
block_migration, disk_over_commit, migration, compute_rpcapi,
servicegroup_api, scheduler_client, report_client,
servicegroup_api, query_client, report_client,
request_spec=None):
super(LiveMigrationTask, self).__init__(context, instance)
self.destination = destination
@ -58,7 +58,7 @@ class LiveMigrationTask(base.TaskBase):
self.compute_rpcapi = compute_rpcapi
self.servicegroup_api = servicegroup_api
self.scheduler_client = scheduler_client
self.query_client = query_client
self.report_client = report_client
self.request_spec = request_spec
self._source_cn = None
@ -380,7 +380,7 @@ class LiveMigrationTask(base.TaskBase):
self._check_not_over_max_retries(attempted_hosts)
request_spec.ignore_hosts = attempted_hosts
try:
selection_lists = self.scheduler_client.select_destinations(
selection_lists = self.query_client.select_destinations(
self.context, request_spec, [self.instance.uuid],
return_objects=True, return_alternates=False)
# We only need the first item in the first list, as there is

View File

@ -112,14 +112,14 @@ def revert_allocation_for_migration(context, source_cn, instance, migration):
class MigrationTask(base.TaskBase):
def __init__(self, context, instance, flavor,
request_spec, clean_shutdown, compute_rpcapi,
scheduler_client, report_client, host_list):
query_client, report_client, host_list):
super(MigrationTask, self).__init__(context, instance)
self.clean_shutdown = clean_shutdown
self.request_spec = request_spec
self.flavor = flavor
self.compute_rpcapi = compute_rpcapi
self.scheduler_client = scheduler_client
self.query_client = query_client
self.reportclient = report_client
self.host_list = host_list
@ -227,7 +227,7 @@ class MigrationTask(base.TaskBase):
# pop the first alternate from the list to use for the destination, and
# pass the remaining alternates to the compute.
if self.host_list is None:
selection_lists = self.scheduler_client.select_destinations(
selection_lists = self.query_client.select_destinations(
self.context, self.request_spec, [self.instance.uuid],
return_objects=True, return_alternates=True)
# Since there is only ever one instance to migrate per call, we

View File

@ -1,45 +0,0 @@
# Copyright (c) 2014 Red Hat, Inc.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from nova.scheduler.client import query
class SchedulerClient(object):
"""Client library for placing calls to the scheduler."""
def __init__(self):
self.queryclient = query.SchedulerQueryClient()
def select_destinations(self, context, spec_obj, instance_uuids,
return_objects=False, return_alternates=False):
return self.queryclient.select_destinations(context, spec_obj,
instance_uuids, return_objects, return_alternates)
def update_aggregates(self, context, aggregates):
self.queryclient.update_aggregates(context, aggregates)
def delete_aggregate(self, context, aggregate):
self.queryclient.delete_aggregate(context, aggregate)
def update_instance_info(self, context, host_name, instance_info):
self.queryclient.update_instance_info(context, host_name,
instance_info)
def delete_instance_info(self, context, host_name, instance_uuid):
self.queryclient.delete_instance_info(context, host_name,
instance_uuid)
def sync_instance_info(self, context, host_name, instance_uuids):
self.queryclient.sync_instance_info(context, host_name, instance_uuids)

View File

@ -4577,7 +4577,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
self.assertFalse(log_mock.called)
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'update_instance_info')
def test_update_scheduler_instance_info(self, mock_update):
instance = objects.Instance(uuid=uuids.instance)
@ -4592,7 +4592,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
self.assertIsInstance(args[2], objects.InstanceList)
self.assertEqual(args[2].objects[0], instance)
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'delete_instance_info')
def test_delete_scheduler_instance_info(self, mock_delete):
self.compute._delete_scheduler_instance_info(self.context,
@ -4660,7 +4660,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
@mock.patch.object(nova.context.RequestContext, 'elevated')
@mock.patch.object(nova.objects.InstanceList, 'get_by_host')
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'sync_instance_info')
def test_sync_scheduler_instance_info(self, mock_sync, mock_get_by_host,
mock_elevated):
@ -4679,11 +4679,11 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
mock_sync.assert_called_once_with(fake_elevated, self.compute.host,
exp_uuids)
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'sync_instance_info')
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'delete_instance_info')
@mock.patch.object(nova.scheduler.client.SchedulerClient,
@mock.patch.object(nova.scheduler.client.query.SchedulerQueryClient,
'update_instance_info')
def test_scheduler_info_updates_off(self, mock_update, mock_delete,
mock_sync):

View File

@ -433,7 +433,7 @@ def setup_rt(hostname, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES,
return overhead of memory given an instance
object. Defaults to returning zero overhead.
"""
sched_client_mock = mock.MagicMock()
query_client_mock = mock.MagicMock()
report_client_mock = mock.MagicMock()
notifier_mock = mock.MagicMock()
vd = mock.MagicMock(autospec=driver.ComputeDriver)
@ -447,13 +447,13 @@ def setup_rt(hostname, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES,
vd.rebalances_nodes = False
with test.nested(
mock.patch('nova.scheduler.client.SchedulerClient',
return_value=sched_client_mock),
mock.patch('nova.scheduler.client.query.SchedulerQueryClient',
return_value=query_client_mock),
mock.patch('nova.scheduler.client.report.SchedulerReportClient',
return_value=report_client_mock),
mock.patch('nova.rpc.get_notifier', return_value=notifier_mock)):
rt = resource_tracker.ResourceTracker(hostname, vd)
return (rt, sched_client_mock, report_client_mock, vd)
return (rt, query_client_mock, report_client_mock, vd)
def compute_update_usage(resources, flavor, sign=1):

View File

@ -23,7 +23,7 @@ from nova import context as nova_context
from nova import exception
from nova.network import model as network_model
from nova import objects
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils
from nova import servicegroup
@ -74,7 +74,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
self.task = live_migrate.LiveMigrationTask(self.context,
self.instance, self.destination, self.block_migration,
self.disk_over_commit, self.migration, compute_rpcapi.ComputeAPI(),
servicegroup.API(), scheduler_client.SchedulerClient(),
servicegroup.API(), query.SchedulerQueryClient(),
report.SchedulerReportClient(), self.fake_spec)
def test_execute_with_destination(self):
@ -375,7 +375,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
'_call_livem_checks_on_host')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
return_value=[[fake_selection1]])
@mock.patch.object(objects.RequestSpec, 'reset_forced_destinations')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@ -404,7 +404,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
'_call_livem_checks_on_host')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
return_value=[[fake_selection1]])
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def test_find_destination_no_image_works(self, mock_setup, mock_select,
@ -426,7 +426,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
'_call_livem_checks_on_host')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
side_effect=[[[fake_selection1]], [[fake_selection2]]])
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def _test_find_destination_retry_hypervisor_raises(
@ -460,7 +460,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
'_call_livem_checks_on_host')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
side_effect=[[[fake_selection1]], [[fake_selection2]]])
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def test_find_destination_retry_with_invalid_livem_checks(
@ -486,7 +486,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
'_call_livem_checks_on_host')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
side_effect=[[[fake_selection1]], [[fake_selection2]]])
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def test_find_destination_retry_with_failed_migration_pre_checks(
@ -513,7 +513,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
@mock.patch.object(live_migrate.LiveMigrationTask,
'_check_compatible_with_source_hypervisor',
side_effect=exception.DestinationHypervisorTooOld())
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
return_value=[[fake_selection1]])
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def test_find_destination_retry_exceeds_max(
@ -532,7 +532,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
return_objects=True, return_alternates=False)
mock_check.assert_called_once_with('host1')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations',
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations',
side_effect=exception.NoValidHost(reason=""))
@mock.patch.object(scheduler_utils, 'setup_instance_group')
def test_find_destination_when_runs_out_of_hosts(self, mock_setup,
@ -554,7 +554,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
m_build_request_spec.return_value = {}
fake_spec = objects.RequestSpec()
m_from_primitives.return_value = fake_spec
with mock.patch.object(self.task.scheduler_client,
with mock.patch.object(self.task.query_client,
'select_destinations') as m_select_destinations:
error = messaging.RemoteError()
m_select_destinations.side_effect = error

View File

@ -18,7 +18,7 @@ from nova.conductor.tasks import migrate
from nova import context
from nova import exception
from nova import objects
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils
from nova import test
@ -65,7 +65,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
self.request_spec,
self.clean_shutdown,
compute_rpcapi.ComputeAPI(),
scheduler_client.SchedulerClient(),
query.SchedulerQueryClient(),
report.SchedulerReportClient(),
host_list=None)
@ -77,7 +77,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
@mock.patch('nova.objects.Service.get_minimum_version_multi')
@mock.patch('nova.availability_zones.get_host_availability_zone')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
def _test_execute(self, prep_resize_mock, sel_dest_mock, sig_mock, az_mock,
gmv_mock, cm_mock, sm_mock, cn_mock, rc_mock, gbf_mock,
@ -117,7 +117,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
self.heal_reqspec_is_bfv_mock.assert_called_once_with(
self.context, self.request_spec, self.instance)
sig_mock.assert_called_once_with(self.context, self.request_spec)
task.scheduler_client.select_destinations.assert_called_once_with(
task.query_client.select_destinations.assert_called_once_with(
self.context, self.request_spec, [self.instance.uuid],
return_objects=True, return_alternates=True)
selection = self.host_lists[0][0]
@ -170,7 +170,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
@mock.patch('nova.objects.Service.get_minimum_version_multi')
@mock.patch('nova.availability_zones.get_host_availability_zone')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'prep_resize')
def test_execute_rollback(self, prep_resize_mock, sel_dest_mock, sig_mock,
az_mock, gmv_mock, cm_mock, sm_mock, cn_mock,

View File

@ -46,7 +46,7 @@ from nova import objects
from nova.objects import base as obj_base
from nova.objects import block_device as block_device_obj
from nova.objects import fields
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query
from nova.scheduler import utils as scheduler_utils
from nova import test
from nova.tests import fixtures
@ -525,8 +525,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(scheduler_utils, 'build_request_spec')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(scheduler_utils, 'set_vm_state_and_notify')
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_cleanup_allocated_networks')
@mock.patch.object(conductor_manager.ComputeTaskManager,
@ -669,8 +668,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid',
side_effect=exc.InstanceMappingNotFound(uuid='fake'))
@mock.patch.object(objects.HostMapping, 'get_by_host')
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
def test_build_instances_no_instance_mapping(self, _mock_set_state,
@ -765,8 +763,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid')
@mock.patch.object(objects.HostMapping, 'get_by_host',
side_effect=exc.HostMappingNotFound(name='fake'))
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
def test_build_instances_no_host_mapping(self, _mock_set_state,
@ -811,8 +808,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(objects.Instance, 'save')
@mock.patch.object(objects.InstanceMapping, 'get_by_instance_uuid')
@mock.patch.object(objects.HostMapping, 'get_by_host')
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
def test_build_instances_update_instance_mapping(self, _mock_set_state,
@ -861,8 +857,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(objects.Instance, 'save', new=mock.MagicMock())
@mock.patch.object(objects.BuildRequest, 'get_by_instance_uuid')
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify', new=mock.MagicMock())
def test_build_instances_destroy_build_request(self, mock_select_dests,
@ -903,8 +898,7 @@ class _BaseTaskTestCase(object):
build_req.destroy.assert_called_once_with()
@mock.patch.object(objects.Instance, 'save', new=mock.MagicMock())
@mock.patch.object(scheduler_client.SchedulerClient,
'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify', new=mock.MagicMock())
def test_build_instances_reschedule_ignores_build_request(self,
@ -1342,7 +1336,7 @@ class _BaseTaskTestCase(object):
with test.nested(
mock.patch.object(self.conductor_manager.compute_rpcapi,
'rebuild_instance'),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations')
) as (rebuild_mock, select_dest_mock):
self.conductor_manager.rebuild_instance(context=self.context,
@ -1372,7 +1366,7 @@ class _BaseTaskTestCase(object):
'rebuild_instance'),
mock.patch.object(scheduler_utils, 'setup_instance_group',
return_value=False),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations',
return_value=[[fake_selection]])
) as (rebuild_mock, sig_mock, select_dest_mock):
@ -1409,7 +1403,7 @@ class _BaseTaskTestCase(object):
'rebuild_instance'),
mock.patch.object(scheduler_utils, 'setup_instance_group',
return_value=False),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations',
side_effect=exc.NoValidHost(reason='')),
mock.patch.object(scheduler_utils, 'set_vm_state_and_notify')
@ -1430,7 +1424,7 @@ class _BaseTaskTestCase(object):
@mock.patch.object(conductor_manager.compute_rpcapi.ComputeAPI,
'rebuild_instance')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(conductor_manager.scheduler_client.SchedulerClient,
@mock.patch.object(conductor_manager.query.SchedulerQueryClient,
'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@ -1488,7 +1482,7 @@ class _BaseTaskTestCase(object):
with test.nested(
mock.patch.object(self.conductor_manager.compute_rpcapi,
'rebuild_instance'),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations'),
mock.patch.object(objects.Migration, 'get_by_instance_and_status',
return_value=migration)
@ -1519,7 +1513,7 @@ class _BaseTaskTestCase(object):
'rebuild_instance'),
mock.patch.object(scheduler_utils, 'setup_instance_group',
return_value=False),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations',
return_value=[[fake_selection]]),
mock.patch.object(fake_spec, 'reset_forced_destinations'),
@ -2359,7 +2353,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
@ -2410,7 +2404,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
@ -2536,7 +2530,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
@mock.patch.object(scheduler_utils, 'setup_instance_group')
@mock.patch.object(objects.RequestSpec, 'from_components')
@mock.patch.object(utils, 'get_image_from_system_metadata')
@mock.patch.object(scheduler_client.SchedulerClient, 'select_destinations')
@mock.patch.object(query.SchedulerQueryClient, 'select_destinations')
@mock.patch.object(conductor_manager.ComputeTaskManager,
'_set_vm_state_and_notify')
@mock.patch.object(migrate.MigrationTask, 'rollback')
@ -2753,7 +2747,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
mock.patch.object(instances[1], 'save'),
mock.patch.object(objects.RequestSpec, 'from_primitives',
return_value=fake_spec),
mock.patch.object(self.conductor_manager.scheduler_client,
mock.patch.object(self.conductor_manager.query_client,
'select_destinations', return_value=destinations),
mock.patch.object(self.conductor_manager.compute_rpcapi,
'build_and_run_instance'),
@ -2847,7 +2841,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
with mock.patch.object(self.conductor, '_destroy_build_request',
new_callable=mock.NonCallableMock):
with mock.patch.object(
self.conductor.scheduler_client, 'select_destinations',
self.conductor.query_client, 'select_destinations',
side_effect=exc.NoValidHost(reason='oops')):
self.conductor.build_instances(
self.context, [instance], image, filter_props,

View File

@ -1,79 +0,0 @@
# Copyright (c) 2014 Red Hat, Inc.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import mock
import oslo_messaging as messaging
from oslo_utils.fixture import uuidsentinel as uuids
from nova import objects
from nova.scheduler import client as scheduler_client
from nova.scheduler.client import query as scheduler_query_client
from nova import test
"""Tests for Scheduler Client."""
class SchedulerClientTestCase(test.NoDBTestCase):
def setUp(self):
super(SchedulerClientTestCase, self).setUp()
self.client = scheduler_client.SchedulerClient()
def test_constructor(self):
self.assertIsNotNone(self.client.queryclient)
@mock.patch.object(scheduler_query_client.SchedulerQueryClient,
'select_destinations')
def test_select_destinations(self, mock_select_destinations):
fake_spec = objects.RequestSpec()
fake_spec.instance_uuid = uuids.instance
self.client.select_destinations('ctxt', fake_spec,
[fake_spec.instance_uuid])
mock_select_destinations.assert_called_once_with('ctxt', fake_spec,
[fake_spec.instance_uuid], False, False)
@mock.patch.object(scheduler_query_client.SchedulerQueryClient,
'select_destinations',
side_effect=messaging.MessagingTimeout())
def test_select_destinations_timeout(self, mock_select_destinations):
# check if the scheduler service times out properly
fake_spec = objects.RequestSpec()
fake_spec.instance_uuid = uuids.instance
fake_args = ['ctxt', fake_spec, [fake_spec.instance_uuid], False,
False]
self.assertRaises(messaging.MessagingTimeout,
self.client.select_destinations, *fake_args)
mock_select_destinations.assert_called_once_with(*fake_args)
@mock.patch.object(scheduler_query_client.SchedulerQueryClient,
'update_aggregates')
def test_update_aggregates(self, mock_update_aggs):
aggregates = [objects.Aggregate(id=1)]
self.client.update_aggregates(
context='context',
aggregates=aggregates)
mock_update_aggs.assert_called_once_with(
'context', aggregates)
@mock.patch.object(scheduler_query_client.SchedulerQueryClient,
'delete_aggregate')
def test_delete_aggregate(self, mock_delete_agg):
aggregate = objects.Aggregate(id=1)
self.client.delete_aggregate(
context='context',
aggregate=aggregate)
mock_delete_agg.assert_called_once_with(
'context', aggregate)

View File

@ -22,7 +22,6 @@ from oslo_utils.fixture import uuidsentinel as uuids
from nova import exception
from nova import objects
from nova.scheduler import client
from nova.scheduler import filter_scheduler
from nova.scheduler import host_manager
from nova.scheduler import utils as scheduler_utils
@ -56,12 +55,12 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
driver_cls = filter_scheduler.FilterScheduler
@mock.patch('nova.scheduler.client.SchedulerClient')
def setUp(self, mock_client):
sched_client = mock.Mock(spec=client.SchedulerClient)
mock_client.return_value = sched_client
with mock.patch('nova.scheduler.client.report.SchedulerReportClient',
autospec=True):
def setUp(self):
with mock.patch(
'nova.scheduler.client.report.SchedulerReportClient',
autospec=True), mock.patch(
'nova.scheduler.client.query.SchedulerQueryClient',
autospec=True):
super(FilterSchedulerTestCase, self).setUp()
@mock.patch('nova.scheduler.utils.claim_resources')