Store allocated RP in InstancePCIRequest

After the scheduler selected a target host and allocated an allocation
candidate that is passed the filters nova need to make sure that PCI
claim will allocate the real PCI devices from the RP which is allocated
in placement. Placement returns the request group - provider mapping for
each allocation candidate so nova can map which InstancePCIRequest was
fulfilled from which RP in the selected allocation candidate. This
mapping is then recorded in the InstancePCIRequest object and used
during the PCI claim to filter for PCI pools that can be used to claim
PCI devices from.

blueprint: pci-device-tracking-in-placement
Change-Id: I18bb31e23cc014411db68c31317ed983886d1a8e
This commit is contained in:
Balazs Gibizer 2022-08-19 16:46:23 +02:00
parent f1d82c0d0a
commit f86f1800f0
11 changed files with 227 additions and 59 deletions

View File

@ -2472,7 +2472,7 @@ class ComputeManager(manager.Manager):
if provider_mapping:
try:
compute_utils\
.update_pci_request_spec_with_allocated_interface_name(
.update_pci_request_with_placement_allocations(
context, self.reportclient,
instance.pci_requests.requests, provider_mapping)
except (exception.AmbiguousResourceProviderForPCIRequest,
@ -3789,7 +3789,7 @@ class ComputeManager(manager.Manager):
if provider_mapping:
compute_utils.\
update_pci_request_spec_with_allocated_interface_name(
update_pci_request_with_placement_allocations(
context, self.reportclient,
instance.pci_requests.requests, provider_mapping)
@ -5415,7 +5415,7 @@ class ComputeManager(manager.Manager):
if provider_mapping:
try:
compute_utils.\
update_pci_request_spec_with_allocated_interface_name(
update_pci_request_with_placement_allocations(
context, self.reportclient,
instance.pci_requests.requests, provider_mapping)
except (exception.AmbiguousResourceProviderForPCIRequest,
@ -5520,7 +5520,7 @@ class ComputeManager(manager.Manager):
clean_shutdown)
except exception.BuildAbortException:
# NOTE(gibi): We failed
# update_pci_request_spec_with_allocated_interface_name so
# update_pci_request_with_placement_allocations so
# there is no reason to re-schedule. Just revert the allocation
# and fail the migration.
with excutils.save_and_reraise_exception():
@ -5651,7 +5651,7 @@ class ComputeManager(manager.Manager):
'host (%s).', self.host, instance=instance)
self._send_prep_resize_notifications(
ctxt, instance, fields.NotificationPhase.START, flavor)
# TODO(mriedem): update_pci_request_spec_with_allocated_interface_name
# TODO(mriedem): update_pci_request_with_placement_allocations
# should be called here if the request spec has request group mappings,
# e.g. for things like QoS ports with resource requests. Do it outside
# the try/except so if it raises BuildAbortException we do not attempt
@ -6900,7 +6900,7 @@ class ComputeManager(manager.Manager):
if provider_mappings:
update = (
compute_utils.
update_pci_request_spec_with_allocated_interface_name)
update_pci_request_with_placement_allocations)
update(
context, self.reportclient, instance.pci_requests.requests,
provider_mappings)
@ -7985,7 +7985,7 @@ class ComputeManager(manager.Manager):
try:
update = (
compute_utils.
update_pci_request_spec_with_allocated_interface_name)
update_pci_request_with_placement_allocations)
update(
context, self.reportclient, pci_reqs.requests,
provider_mappings)

View File

@ -1491,7 +1491,7 @@ def notify_about_instance_delete(notifier, context, instance,
phase=fields.NotificationPhase.END)
def update_pci_request_spec_with_allocated_interface_name(
def update_pci_request_with_placement_allocations(
context, report_client, pci_requests, provider_mapping):
"""Update the instance's PCI request based on the request group -
resource provider mapping and the device RP name from placement.
@ -1512,12 +1512,33 @@ def update_pci_request_spec_with_allocated_interface_name(
if not pci_requests:
return
def needs_update(pci_request, mapping):
def needs_update_due_to_qos(pci_request, mapping):
return (pci_request.requester_id and
pci_request.requester_id in mapping)
def get_group_mapping_for_flavor_based_pci_request(pci_request, mapping):
# NOTE(gibi): for flavor based PCI requests nova generates RequestGroup
# suffixes from InstancePCIRequests in the form of
# {request_id}-{count_index}
# NOTE(gibi): a suffixed request group always fulfilled from a single
# RP
return {
group_id: rp_uuids[0]
for group_id, rp_uuids in mapping.items()
if group_id.startswith(pci_request.request_id)
}
for pci_request in pci_requests:
if needs_update(pci_request, provider_mapping):
mapping = get_group_mapping_for_flavor_based_pci_request(
pci_request, provider_mapping)
if mapping:
for spec in pci_request.spec:
# FIXME(gibi): this is baaad but spec is a dict of strings so
# we need to serialize
spec['rp_uuids'] = ','.join(set(mapping.values()))
elif needs_update_due_to_qos(pci_request, provider_mapping):
provider_uuids = provider_mapping[pci_request.requester_id]
if len(provider_uuids) != 1:

View File

@ -542,7 +542,7 @@ class LiveMigrationTask(base.TaskBase):
# will be persisted when post_live_migration_at_destination
# runs.
compute_utils.\
update_pci_request_spec_with_allocated_interface_name(
update_pci_request_with_placement_allocations(
self.context, self.report_client,
self.instance.pci_requests.requests, provider_mapping)
try:

View File

@ -69,6 +69,10 @@ class PciDeviceStats(object):
# allocation_candidates query, so we can ignore them during pool creation
# and during filtering here
ignored_tags = ['resource_class', 'traits']
# these are metadata keys in the pool and in the request that are matched
# specially in _filter_pools_based_on_placement_allocation. So we can
# ignore them in the general matching logic.
ignored_tags += ['rp_uuid', 'rp_uuids']
def __init__(
self,
@ -253,10 +257,10 @@ class PciDeviceStats(object):
for request in pci_requests:
count = request.count
# FIXME(gibi): we need to gather the rp_uuids from the
# InstancePCIRequests once stored there
rp_uuids = self._get_rp_uuids_for_request(
request=request, provider_mapping=None)
pools = self._filter_pools(
self.pools, request, numa_cells, rp_uuids=set())
self.pools, request, numa_cells, rp_uuids=rp_uuids)
# Failed to allocate the required number of devices. Return the
# devices already allocated during previous iterations back to
@ -795,15 +799,26 @@ class PciDeviceStats(object):
) -> ty.Set[str]:
"""Return the list of RP uuids that are fulfilling the request"""
if not provider_mapping:
# FIXME(gibi): read the mapping from the request
return set()
if request.source == objects.InstancePCIRequest.NEUTRON_PORT:
# TODO(gibi): support neutron based requests in a later cycle
# set() will signal that any PCI pool can be used for this request
return set()
if not provider_mapping:
# NOTE(gibi): AFAIK specs is always a list of a single dict
# but the object is hard to change retroactively
rp_uuids = request.spec[0].get('rp_uuids')
if not rp_uuids:
# This can happen if [scheduler]pci_in_placement is not
# enabled yet
# set() will signal that any PCI pool can be used for this
# request
return set()
# TODO(gibi): this is baaad but spec is a dict of string so
# the list is serialized
return set(rp_uuids.split(','))
# NOTE(gibi): the PCI prefilter generates RequestGroup suffixes from
# InstancePCIRequests in the form of {request_id}-{count_index}
# NOTE(gibi): a suffixed request group always fulfilled from a single

View File

@ -1068,7 +1068,7 @@ class PortResourceRequestBasedSchedulingTest(
def test_interface_attach_sriov_with_qos_pci_update_fails(self):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the sriov interface is attached.
rsp = self.placement.put(
'/resource_providers/%s'
@ -1115,7 +1115,7 @@ class PortResourceRequestBasedSchedulingTest(
):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the sriov interface is attached.
rsp = self.placement.put(
'/resource_providers/%s'
@ -1923,7 +1923,7 @@ class ServerMoveWithPortResourceRequestTest(
def test_migrate_server_with_qos_port_pci_update_fail_not_reschedule(self):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the instance is migrated to the host2.
rsp = self.placement.put(
'/resource_providers/%s'
@ -1943,7 +1943,7 @@ class ServerMoveWithPortResourceRequestTest(
non_qos_port, qos_port, qos_sriov_port)
# The compute manager on host2 will raise from
# update_pci_request_spec_with_allocated_interface_name which will
# update_pci_request_with_placement_allocations which will
# intentionally not trigger a re-schedule even if there is host3 as an
# alternate.
self.api.post_server_action(server['id'], {'migrate': None})
@ -2186,7 +2186,7 @@ class ServerMoveWithPortResourceRequestTest(
def test_evacuate_with_qos_port_pci_update_fail(self):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the instance is evacuated to the host2.
rsp = self.placement.put(
'/resource_providers/%s'
@ -2207,7 +2207,7 @@ class ServerMoveWithPortResourceRequestTest(
self.compute1_service_id, {'forced_down': 'true'})
# The compute manager on host2 will raise from
# update_pci_request_spec_with_allocated_interface_name
# update_pci_request_with_placement_allocations
server = self._evacuate_server(
server, expected_host='host1', expected_state='ERROR',
expected_task_state=None, expected_migration_status='failed')
@ -2363,7 +2363,7 @@ class ServerMoveWithPortResourceRequestTest(
def test_live_migrate_with_qos_port_pci_update_fails(self):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the instance is live migrated to the host2.
rsp = self.placement.put(
'/resource_providers/%s'
@ -2504,7 +2504,7 @@ class ServerMoveWithPortResourceRequestTest(
def test_unshelve_offloaded_server_with_qos_port_pci_update_fails(self):
# Update the name of the network device RP of PF2 on host2 to something
# unexpected. This will cause
# update_pci_request_spec_with_allocated_interface_name() to raise
# update_pci_request_with_placement_allocations() to raise
# when the instance is unshelved to the host2.
rsp = self.placement.put(
'/resource_providers/%s'
@ -2537,7 +2537,7 @@ class ServerMoveWithPortResourceRequestTest(
self.api.post_server_action(server['id'], {'unshelve': None})
# Unshelve fails on host2 due to
# update_pci_request_spec_with_allocated_interface_name fails so the
# update_pci_request_with_placement_allocations fails so the
# instance goes back to shelve offloaded state
self.notifier.wait_for_versioned_notifications(
'instance.unshelve.start')

View File

@ -10843,7 +10843,7 @@ class ComputeAPITestCase(BaseTestCase):
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name'),
'update_pci_request_with_placement_allocations'),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
mock_update_pci
@ -10913,7 +10913,7 @@ class ComputeAPITestCase(BaseTestCase):
new=mock.NonCallableMock()),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name',
'update_pci_request_with_placement_allocations',
new=mock.NonCallableMock()),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
@ -10958,7 +10958,7 @@ class ComputeAPITestCase(BaseTestCase):
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name',
'update_pci_request_with_placement_allocations',
new=mock.NonCallableMock()),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
@ -11025,7 +11025,7 @@ class ComputeAPITestCase(BaseTestCase):
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name'),
'update_pci_request_with_placement_allocations'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'remove_resources_from_instance_allocation'),
@ -13520,7 +13520,7 @@ class EvacuateHostTestCase(BaseTestCase):
return_value=mock.sentinel.mapping)
@mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
@mock.patch('nova.compute.utils.notify_about_instance_action')
@mock.patch('nova.compute.utils.notify_about_instance_rebuild')
@mock.patch.object(network_api, 'setup_networks_on_host')

View File

@ -8487,11 +8487,17 @@ class ComputeManagerBuildInstanceTestCase(test.NoDBTestCase):
# resource request and therefore no matching request group exists in
# the request spec.
self.instance.pci_requests = objects.InstancePCIRequests(requests=[
objects.InstancePCIRequest(),
objects.InstancePCIRequest(
request_id=uuids.req0,
),
objects.InstancePCIRequest(
request_id=uuids.req1,
requester_id=uuids.port1,
spec=[{'vendor_id': '1377', 'product_id': '0047'}]),
objects.InstancePCIRequest(requester_id=uuids.port2),
objects.InstancePCIRequest(
request_id=uuids.req2,
requester_id=uuids.port2,
),
])
with test.nested(
mock.patch.object(self.compute.driver, 'spawn'),
@ -8536,8 +8542,13 @@ class ComputeManagerBuildInstanceTestCase(test.NoDBTestCase):
requester_id=uuids.port1,
provider_uuids=[uuids.rp1])])
self.instance.pci_requests = objects.InstancePCIRequests(requests=[
objects.InstancePCIRequest(requester_id=uuids.port1)])
self.instance.pci_requests = objects.InstancePCIRequests(
requests=[
objects.InstancePCIRequest(
requester_id=uuids.port1, request_id=uuids.req1
)
]
)
with mock.patch('nova.scheduler.client.report.SchedulerReportClient.'
'_get_resource_provider') as (mock_get_rp):
mock_get_rp.return_value = None
@ -8559,8 +8570,13 @@ class ComputeManagerBuildInstanceTestCase(test.NoDBTestCase):
requester_id=uuids.port1,
provider_uuids=[uuids.rp1])])
self.instance.pci_requests = objects.InstancePCIRequests(requests=[
objects.InstancePCIRequest(requester_id=uuids.port1)])
self.instance.pci_requests = objects.InstancePCIRequests(
requests=[
objects.InstancePCIRequest(
requester_id=uuids.port1, request_id=uuids.req1
)
]
)
with mock.patch('nova.scheduler.client.report.SchedulerReportClient.'
'_get_resource_provider') as (mock_get_rp):
mock_get_rp.return_value = {
@ -8584,8 +8600,13 @@ class ComputeManagerBuildInstanceTestCase(test.NoDBTestCase):
requester_id=uuids.port1,
provider_uuids=[uuids.rp1, uuids.rp2])])
self.instance.pci_requests = objects.InstancePCIRequests(requests=[
objects.InstancePCIRequest(requester_id=uuids.port1)])
self.instance.pci_requests = objects.InstancePCIRequests(
requests=[
objects.InstancePCIRequest(
requester_id=uuids.port1, request_id=uuids.req1
)
]
)
self.assertRaises(
exception.BuildAbortException,
@ -11040,7 +11061,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase,
@mock.patch('nova.compute.resource_tracker.ResourceTracker.resize_claim')
@mock.patch('nova.objects.Instance.save')
@mock.patch('nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
@mock.patch('nova.compute.utils.notify_usage_exists')
@mock.patch('nova.compute.manager.ComputeManager.'
'_notify_about_instance_usage')
@ -11074,7 +11095,7 @@ class ComputeManagerMigrationTestCase(test.NoDBTestCase,
@mock.patch('nova.compute.resource_tracker.ResourceTracker.resize_claim')
@mock.patch('nova.objects.Instance.save')
@mock.patch('nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
@mock.patch('nova.compute.utils.notify_usage_exists')
@mock.patch('nova.compute.manager.ComputeManager.'
'_notify_about_instance_usage')

View File

@ -281,7 +281,7 @@ class ShelveComputeManagerTestCase(test_compute.BaseTestCase):
return instance
@mock.patch('nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name',
'update_pci_request_with_placement_allocations',
new=mock.NonCallableMock())
@mock.patch('nova.objects.BlockDeviceMappingList.get_by_instance_uuid')
@mock.patch('nova.compute.utils.notify_about_instance_action')
@ -631,7 +631,7 @@ class ShelveComputeManagerTestCase(test_compute.BaseTestCase):
@mock.patch('nova.network.neutron.API.setup_instance_network_on_host')
@mock.patch('nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
def test_unshelve_with_resource_request(
self, mock_update_pci, mock_setup_network):
requested_res = [objects.RequestGroup(
@ -655,7 +655,7 @@ class ShelveComputeManagerTestCase(test_compute.BaseTestCase):
@mock.patch('nova.network.neutron.API.setup_instance_network_on_host',
new=mock.NonCallableMock())
@mock.patch('nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
def test_unshelve_with_resource_request_update_raises(
self, mock_update_pci):
requested_res = [objects.RequestGroup(

View File

@ -1558,47 +1558,86 @@ class PciRequestUpdateTestCase(test.NoDBTestCase):
def test_no_pci_request(self):
provider_mapping = {}
compute_utils.update_pci_request_spec_with_allocated_interface_name(
compute_utils.update_pci_request_with_placement_allocations(
self.context, mock.sentinel.report_client, [], provider_mapping)
def test_pci_request_from_flavor(self):
pci_requests = [objects.InstancePCIRequest(requester_id=None)]
def test_pci_request_from_flavor_no_mapping(self):
req = objects.InstancePCIRequest(
requester_id=None,
request_id=uuids.req1,
alias_name="a-dev",
spec=[{}],
)
pci_requests = [req]
provider_mapping = {}
compute_utils.update_pci_request_spec_with_allocated_interface_name(
compute_utils.update_pci_request_with_placement_allocations(
self.context, mock.sentinel.report_client, pci_requests,
provider_mapping)
self.assertNotIn('rp_uuids', req.spec[0])
def test_pci_request_from_flavor_with_mapping(self):
req = objects.InstancePCIRequest(
requester_id=None,
request_id=uuids.req1,
alias_name="a-dev",
spec=[{}],
)
pci_requests = [req]
provider_mapping = {
f"{uuids.req1}-0": [uuids.rp1],
f"{uuids.req1}-1": [uuids.rp2],
}
compute_utils.update_pci_request_with_placement_allocations(
self.context, mock.sentinel.report_client, pci_requests,
provider_mapping)
self.assertEqual(
",".join({uuids.rp1, uuids.rp2}), req.spec[0]["rp_uuids"]
)
def test_pci_request_has_no_mapping(self):
pci_requests = [objects.InstancePCIRequest(requester_id=uuids.port_1)]
provider_mapping = {}
compute_utils.update_pci_request_spec_with_allocated_interface_name(
compute_utils.update_pci_request_with_placement_allocations(
self.context, mock.sentinel.report_client, pci_requests,
provider_mapping)
def test_pci_request_ambiguous_mapping(self):
pci_requests = [objects.InstancePCIRequest(requester_id=uuids.port_1)]
pci_requests = [
objects.InstancePCIRequest(
requester_id=uuids.port_1, request_id=uuids.req1
)
]
provider_mapping = {uuids.port_1: [uuids.rp1, uuids.rp2]}
self.assertRaises(
exception.AmbiguousResourceProviderForPCIRequest,
(compute_utils.
update_pci_request_spec_with_allocated_interface_name),
update_pci_request_with_placement_allocations),
self.context, mock.sentinel.report_client, pci_requests,
provider_mapping)
def test_unexpected_provider_name(self):
report_client = mock.Mock(spec=report.SchedulerReportClient)
report_client.get_resource_provider_name.return_value = 'unexpected'
pci_requests = [objects.InstancePCIRequest(
requester_id=uuids.port_1, spec=[{}])]
pci_requests = [
objects.InstancePCIRequest(
requester_id=uuids.port_1, spec=[{}], request_id=uuids.req1
)
]
provider_mapping = {uuids.port_1: [uuids.rp1]}
self.assertRaises(
exception.UnexpectedResourceProviderNameForPCIRequest,
(compute_utils.
update_pci_request_spec_with_allocated_interface_name),
update_pci_request_with_placement_allocations),
self.context, report_client, pci_requests,
provider_mapping)
@ -1610,11 +1649,14 @@ class PciRequestUpdateTestCase(test.NoDBTestCase):
report_client = mock.Mock(spec=report.SchedulerReportClient)
report_client.get_resource_provider_name.return_value = (
'host:agent:enp0s31f6')
pci_requests = [objects.InstancePCIRequest(
requester_id=uuids.port_1, spec=[{}],)]
pci_requests = [
objects.InstancePCIRequest(
requester_id=uuids.port_1, spec=[{}], request_id=uuids.req1
)
]
provider_mapping = {uuids.port_1: [uuids.rp1]}
compute_utils.update_pci_request_spec_with_allocated_interface_name(
compute_utils.update_pci_request_with_placement_allocations(
self.context, report_client, pci_requests, provider_mapping)
report_client.get_resource_provider_name.assert_called_once_with(

View File

@ -761,7 +761,7 @@ class LiveMigrationTaskTestCase(test.NoDBTestCase):
@mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name')
'update_pci_request_with_placement_allocations')
@mock.patch('nova.scheduler.utils.fill_provider_mapping')
@mock.patch.object(live_migrate.LiveMigrationTask,
'_call_livem_checks_on_host')

View File

@ -1349,6 +1349,75 @@ class PciDeviceStatsProviderMappingTestCase(test.NoDBTestCase):
{pool['rp_uuid'] for pool in self.pci_stats.pools},
)
def test_apply_gets_requested_uuids_from_pci_req(self):
pf_req = objects.InstancePCIRequest(
count=1,
alias_name='a-dev',
request_id=uuids.req1,
spec=[
{
"vendor_id": "dead",
"product_id": "beef",
"dev_type": "type-PF",
# Simulate that the scheduler already allocate a candidate
# and the mapping is stored in the request.
# The allocation restricts that we can only consume from
# PF3
"rp_uuids": ",".join([uuids.pf3])
}
],
)
# call apply with None mapping signalling that the allocation is
# already done and the resulted mapping is stored in the request
self.pci_stats.apply_requests([pf_req], provider_mapping=None)
# assert that the right device is consumed
self.assertEqual(self.num_pools - 1, len(self.pci_stats.pools))
self.assertEqual(
self.num_devs - 1,
sum(pool["count"] for pool in self.pci_stats.pools),
)
# pf3 is not available in the pools anymore
self.assertEqual(
{uuids.pf1, uuids.pf2, uuids.pci1},
{pool['rp_uuid'] for pool in self.pci_stats.pools},
)
def test_consume_restricted_by_allocation(self):
pf_req = objects.InstancePCIRequest(
count=1,
alias_name='a-dev',
request_id=uuids.req1,
spec=[
{
"vendor_id": "dead",
"product_id": "beef",
"dev_type": "type-PF",
# Simulate that the scheduler already allocate a candidate
# and the mapping is stored in the request.
# The allocation restricts that we can only consume from
# PF3
"rp_uuids": ",".join([uuids.pf3])
}
],
)
# Call consume. It always expects the allocated mapping to be stores
# the in PCI request as it is always called from the compute side.
consumed_devs = self.pci_stats.consume_requests([pf_req])
# assert that the right device is consumed
self.assertEqual([self.pf3], consumed_devs)
# pf3 is not available in the pools anymore
self.assertEqual(
{uuids.pf1, uuids.pf2, uuids.pci1},
{
pool["rp_uuid"]
for pool in self.pci_stats.pools
if pool["count"] > 0
},
)
class PciDeviceVFPFStatsTestCase(test.NoDBTestCase):