Rip the report client out of SchedulerClient

A step toward getting rid of the SchedulerClient intermediary, this
patch removes the reportclient member from SchedulerClient, instead
instantiating SchedulerReportClient directly wherever it's needed.

Change-Id: I14d1a648843c6311a962aaf99a47bb1bebf7f5ea
This commit is contained in:
Eric Fried 2018-11-09 16:47:39 -06:00
parent 570ad36992
commit 86e94a11f7
17 changed files with 92 additions and 90 deletions

View File

@ -507,7 +507,7 @@ class ComputeManager(manager.Manager):
openstack_driver.is_neutron_security_groups()) openstack_driver.is_neutron_security_groups())
self.cells_rpcapi = cells_rpcapi.CellsAPI() self.cells_rpcapi = cells_rpcapi.CellsAPI()
self.scheduler_client = scheduler_client.SchedulerClient() self.scheduler_client = scheduler_client.SchedulerClient()
self.reportclient = self.scheduler_client.reportclient self._reportclient = None
self._resource_tracker = None self._resource_tracker = None
self.instance_events = InstanceEvents() self.instance_events = InstanceEvents()
self._sync_power_pool = eventlet.GreenPool( self._sync_power_pool = eventlet.GreenPool(
@ -551,6 +551,12 @@ class ComputeManager(manager.Manager):
self._resource_tracker = rt self._resource_tracker = rt
return self._resource_tracker return self._resource_tracker
@property
def reportclient(self):
if not self._reportclient:
self._reportclient = self._get_resource_tracker().reportclient
return self._reportclient
def _update_resource_tracker(self, context, instance): def _update_resource_tracker(self, context, instance):
"""Let the resource tracker know that an instance has changed state.""" """Let the resource tracker know that an instance has changed state."""
@ -763,8 +769,8 @@ class ComputeManager(manager.Manager):
def _complete_deletion(self, context, instance): def _complete_deletion(self, context, instance):
self._update_resource_tracker(context, instance) self._update_resource_tracker(context, instance)
rt = self._get_resource_tracker() self.reportclient.delete_allocation_for_instance(context,
rt.reportclient.delete_allocation_for_instance(context, instance.uuid) instance.uuid)
self._clean_instance_console_tokens(context, instance) self._clean_instance_console_tokens(context, instance)
self._delete_scheduler_instance_info(context, instance.uuid) self._delete_scheduler_instance_info(context, instance.uuid)
@ -1847,8 +1853,8 @@ class ComputeManager(manager.Manager):
# call this for a reschedule, as the allocations will # call this for a reschedule, as the allocations will
# have already been removed in # have already been removed in
# self._do_build_and_run_instance(). # self._do_build_and_run_instance().
self._delete_allocation_for_instance(context, self.reportclient.delete_allocation_for_instance(
instance.uuid) context, instance.uuid)
if result in (build_results.FAILED, if result in (build_results.FAILED,
build_results.RESCHEDULED): build_results.RESCHEDULED):
@ -1865,10 +1871,6 @@ class ComputeManager(manager.Manager):
requested_networks, security_groups, requested_networks, security_groups,
block_device_mapping, node, limits, host_list) block_device_mapping, node, limits, host_list)
def _delete_allocation_for_instance(self, context, instance_uuid):
rt = self._get_resource_tracker()
rt.reportclient.delete_allocation_for_instance(context, instance_uuid)
def _check_device_tagging(self, requested_networks, block_device_mapping): def _check_device_tagging(self, requested_networks, block_device_mapping):
tagging_requested = False tagging_requested = False
if requested_networks: if requested_networks:
@ -1996,7 +1998,8 @@ class ComputeManager(manager.Manager):
# to unclaim those resources before casting to the conductor, so # to unclaim those resources before casting to the conductor, so
# that if there are alternate hosts available for a retry, it can # that if there are alternate hosts available for a retry, it can
# claim resources on that new host for the instance. # claim resources on that new host for the instance.
self._delete_allocation_for_instance(context, instance.uuid) self.reportclient.delete_allocation_for_instance(context,
instance.uuid)
self.compute_task_api.build_instances(context, [instance], self.compute_task_api.build_instances(context, [instance],
image, filter_properties, admin_password, image, filter_properties, admin_password,
@ -5053,8 +5056,8 @@ class ComputeManager(manager.Manager):
# the instance claim failed with ComputeResourcesUnavailable # the instance claim failed with ComputeResourcesUnavailable
# or if we did claim but the spawn failed, because aborting the # or if we did claim but the spawn failed, because aborting the
# instance claim will not remove the allocations. # instance claim will not remove the allocations.
rt.reportclient.delete_allocation_for_instance(context, self.reportclient.delete_allocation_for_instance(context,
instance.uuid) instance.uuid)
# FIXME: Umm, shouldn't we be rolling back port bindings too? # FIXME: Umm, shouldn't we be rolling back port bindings too?
self._terminate_volume_connections(context, instance, bdms) self._terminate_volume_connections(context, instance, bdms)
# The reverts_task_state decorator on unshelve_instance will # The reverts_task_state decorator on unshelve_instance will
@ -7766,9 +7769,8 @@ class ComputeManager(manager.Manager):
rt.remove_node(cn.hypervisor_hostname) rt.remove_node(cn.hypervisor_hostname)
# Delete the corresponding resource provider in placement, # Delete the corresponding resource provider in placement,
# along with any associated allocations and inventory. # along with any associated allocations and inventory.
# TODO(cdent): Move use of reportclient into resource tracker. self.reportclient.delete_resource_provider(context, cn,
self.scheduler_client.reportclient.delete_resource_provider( cascade=True)
context, cn, cascade=True)
for nodename in nodenames: for nodename in nodenames:
self._update_available_resource_for_node(context, nodename, self._update_available_resource_for_node(context, nodename,

View File

@ -43,6 +43,7 @@ from nova.pci import request as pci_request
from nova import rc_fields as fields from nova import rc_fields as fields
from nova import rpc from nova import rpc
from nova.scheduler import client as scheduler_client from nova.scheduler import client as scheduler_client
from nova.scheduler.client import report
from nova import utils from nova import utils
from nova.virt import hardware from nova.virt import hardware
@ -147,7 +148,7 @@ class ResourceTracker(object):
self.monitors = monitor_handler.monitors self.monitors = monitor_handler.monitors
self.old_resources = collections.defaultdict(objects.ComputeNode) self.old_resources = collections.defaultdict(objects.ComputeNode)
self.scheduler_client = scheduler_client.SchedulerClient() self.scheduler_client = scheduler_client.SchedulerClient()
self.reportclient = self.scheduler_client.reportclient self.reportclient = report.SchedulerReportClient()
self.ram_allocation_ratio = CONF.ram_allocation_ratio self.ram_allocation_ratio = CONF.ram_allocation_ratio
self.cpu_allocation_ratio = CONF.cpu_allocation_ratio self.cpu_allocation_ratio = CONF.cpu_allocation_ratio
self.disk_allocation_ratio = CONF.disk_allocation_ratio self.disk_allocation_ratio = CONF.disk_allocation_ratio
@ -928,8 +929,7 @@ class ResourceTracker(object):
# Retrieve the provider tree associated with this compute node. If # Retrieve the provider tree associated with this compute node. If
# it doesn't exist yet, this will create it with a (single, root) # it doesn't exist yet, this will create it with a (single, root)
# provider corresponding to the compute node. # provider corresponding to the compute node.
reportclient = self.scheduler_client.reportclient prov_tree = self.reportclient.get_provider_tree_and_ensure_root(
prov_tree = reportclient.get_provider_tree_and_ensure_root(
context, compute_node.uuid, name=compute_node.hypervisor_hostname) context, compute_node.uuid, name=compute_node.hypervisor_hostname)
# Let the virt driver rearrange the provider tree and set/update # Let the virt driver rearrange the provider tree and set/update
# the inventory, traits, and aggregates throughout. # the inventory, traits, and aggregates throughout.
@ -945,7 +945,7 @@ class ResourceTracker(object):
LOG.info("Performing resource provider inventory and " LOG.info("Performing resource provider inventory and "
"allocation data migration during compute service " "allocation data migration during compute service "
"startup or fast-forward upgrade.") "startup or fast-forward upgrade.")
allocs = reportclient.get_allocations_for_provider_tree( allocs = self.reportclient.get_allocations_for_provider_tree(
context, nodename) context, nodename)
self.driver.update_provider_tree(prov_tree, nodename, self.driver.update_provider_tree(prov_tree, nodename,
allocations=allocs) allocations=allocs)
@ -966,8 +966,8 @@ class ResourceTracker(object):
# Flush any changes. If we processed ReshapeNeeded above, allocs is not # Flush any changes. If we processed ReshapeNeeded above, allocs is not
# None, and this will hit placement's POST /reshaper route. # None, and this will hit placement's POST /reshaper route.
reportclient.update_from_provider_tree(context, prov_tree, self.reportclient.update_from_provider_tree(context, prov_tree,
allocations=allocs) allocations=allocs)
@retrying.retry(stop_max_attempt_number=4, @retrying.retry(stop_max_attempt_number=4,
retry_on_exception=lambda e: isinstance( retry_on_exception=lambda e: isinstance(

View File

@ -51,6 +51,7 @@ from nova.objects import fields
from nova import profiler from nova import profiler
from nova import rpc from nova import rpc
from nova.scheduler import client as scheduler_client from nova.scheduler import client as scheduler_client
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils from nova.scheduler import utils as scheduler_utils
from nova import servicegroup from nova import servicegroup
from nova import utils from nova import utils
@ -233,7 +234,7 @@ class ComputeTaskManager(base.Base):
self.network_api = network.API() self.network_api = network.API()
self.servicegroup_api = servicegroup.API() self.servicegroup_api = servicegroup.API()
self.scheduler_client = scheduler_client.SchedulerClient() self.scheduler_client = scheduler_client.SchedulerClient()
self.report_client = self.scheduler_client.reportclient self.report_client = report.SchedulerReportClient()
self.notifier = rpc.get_notifier('compute', CONF.host) self.notifier = rpc.get_notifier('compute', CONF.host)
# Help us to record host in EventReporter # Help us to record host in EventReporter
self.host = CONF.host self.host = CONF.host
@ -465,15 +466,16 @@ class ComputeTaskManager(base.Base):
self.compute_rpcapi, self.compute_rpcapi,
self.servicegroup_api, self.servicegroup_api,
self.scheduler_client, self.scheduler_client,
self.report_client,
request_spec) request_spec)
def _build_cold_migrate_task(self, context, instance, flavor, request_spec, def _build_cold_migrate_task(self, context, instance, flavor, request_spec,
clean_shutdown, host_list): clean_shutdown, host_list):
return migrate.MigrationTask(context, instance, flavor, return migrate.MigrationTask(context, instance, flavor,
request_spec, request_spec, clean_shutdown,
clean_shutdown,
self.compute_rpcapi, self.compute_rpcapi,
self.scheduler_client, host_list) self.scheduler_client, self.report_client,
host_list)
def _destroy_build_request(self, context, instance): def _destroy_build_request(self, context, instance):
# The BuildRequest needs to be stored until the instance is mapped to # The BuildRequest needs to be stored until the instance is mapped to
@ -1290,8 +1292,8 @@ class ComputeTaskManager(base.Base):
'was already deleted.', instance=instance) 'was already deleted.', instance=instance)
# This is a placeholder in case the quota recheck fails. # This is a placeholder in case the quota recheck fails.
instances.append(None) instances.append(None)
rc = self.scheduler_client.reportclient self.report_client.delete_allocation_for_instance(
rc.delete_allocation_for_instance(context, instance.uuid) context, instance.uuid)
continue continue
else: else:
if host.service_host not in host_az: if host.service_host not in host_az:

View File

@ -46,7 +46,8 @@ def supports_extended_port_binding(context, host):
class LiveMigrationTask(base.TaskBase): class LiveMigrationTask(base.TaskBase):
def __init__(self, context, instance, destination, def __init__(self, context, instance, destination,
block_migration, disk_over_commit, migration, compute_rpcapi, block_migration, disk_over_commit, migration, compute_rpcapi,
servicegroup_api, scheduler_client, request_spec=None): servicegroup_api, scheduler_client, report_client,
request_spec=None):
super(LiveMigrationTask, self).__init__(context, instance) super(LiveMigrationTask, self).__init__(context, instance)
self.destination = destination self.destination = destination
self.block_migration = block_migration self.block_migration = block_migration
@ -58,6 +59,7 @@ class LiveMigrationTask(base.TaskBase):
self.compute_rpcapi = compute_rpcapi self.compute_rpcapi = compute_rpcapi
self.servicegroup_api = servicegroup_api self.servicegroup_api = servicegroup_api
self.scheduler_client = scheduler_client self.scheduler_client = scheduler_client
self.report_client = report_client
self.request_spec = request_spec self.request_spec = request_spec
self._source_cn = None self._source_cn = None
self._held_allocations = None self._held_allocations = None
@ -107,7 +109,7 @@ class LiveMigrationTask(base.TaskBase):
# generation conflict and this call raise a AllocationUpdateFailed # generation conflict and this call raise a AllocationUpdateFailed
# exception. We let that propagate here to abort the migration. # exception. We let that propagate here to abort the migration.
scheduler_utils.claim_resources_on_destination( scheduler_utils.claim_resources_on_destination(
self.context, self.scheduler_client.reportclient, self.context, self.report_client,
self.instance, source_node, dest_node, self.instance, source_node, dest_node,
source_allocations=self._held_allocations, source_allocations=self._held_allocations,
consumer_generation=None) consumer_generation=None)
@ -431,8 +433,7 @@ class LiveMigrationTask(base.TaskBase):
# Note that this does not remove allocations against any other node # Note that this does not remove allocations against any other node
# or shared resource provider, it's just undoing what the scheduler # or shared resource provider, it's just undoing what the scheduler
# allocated for the given (destination) node. # allocated for the given (destination) node.
self.scheduler_client.reportclient.\ self.report_client.remove_provider_tree_from_instance_allocation(
remove_provider_tree_from_instance_allocation(
self.context, self.instance.uuid, compute_node.uuid) self.context, self.instance.uuid, compute_node.uuid)
def _check_not_over_max_retries(self, attempted_hosts): def _check_not_over_max_retries(self, attempted_hosts):

View File

@ -19,7 +19,7 @@ from nova.conductor.tasks import base
from nova import exception from nova import exception
from nova.i18n import _ from nova.i18n import _
from nova import objects from nova import objects
from nova.scheduler import client as scheduler_client from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils from nova.scheduler import utils as scheduler_utils
LOG = logging.getLogger(__name__) LOG = logging.getLogger(__name__)
@ -53,8 +53,7 @@ def replace_allocation_with_migration(context, instance, migration):
# and do any rollback required # and do any rollback required
raise raise
schedclient = scheduler_client.SchedulerClient() reportclient = report.SchedulerReportClient()
reportclient = schedclient.reportclient
orig_alloc = reportclient.get_allocs_for_consumer( orig_alloc = reportclient.get_allocs_for_consumer(
context, instance.uuid)['allocations'] context, instance.uuid)['allocations']
@ -92,8 +91,7 @@ def replace_allocation_with_migration(context, instance, migration):
def revert_allocation_for_migration(context, source_cn, instance, migration): def revert_allocation_for_migration(context, source_cn, instance, migration):
"""Revert an allocation made for a migration back to the instance.""" """Revert an allocation made for a migration back to the instance."""
schedclient = scheduler_client.SchedulerClient() reportclient = report.SchedulerReportClient()
reportclient = schedclient.reportclient
# FIXME(danms): This method is flawed in that it asssumes allocations # FIXME(danms): This method is flawed in that it asssumes allocations
# against only one provider. So, this may overwite allocations against # against only one provider. So, this may overwite allocations against
@ -114,7 +112,7 @@ def revert_allocation_for_migration(context, source_cn, instance, migration):
class MigrationTask(base.TaskBase): class MigrationTask(base.TaskBase):
def __init__(self, context, instance, flavor, def __init__(self, context, instance, flavor,
request_spec, clean_shutdown, compute_rpcapi, request_spec, clean_shutdown, compute_rpcapi,
scheduler_client, host_list): scheduler_client, report_client, host_list):
super(MigrationTask, self).__init__(context, instance) super(MigrationTask, self).__init__(context, instance)
self.clean_shutdown = clean_shutdown self.clean_shutdown = clean_shutdown
self.request_spec = request_spec self.request_spec = request_spec
@ -122,7 +120,7 @@ class MigrationTask(base.TaskBase):
self.compute_rpcapi = compute_rpcapi self.compute_rpcapi = compute_rpcapi
self.scheduler_client = scheduler_client self.scheduler_client = scheduler_client
self.reportclient = scheduler_client.reportclient self.reportclient = report_client
self.host_list = host_list self.host_list = host_list
# Persist things from the happy path so we don't have to look # Persist things from the happy path so we don't have to look

View File

@ -14,7 +14,6 @@
# under the License. # under the License.
from nova.scheduler.client import query from nova.scheduler.client import query
from nova.scheduler.client import report
class SchedulerClient(object): class SchedulerClient(object):
@ -22,7 +21,6 @@ class SchedulerClient(object):
def __init__(self): def __init__(self):
self.queryclient = query.SchedulerQueryClient() self.queryclient = query.SchedulerQueryClient()
self.reportclient = report.SchedulerReportClient()
def select_destinations(self, context, spec_obj, instance_uuids, def select_destinations(self, context, spec_obj, instance_uuids,
return_objects=False, return_alternates=False): return_objects=False, return_alternates=False):

View File

@ -31,7 +31,7 @@ from nova.i18n import _
from nova import objects from nova import objects
from nova.objects import fields as fields_obj from nova.objects import fields as fields_obj
from nova import rpc from nova import rpc
from nova.scheduler import client from nova.scheduler.client import report
from nova.scheduler import driver from nova.scheduler import driver
from nova.scheduler import utils from nova.scheduler import utils
@ -44,8 +44,7 @@ class FilterScheduler(driver.Scheduler):
def __init__(self, *args, **kwargs): def __init__(self, *args, **kwargs):
super(FilterScheduler, self).__init__(*args, **kwargs) super(FilterScheduler, self).__init__(*args, **kwargs)
self.notifier = rpc.get_notifier('scheduler') self.notifier = rpc.get_notifier('scheduler')
scheduler_client = client.SchedulerClient() self.placement_client = report.SchedulerReportClient()
self.placement_client = scheduler_client.reportclient
def select_destinations(self, context, spec_obj, instance_uuids, def select_destinations(self, context, spec_obj, instance_uuids,
alloc_reqs_by_rp_uuid, provider_summaries, alloc_reqs_by_rp_uuid, provider_summaries,

View File

@ -33,7 +33,7 @@ from nova import manager
from nova import objects from nova import objects
from nova.objects import host_mapping as host_mapping_obj from nova.objects import host_mapping as host_mapping_obj
from nova import quota from nova import quota
from nova.scheduler import client as scheduler_client from nova.scheduler.client import report
from nova.scheduler import request_filter from nova.scheduler import request_filter
from nova.scheduler import utils from nova.scheduler import utils
@ -53,8 +53,7 @@ class SchedulerManager(manager.Manager):
_sentinel = object() _sentinel = object()
def __init__(self, scheduler_driver=None, *args, **kwargs): def __init__(self, scheduler_driver=None, *args, **kwargs):
client = scheduler_client.SchedulerClient() self.placement_client = report.SchedulerReportClient()
self.placement_client = client.reportclient
if not scheduler_driver: if not scheduler_driver:
scheduler_driver = CONF.scheduler.driver scheduler_driver = CONF.scheduler.driver
self.driver = driver.DriverManager( self.driver = driver.DriverManager(

View File

@ -136,7 +136,6 @@ class IronicResourceTrackerTest(test_base.SchedulerReportClientTestBase):
interceptor. interceptor.
""" """
self.report_client = client self.report_client = client
self.rt.scheduler_client.reportclient = client
self.rt.reportclient = client self.rt.reportclient = client
def setUp(self): def setUp(self):

View File

@ -6229,7 +6229,7 @@ class ComputeTestCase(BaseTestCase,
migration = objects.Migration(uuid=uuids.migration) migration = objects.Migration(uuid=uuids.migration)
@mock.patch.object(self.compute.network_api, 'setup_networks_on_host') @mock.patch.object(self.compute.network_api, 'setup_networks_on_host')
@mock.patch.object(self.compute, 'reportclient') @mock.patch.object(self.compute, '_reportclient')
def do_it(mock_client, mock_setup): def do_it(mock_client, mock_setup):
mock_client.get_allocations_for_consumer.return_value = { mock_client.get_allocations_for_consumer.return_value = {
mock.sentinel.source: { mock.sentinel.source: {

View File

@ -334,14 +334,12 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
log_mock.exception.assert_called_once() log_mock.exception.assert_called_once()
@mock.patch.object(manager.ComputeManager, '_get_resource_tracker') @mock.patch.object(manager.ComputeManager, '_get_resource_tracker')
@mock.patch('nova.scheduler.client.report.SchedulerReportClient.'
'delete_resource_provider')
@mock.patch.object(manager.ComputeManager, @mock.patch.object(manager.ComputeManager,
'_update_available_resource_for_node') '_update_available_resource_for_node')
@mock.patch.object(fake_driver.FakeDriver, 'get_available_nodes') @mock.patch.object(fake_driver.FakeDriver, 'get_available_nodes')
@mock.patch.object(manager.ComputeManager, '_get_compute_nodes_in_db') @mock.patch.object(manager.ComputeManager, '_get_compute_nodes_in_db')
def test_update_available_resource(self, get_db_nodes, get_avail_nodes, def test_update_available_resource(self, get_db_nodes, get_avail_nodes,
update_mock, del_rp_mock, mock_get_rt): update_mock, mock_get_rt):
db_nodes = [self._make_compute_node('node%s' % i, i) db_nodes = [self._make_compute_node('node%s' % i, i)
for i in range(1, 5)] for i in range(1, 5)]
avail_nodes = set(['node2', 'node3', 'node4', 'node5']) avail_nodes = set(['node2', 'node3', 'node4', 'node5'])
@ -357,12 +355,13 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase):
for node in avail_nodes_l] for node in avail_nodes_l]
) )
rc_mock = mock_get_rt.return_value.reportclient
# First node in set should have been removed from DB # First node in set should have been removed from DB
for db_node in db_nodes: for db_node in db_nodes:
if db_node.hypervisor_hostname == 'node1': if db_node.hypervisor_hostname == 'node1':
db_node.destroy.assert_called_once_with() db_node.destroy.assert_called_once_with()
del_rp_mock.assert_called_once_with(self.context, db_node, rc_mock.delete_resource_provider.assert_called_once_with(
cascade=True) self.context, db_node, cascade=True)
mock_get_rt.return_value.remove_node.assert_called_once_with( mock_get_rt.return_value.remove_node.assert_called_once_with(
'node1') 'node1')
else: else:
@ -5010,9 +5009,8 @@ class ComputeManagerBuildInstanceTestCase(test.NoDBTestCase):
}, },
}] }]
self.mock_get_allocs = self.useFixture( self.mock_get_allocs = self.useFixture(
fixtures.fixtures.MockPatch( fixtures.fixtures.MockPatchObject(
'nova.scheduler.client.report.SchedulerReportClient.' fake_rt.reportclient, 'get_allocations_for_consumer')).mock
'get_allocations_for_consumer')).mock
self.mock_get_allocs.return_value = self.allocations self.mock_get_allocs.return_value = self.allocations
def _do_build_instance_update(self, mock_save, reschedule_update=False): def _do_build_instance_update(self, mock_save, reschedule_update=False):
@ -6866,6 +6864,8 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
def _get_instance_nw_info(context, instance): def _get_instance_nw_info(context, instance):
return self.nw_info return self.nw_info
reportclient = self.compute.reportclient
@mock.patch.object(self.compute, '_get_resource_tracker') @mock.patch.object(self.compute, '_get_resource_tracker')
@mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename') @mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename')
@mock.patch.object(self.compute.driver, 'finish_revert_migration') @mock.patch.object(self.compute.driver, 'finish_revert_migration')
@ -6896,6 +6896,8 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
mock_get_cn, mock_get_cn,
get_resource_tracker): get_resource_tracker):
# Restore the report client
get_resource_tracker.return_value.reportclient = reportclient
fault_create.return_value = ( fault_create.return_value = (
test_instance_fault.fake_faults['fake-uuid'][0]) test_instance_fault.fake_faults['fake-uuid'][0])
self.instance.migration_context = objects.MigrationContext() self.instance.migration_context = objects.MigrationContext()
@ -7051,7 +7053,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
do_confirm_resize() do_confirm_resize()
def test_delete_allocation_after_move_confirm_by_migration(self): def test_delete_allocation_after_move_confirm_by_migration(self):
with mock.patch.object(self.compute, 'reportclient') as mock_report: with mock.patch.object(self.compute, '_reportclient') as mock_report:
mock_report.delete_allocation_for_instance.return_value = True mock_report.delete_allocation_for_instance.return_value = True
self.compute._delete_allocation_after_move(self.context, self.compute._delete_allocation_after_move(self.context,
self.instance, self.instance,
@ -7062,7 +7064,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
def test_revert_allocation(self): def test_revert_allocation(self):
"""New-style migration-based allocation revert.""" """New-style migration-based allocation revert."""
@mock.patch.object(self.compute, 'reportclient') @mock.patch.object(self.compute, '_reportclient')
def doit(mock_report): def doit(mock_report):
cu = uuids.node cu = uuids.node
a = {cu: {'resources': {'DISK_GB': 1}}} a = {cu: {'resources': {'DISK_GB': 1}}}
@ -7081,7 +7083,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
def test_revert_allocation_old_style(self): def test_revert_allocation_old_style(self):
"""Test that we don't delete allocs for migration if none found.""" """Test that we don't delete allocs for migration if none found."""
@mock.patch.object(self.compute, 'reportclient') @mock.patch.object(self.compute, '_reportclient')
def doit(mock_report): def doit(mock_report):
mock_report.get_allocations_for_consumer.return_value = {} mock_report.get_allocations_for_consumer.return_value = {}
self.migration.uuid = uuids.migration self.migration.uuid = uuids.migration
@ -7102,7 +7104,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
we can do. we can do.
""" """
@mock.patch.object(self.compute, 'reportclient') @mock.patch.object(self.compute, '_reportclient')
def doit(mock_report): def doit(mock_report):
a = { a = {
uuids.node: {'resources': {'DISK_GB': 1}}, uuids.node: {'resources': {'DISK_GB': 1}},
@ -7897,8 +7899,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
is_shared_instance_path=False, is_shared_instance_path=False,
is_shared_block_storage=False) is_shared_block_storage=False)
with test.nested( with test.nested(
mock.patch.object(self.compute.scheduler_client, mock.patch.object(self.compute, '_reportclient'),
'reportclient'),
mock.patch.object(self.compute, mock.patch.object(self.compute,
'_delete_allocation_after_move'), '_delete_allocation_after_move'),
) as ( ) as (
@ -8369,10 +8370,14 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase):
cn = objects.ComputeNode(uuid=uuids.compute) cn = objects.ComputeNode(uuid=uuids.compute)
mock_cn.return_value = cn mock_cn.return_value = cn
reportclient = self.compute.reportclient
@mock.patch.object(self.compute, '_reschedule') @mock.patch.object(self.compute, '_reschedule')
@mock.patch.object(self.compute, '_prep_resize') @mock.patch.object(self.compute, '_prep_resize')
@mock.patch.object(self.compute, '_get_resource_tracker') @mock.patch.object(self.compute, '_get_resource_tracker')
def doit(mock_grt, mock_pr, mock_r): def doit(mock_grt, mock_pr, mock_r):
# Restore the report client
mock_grt.return_value.reportclient = reportclient
mock_r.return_value = False mock_r.return_value = False
mock_pr.side_effect = test.TestingException mock_pr.side_effect = test.TestingException

View File

@ -434,6 +434,7 @@ def setup_rt(hostname, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES,
object. Defaults to returning zero overhead. object. Defaults to returning zero overhead.
""" """
sched_client_mock = mock.MagicMock() sched_client_mock = mock.MagicMock()
report_client_mock = mock.MagicMock()
notifier_mock = mock.MagicMock() notifier_mock = mock.MagicMock()
vd = mock.MagicMock(autospec=driver.ComputeDriver) vd = mock.MagicMock(autospec=driver.ComputeDriver)
# Make sure we don't change any global fixtures during tests # Make sure we don't change any global fixtures during tests
@ -448,9 +449,11 @@ def setup_rt(hostname, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES,
with test.nested( with test.nested(
mock.patch('nova.scheduler.client.SchedulerClient', mock.patch('nova.scheduler.client.SchedulerClient',
return_value=sched_client_mock), return_value=sched_client_mock),
mock.patch('nova.scheduler.client.report.SchedulerReportClient',
return_value=report_client_mock),
mock.patch('nova.rpc.get_notifier', return_value=notifier_mock)): mock.patch('nova.rpc.get_notifier', return_value=notifier_mock)):
rt = resource_tracker.ResourceTracker(hostname, vd) rt = resource_tracker.ResourceTracker(hostname, vd)
return (rt, sched_client_mock, vd) return (rt, sched_client_mock, report_client_mock, vd)
def compute_update_usage(resources, flavor, sign=1): def compute_update_usage(resources, flavor, sign=1):
@ -474,7 +477,7 @@ class BaseTestCase(test.NoDBTestCase):
def _setup_rt(self, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES, def _setup_rt(self, virt_resources=_VIRT_DRIVER_AVAIL_RESOURCES,
estimate_overhead=overhead_zero): estimate_overhead=overhead_zero):
(self.rt, self.sched_client_mock, (self.rt, self.sched_client_mock, self.report_client_mock,
self.driver_mock) = setup_rt( self.driver_mock) = setup_rt(
_HOSTNAME, virt_resources, estimate_overhead) _HOSTNAME, virt_resources, estimate_overhead)
@ -3005,13 +3008,7 @@ class TestUpdateUsageFromInstance(BaseTestCase):
# Scheduled instances should not have their allocations removed # Scheduled instances should not have their allocations removed
rc.delete_allocation_for_instance.assert_not_called() rc.delete_allocation_for_instance.assert_not_called()
@mock.patch('nova.scheduler.client.report.SchedulerReportClient.' def test_remove_deleted_instances_allocations_move_ops(self):
'get_allocations_for_resource_provider')
@mock.patch('nova.scheduler.client.report.SchedulerReportClient.'
'delete_allocation_for_instance')
@mock.patch('nova.objects.Instance.get_by_uuid')
def test_remove_deleted_instances_allocations_move_ops(self, mock_get,
mock_delete_allocs, mock_get_allocs):
"""Test that we do NOT delete allocations for instances that are """Test that we do NOT delete allocations for instances that are
currently undergoing move operations. currently undergoing move operations.
""" """
@ -3022,14 +3019,16 @@ class TestUpdateUsageFromInstance(BaseTestCase):
# Instances in resizing/move will be ACTIVE or STOPPED # Instances in resizing/move will be ACTIVE or STOPPED
instance.vm_state = vm_states.ACTIVE instance.vm_state = vm_states.ACTIVE
# Mock out the allocation call # Mock out the allocation call
allocs = {uuids.inst0: mock.sentinel.moving_instance} rpt_clt = self.report_client_mock
mock_get_allocs.return_value = allocs allocs = report.ProviderAllocInfo(
mock_get.return_value = instance allocations={uuids.inst0: mock.sentinel.moving_instance})
rpt_clt.get_allocations_for_resource_provider.return_value = allocs
cn = self.rt.compute_nodes[_NODENAME] cn = self.rt.compute_nodes[_NODENAME]
ctx = mock.MagicMock() ctx = mock.MagicMock()
self.rt._remove_deleted_instances_allocations(ctx, cn, [], mock.ANY) self.rt._remove_deleted_instances_allocations(
mock_delete_allocs.assert_not_called() ctx, cn, [], {uuids.inst0: instance})
rpt_clt.delete_allocation_for_instance.assert_not_called()
def test_remove_deleted_instances_allocations_known_instance(self): def test_remove_deleted_instances_allocations_known_instance(self):
"""Tests the case that actively tracked instances for the """Tests the case that actively tracked instances for the

View File

@ -24,6 +24,7 @@ from nova import exception
from nova.network import model as network_model from nova.network import model as network_model
from nova import objects from nova import objects
from nova.scheduler import client as scheduler_client from nova.scheduler import client as scheduler_client
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils from nova.scheduler import utils as scheduler_utils
from nova import servicegroup from nova import servicegroup
from nova import test from nova import test
@ -74,7 +75,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
self.instance, self.destination, self.block_migration, self.instance, self.destination, self.block_migration,
self.disk_over_commit, self.migration, compute_rpcapi.ComputeAPI(), self.disk_over_commit, self.migration, compute_rpcapi.ComputeAPI(),
servicegroup.API(), scheduler_client.SchedulerClient(), servicegroup.API(), scheduler_client.SchedulerClient(),
self.fake_spec) report.SchedulerReportClient(), self.fake_spec)
def test_execute_with_destination(self): def test_execute_with_destination(self):
dest_node = objects.ComputeNode(hypervisor_hostname='dest_node') dest_node = objects.ComputeNode(hypervisor_hostname='dest_node')
@ -99,7 +100,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
mock_check_dest.assert_called_once_with() mock_check_dest.assert_called_once_with()
allocs = mock.sentinel.allocs allocs = mock.sentinel.allocs
mock_claim.assert_called_once_with( mock_claim.assert_called_once_with(
self.context, self.task.scheduler_client.reportclient, self.context, self.task.report_client,
self.instance, mock.sentinel.source_node, dest_node, self.instance, mock.sentinel.source_node, dest_node,
source_allocations=allocs, consumer_generation=None) source_allocations=allocs, consumer_generation=None)
mock_mig.assert_called_once_with( mock_mig.assert_called_once_with(

View File

@ -19,6 +19,7 @@ from nova import context
from nova import exception from nova import exception
from nova import objects from nova import objects
from nova.scheduler import client as scheduler_client from nova.scheduler import client as scheduler_client
from nova.scheduler.client import report
from nova.scheduler import utils as scheduler_utils from nova.scheduler import utils as scheduler_utils
from nova import test from nova import test
from nova.tests.unit.conductor.test_conductor import FakeContext from nova.tests.unit.conductor.test_conductor import FakeContext
@ -65,6 +66,7 @@ class MigrationTaskTestCase(test.NoDBTestCase):
self.clean_shutdown, self.clean_shutdown,
compute_rpcapi.ComputeAPI(), compute_rpcapi.ComputeAPI(),
scheduler_client.SchedulerClient(), scheduler_client.SchedulerClient(),
report.SchedulerReportClient(),
host_list=None) host_list=None)
@mock.patch.object(objects.MigrationList, 'get_by_filters') @mock.patch.object(objects.MigrationList, 'get_by_filters')

View File

@ -2981,7 +2981,7 @@ class ConductorTaskTestCase(_BaseTaskTestCase, test_compute.BaseTestCase):
get_dest_node.assert_called_once_with( get_dest_node.assert_called_once_with(
self.ctxt, 'dest-host', use_slave=True) self.ctxt, 'dest-host', use_slave=True)
claim.assert_called_once_with( claim.assert_called_once_with(
self.ctxt, self.conductor.scheduler_client.reportclient, instance, self.ctxt, self.conductor.report_client, instance,
get_source_node.return_value, get_dest_node.return_value) get_source_node.return_value, get_dest_node.return_value)
notify.assert_called_once_with( notify.assert_called_once_with(
self.ctxt, instance.uuid, 'rebuild_server', self.ctxt, instance.uuid, 'rebuild_server',

View File

@ -32,7 +32,6 @@ class SchedulerClientTestCase(test.NoDBTestCase):
def test_constructor(self): def test_constructor(self):
self.assertIsNotNone(self.client.queryclient) self.assertIsNotNone(self.client.queryclient)
self.assertIsNotNone(self.client.reportclient)
@mock.patch.object(scheduler_query_client.SchedulerQueryClient, @mock.patch.object(scheduler_query_client.SchedulerQueryClient,
'select_destinations') 'select_destinations')

View File

@ -23,7 +23,6 @@ from oslo_utils.fixture import uuidsentinel as uuids
from nova import exception from nova import exception
from nova import objects from nova import objects
from nova.scheduler import client from nova.scheduler import client
from nova.scheduler.client import report
from nova.scheduler import filter_scheduler from nova.scheduler import filter_scheduler
from nova.scheduler import host_manager from nova.scheduler import host_manager
from nova.scheduler import utils as scheduler_utils from nova.scheduler import utils as scheduler_utils
@ -59,12 +58,11 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
@mock.patch('nova.scheduler.client.SchedulerClient') @mock.patch('nova.scheduler.client.SchedulerClient')
def setUp(self, mock_client): def setUp(self, mock_client):
pc_client = mock.Mock(spec=report.SchedulerReportClient)
sched_client = mock.Mock(spec=client.SchedulerClient) sched_client = mock.Mock(spec=client.SchedulerClient)
sched_client.reportclient = pc_client
mock_client.return_value = sched_client mock_client.return_value = sched_client
self.placement_client = pc_client with mock.patch('nova.scheduler.client.report.SchedulerReportClient',
super(FilterSchedulerTestCase, self).setUp() autospec=True):
super(FilterSchedulerTestCase, self).setUp()
@mock.patch('nova.scheduler.utils.claim_resources') @mock.patch('nova.scheduler.utils.claim_resources')
@mock.patch('nova.scheduler.filter_scheduler.FilterScheduler.' @mock.patch('nova.scheduler.filter_scheduler.FilterScheduler.'
@ -239,7 +237,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
mock.sentinel.provider_summaries) mock.sentinel.provider_summaries)
mock_get_hosts.assert_called() mock_get_hosts.assert_called()
mock_claim.assert_called_once_with(ctx.elevated.return_value, mock_claim.assert_called_once_with(ctx.elevated.return_value,
self.placement_client, spec_obj, uuids.instance, self.driver.placement_client, spec_obj, uuids.instance,
alloc_reqs_by_rp_uuid[uuids.cn1][0], alloc_reqs_by_rp_uuid[uuids.cn1][0],
allocation_request_version=None) allocation_request_version=None)
@ -307,7 +305,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
mock.sentinel.provider_summaries) mock.sentinel.provider_summaries)
mock_get_hosts.assert_called_once_with(spec_obj, all_host_states, 0) mock_get_hosts.assert_called_once_with(spec_obj, all_host_states, 0)
mock_claim.assert_called_once_with(ctx.elevated.return_value, mock_claim.assert_called_once_with(ctx.elevated.return_value,
self.placement_client, spec_obj, uuids.instance, self.driver.placement_client, spec_obj, uuids.instance,
alloc_reqs_by_rp_uuid[uuids.cn1][0], alloc_reqs_by_rp_uuid[uuids.cn1][0],
allocation_request_version=fake_version) allocation_request_version=fake_version)
@ -562,11 +560,11 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
# Check that we called claim_resources() for both the first and second # Check that we called claim_resources() for both the first and second
# host state # host state
claim_calls = [ claim_calls = [
mock.call(ctx.elevated.return_value, self.placement_client, mock.call(ctx.elevated.return_value, self.driver.placement_client,
spec_obj, uuids.instance0, spec_obj, uuids.instance0,
alloc_reqs_by_rp_uuid[uuids.cn2][0], alloc_reqs_by_rp_uuid[uuids.cn2][0],
allocation_request_version=None), allocation_request_version=None),
mock.call(ctx.elevated.return_value, self.placement_client, mock.call(ctx.elevated.return_value, self.driver.placement_client,
spec_obj, uuids.instance1, spec_obj, uuids.instance1,
alloc_reqs_by_rp_uuid[uuids.cn1][0], alloc_reqs_by_rp_uuid[uuids.cn1][0],
allocation_request_version=None), allocation_request_version=None),
@ -735,7 +733,7 @@ class FilterSchedulerTestCase(test_scheduler.SchedulerTestCase):
instance_uuids = [] instance_uuids = []
# Check we don't do anything if there's no instance UUIDs to cleanup # Check we don't do anything if there's no instance UUIDs to cleanup
# allocations for # allocations for
pc = self.placement_client pc = self.driver.placement_client
self.driver._cleanup_allocations(self.context, instance_uuids) self.driver._cleanup_allocations(self.context, instance_uuids)
self.assertFalse(pc.delete_allocation_for_instance.called) self.assertFalse(pc.delete_allocation_for_instance.called)