Trigger reschedule if PCI consumption fail on compute

The PciPassthroughFilter logic checks each InstancePCIRequest
individually against the available PCI pools of a given host and given
boot request. So it is possible that the scheduler accepts a host that
has a single PCI device available even if two devices are requested for
a single instance via two separate PCI aliases. Then the PCI claim on
the compute detects this but does not stop the boot just logs an ERROR.
This results in the instance booted without any PCI device.

This patch does two things:
1) changes the PCI claim to fail with an exception and trigger a
   re-schedule instead of just logging an ERROR.
2) change the PciDeviceStats.support_requests that is called during
   scheduling to not just filter pools for individual requests but also
   consume the request from the pool within the scope of a single boot
   request.

The fix in #2) would not be enough alone as two parallel scheduling
request could race for a single device on the same host. #1) is the
ultimate place where we consume devices under a compute global lock so
we need the fix there too.

Closes-Bug: #1986838
Change-Id: Iea477be57ae4e95dfc03acc9368f31d4be895343
This commit is contained in:
Balazs Gibizer 2022-08-17 18:19:15 +02:00
parent 2aeb0a96b7
commit 2b447b7236
8 changed files with 125 additions and 73 deletions

@ -7712,10 +7712,10 @@ class ComputeManager(manager.Manager):
if not pci_reqs.requests: if not pci_reqs.requests:
return None return None
try:
devices = self.rt.claim_pci_devices( devices = self.rt.claim_pci_devices(
context, pci_reqs, instance.numa_topology) context, pci_reqs, instance.numa_topology)
except exception.PciDeviceRequestFailed:
if not devices:
LOG.info('Failed to claim PCI devices during interface attach ' LOG.info('Failed to claim PCI devices during interface attach '
'for PCI request %s', pci_reqs, instance=instance) 'for PCI request %s', pci_reqs, instance=instance)
raise exception.InterfaceAttachPciClaimFailed( raise exception.InterfaceAttachPciClaimFailed(

@ -249,7 +249,7 @@ class PciDeviceStats(object):
"on the compute node semaphore.") "on the compute node semaphore.")
for d in range(len(alloc_devices)): for d in range(len(alloc_devices)):
self.add_device(alloc_devices.pop()) self.add_device(alloc_devices.pop())
return None raise exception.PciDeviceRequestFailed(requests=pci_requests)
for pool in pools: for pool in pools:
if pool['count'] >= count: if pool['count'] >= count:
@ -639,11 +639,22 @@ class PciDeviceStats(object):
corresponds to the ``id`` of host NUMACells, or None. corresponds to the ``id`` of host NUMACells, or None.
:returns: Whether this compute node can satisfy the given request. :returns: Whether this compute node can satisfy the given request.
""" """
# NOTE(yjiang5): this function has high possibility to fail,
# so no exception should be triggered for performance reason. # try to apply the requests on the copy of the stats if it applies
return all( # cleanly then we know that the requests is supported. We call apply
self._filter_pools(self.pools, r, numa_cells) for r in requests # only on a copy as we don't want to actually consume resources from
) # the pool as at this point this is just a test during host filtering.
# Later the scheduler will call apply_request to consume on the
# selected host. The compute will call consume_request during PCI claim
# to consume not just from the pools but also consume PciDevice
# objects.
stats = copy.deepcopy(self)
try:
stats.apply_requests(requests, numa_cells)
except exception.PciDeviceRequestFailed:
return False
return True
def _apply_request( def _apply_request(
self, self,

@ -1828,7 +1828,7 @@ class PCIServersTest(_PCIServersTestBase):
self.assertPCIDeviceCounts('test_compute1', total=2, free=0) self.assertPCIDeviceCounts('test_compute1', total=2, free=0)
def test_request_two_pci_but_host_has_one(self): def test_request_two_pci_but_host_has_one(self):
# simulate a single dev-PCI device on the host # simulate a single type-PCI device on the host
self.start_compute(pci_info=fakelibvirt.HostPCIDevicesInfo(num_pci=1)) self.start_compute(pci_info=fakelibvirt.HostPCIDevicesInfo(num_pci=1))
self.assertPCIDeviceCounts('compute1', total=1, free=1) self.assertPCIDeviceCounts('compute1', total=1, free=1)
@ -1849,45 +1849,12 @@ class PCIServersTest(_PCIServersTestBase):
# single available device on the host # single available device on the host
extra_spec = {'pci_passthrough:alias': 'a1:1,a2:1'} extra_spec = {'pci_passthrough:alias': 'a1:1,a2:1'}
flavor_id = self._create_flavor(extra_spec=extra_spec) flavor_id = self._create_flavor(extra_spec=extra_spec)
# so we expect that the boot fails with no valid host erro as only # so we expect that the boot fails with no valid host error as only
# one of the requested PCI device can be allocated # one of the requested PCI device can be allocated
# server = self._create_server( server = self._create_server(
# flavor_id=flavor_id, expected_state='ERROR') flavor_id=flavor_id, networks="none", expected_state='ERROR')
# self.assertIn('fault', server) self.assertIn('fault', server)
# self.assertIn('No valid host', server['fault']['message']) self.assertIn('No valid host', server['fault']['message'])
# This is bug 1986838
# The boot succeeds and none of the requested devices will be allocated
# to the instance.
server = self._create_server(flavor_id=flavor_id, networks='none')
self.assertPCIDeviceCounts('compute1', total=1, free=1)
devices = objects.PciDeviceList.get_by_instance_uuid(
self.ctxt, server['id'])
self.assertEqual(0, len(devices))
# the scheduler fails to consume the pci request from the host
# but the scheduler also simply ignore this "small" problem by
# pointing to compute that during the pci_claim the compute service
# will do the right thing and fail loudly and trigger a re-schedule.
# See
# https://github.com/openstack/nova/blob/69bc4c38d1c5b98fcbbe8b16a7dfeb654e3b8173/nova/scheduler/host_manager.py#L81-L87
self.assertIn(
'WARNING [nova.scheduler.host_manager] Selected host: compute1 '
'failed to consume from instance. Error: PCI device request',
self.stdlog.logger.output
)
# And yes, the compute service detects the failure but states that this
# should not happen as the scheduler needed to choose the host
# properly. Then it simply cleans up all the instance PCI allocations
# and ignores the fault, so the instance boots with the requested PCI
# devs.
self.assertIn(
'ERROR [nova.pci.stats] Failed to allocate PCI devices for '
'instance. Unassigning devices back to pools. This should not '
'happen, since the scheduler should have accurate information, '
'and allocation during claims is controlled via a hold on the '
'compute node semaphore.',
self.stdlog.logger.output
)
class PCIServersWithPreferredNUMATest(_PCIServersTestBase): class PCIServersWithPreferredNUMATest(_PCIServersTestBase):

@ -10755,8 +10755,13 @@ class ComputeAPITestCase(BaseTestCase):
supports_attach_interface=True), supports_attach_interface=True),
mock.patch.object(self.compute.network_api, mock.patch.object(self.compute.network_api,
'create_resource_requests'), 'create_resource_requests'),
mock.patch.object(self.compute.rt, 'claim_pci_devices', mock.patch.object(
return_value=[]), self.compute.rt,
'claim_pci_devices',
side_effect=exception.PciDeviceRequestFailed(
requests=instance.pci_requests
)
),
mock.patch.object( mock.patch.object(
self.compute, '_allocate_port_resource_for_instance'), self.compute, '_allocate_port_resource_for_instance'),
mock.patch( mock.patch(

@ -18,6 +18,7 @@ from keystoneauth1 import exceptions as ks_exc
import os_resource_classes as orc import os_resource_classes as orc
import os_traits import os_traits
from oslo_config import cfg from oslo_config import cfg
from oslo_serialization import jsonutils
from oslo_utils.fixture import uuidsentinel as uuids from oslo_utils.fixture import uuidsentinel as uuids
from oslo_utils import timeutils from oslo_utils import timeutils
from oslo_utils import units from oslo_utils import units
@ -2124,26 +2125,45 @@ class TestInstanceClaim(BaseTestCase):
# PCI devices on the host and sends an updated pci_device_pools # PCI devices on the host and sends an updated pci_device_pools
# attribute of the ComputeNode object. # attribute of the ComputeNode object.
self.flags(
group="pci",
device_spec=[
jsonutils.dumps({"vendor_id": "0001", "product_id": "0002"})
],
)
pci_dev = pci_device.PciDevice.create(
None,
dev_dict={
"compute_node_id": 1,
"address": "0000:81:00.0",
"product_id": "0002",
"vendor_id": "0001",
"numa_node": 0,
"dev_type": obj_fields.PciDeviceType.STANDARD,
"status": obj_fields.PciDeviceStatus.AVAILABLE,
"parent_addr": None,
},
)
pci_dev.instance_uuid = None
pci_devs = [pci_dev]
# TODO(jaypipes): Remove once the PCI tracker is always created # TODO(jaypipes): Remove once the PCI tracker is always created
# upon the resource tracker being initialized... # upon the resource tracker being initialized...
with mock.patch.object( with mock.patch.object(
objects.PciDeviceList, 'get_by_compute_node', objects.PciDeviceList, 'get_by_compute_node',
return_value=objects.PciDeviceList() return_value=objects.PciDeviceList(objects=pci_devs)
): ):
self.rt.pci_tracker = pci_manager.PciDevTracker( self.rt.pci_tracker = pci_manager.PciDevTracker(
mock.sentinel.ctx, _COMPUTE_NODE_FIXTURES[0]) mock.sentinel.ctx, _COMPUTE_NODE_FIXTURES[0])
pci_dev = pci_device.PciDevice.create(
None, fake_pci_device.dev_dict)
pci_devs = [pci_dev]
self.rt.pci_tracker.pci_devs = objects.PciDeviceList(objects=pci_devs)
request = objects.InstancePCIRequest(count=1, request = objects.InstancePCIRequest(count=1,
spec=[{'vendor_id': 'v', 'product_id': 'p'}]) spec=[{'vendor_id': '0001', 'product_id': '0002'}])
pci_requests = objects.InstancePCIRequests( pci_requests = objects.InstancePCIRequests(
requests=[request], requests=[request],
instance_uuid=self.instance.uuid) instance_uuid=self.instance.uuid)
self.instance.pci_requests = pci_requests self.instance.pci_requests = pci_requests
self.instance.pci_devices = objects.PciDeviceList()
check_bfv_mock.return_value = False check_bfv_mock.return_value = False
disk_used = self.instance.root_gb + self.instance.ephemeral_gb disk_used = self.instance.root_gb + self.instance.ephemeral_gb
@ -2155,7 +2175,17 @@ class TestInstanceClaim(BaseTestCase):
"free_ram_mb": expected.memory_mb - self.instance.memory_mb, "free_ram_mb": expected.memory_mb - self.instance.memory_mb,
'running_vms': 1, 'running_vms': 1,
'vcpus_used': 1, 'vcpus_used': 1,
'pci_device_pools': objects.PciDevicePoolList(), 'pci_device_pools': objects.PciDevicePoolList(
objects=[
objects.PciDevicePool(
vendor_id='0001',
product_id='0002',
numa_node=0,
tags={'dev_type': 'type-PCI'},
count=0
)
]
),
'stats': { 'stats': {
'io_workload': 0, 'io_workload': 0,
'num_instances': 1, 'num_instances': 1,

@ -651,8 +651,13 @@ class PciDevTrackerTestCase(test.NoDBTestCase):
pci_requests = copy.deepcopy(fake_pci_requests) pci_requests = copy.deepcopy(fake_pci_requests)
pci_requests[0]['count'] = 4 pci_requests[0]['count'] = 4
pci_requests_obj = self._create_pci_requests_object(pci_requests) pci_requests_obj = self._create_pci_requests_object(pci_requests)
self.tracker.claim_instance(mock.sentinel.context, self.assertRaises(
pci_requests_obj, None) exception.PciDeviceRequestFailed,
self.tracker.claim_instance,
mock.sentinel.context,
pci_requests_obj,
None
)
self.assertEqual(len(self.tracker.claims[self.inst['uuid']]), 0) self.assertEqual(len(self.tracker.claims[self.inst['uuid']]), 0)
devs = self.tracker.update_pci_for_instance(None, devs = self.tracker.update_pci_for_instance(None,
self.inst, self.inst,
@ -687,11 +692,13 @@ class PciDevTrackerTestCase(test.NoDBTestCase):
self.inst.numa_topology = objects.InstanceNUMATopology( self.inst.numa_topology = objects.InstanceNUMATopology(
cells=[objects.InstanceNUMACell( cells=[objects.InstanceNUMACell(
id=1, cpuset=set([1, 2]), memory=512)]) id=1, cpuset=set([1, 2]), memory=512)])
claims = self.tracker.claim_instance( self.assertRaises(
exception.PciDeviceRequestFailed,
self.tracker.claim_instance,
mock.sentinel.context, mock.sentinel.context,
pci_requests_obj, pci_requests_obj,
self.inst.numa_topology) self.inst.numa_topology
self.assertEqual([], claims) )
def test_update_pci_for_instance_deleted(self): def test_update_pci_for_instance_deleted(self):
pci_requests_obj = self._create_pci_requests_object(fake_pci_requests) pci_requests_obj = self._create_pci_requests_object(fake_pci_requests)

@ -262,8 +262,11 @@ class PciDeviceStatsTestCase(test.NoDBTestCase):
self.assertEqual(0, len(devs)) self.assertEqual(0, len(devs))
def test_consume_requests_failed(self): def test_consume_requests_failed(self):
self.assertIsNone(self.pci_stats.consume_requests( self.assertRaises(
pci_requests_multiple)) exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests_multiple,
)
def test_consume_requests_numa(self): def test_consume_requests_numa(self):
cells = [ cells = [
@ -282,7 +285,12 @@ class PciDeviceStatsTestCase(test.NoDBTestCase):
objects.InstanceNUMACell( objects.InstanceNUMACell(
id=0, cpuset=set(), pcpuset=set(), memory=0), id=0, cpuset=set(), pcpuset=set(), memory=0),
] ]
self.assertIsNone(self.pci_stats.consume_requests(pci_requests, cells)) self.assertRaises(
exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests,
cells,
)
def test_consume_requests_no_numa_info(self): def test_consume_requests_no_numa_info(self):
cells = [ cells = [
@ -314,11 +322,16 @@ class PciDeviceStatsTestCase(test.NoDBTestCase):
pci_requests = self._get_fake_requests(vendor_ids=[vendor_id], pci_requests = self._get_fake_requests(vendor_ids=[vendor_id],
numa_policy=policy, count=count) numa_policy=policy, count=count)
devs = self.pci_stats.consume_requests(pci_requests, cells)
if expected is None: if expected is None:
self.assertIsNone(devs) self.assertRaises(
exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests,
cells,
)
else: else:
devs = self.pci_stats.consume_requests(pci_requests, cells)
self.assertEqual(set(expected), self.assertEqual(set(expected),
set([dev.product_id for dev in devs])) set([dev.product_id for dev in devs]))
@ -907,13 +920,21 @@ class PciDeviceVFPFStatsTestCase(test.NoDBTestCase):
objects.InstancePCIRequest(count=1, objects.InstancePCIRequest(count=1,
spec=[{'product_id': '1528', spec=[{'product_id': '1528',
'dev_type': 'type-PF'}])] 'dev_type': 'type-PF'}])]
self.assertIsNone(self.pci_stats.consume_requests(pci_requests)) self.assertRaises(
exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests,
)
def test_consume_VF_and_PF_same_product_id_failed(self): def test_consume_VF_and_PF_same_product_id_failed(self):
self._create_pci_devices(pf_product_id=1515) self._create_pci_devices(pf_product_id=1515)
pci_requests = [objects.InstancePCIRequest(count=9, pci_requests = [objects.InstancePCIRequest(count=9,
spec=[{'product_id': '1515'}])] spec=[{'product_id': '1515'}])]
self.assertIsNone(self.pci_stats.consume_requests(pci_requests)) self.assertRaises(
exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests,
)
def test_consume_PF_not_remote_managed(self): def test_consume_PF_not_remote_managed(self):
self._create_pci_devices() self._create_pci_devices()
@ -955,8 +976,11 @@ class PciDeviceVFPFStatsTestCase(test.NoDBTestCase):
objects.InstancePCIRequest(count=1, objects.InstancePCIRequest(count=1,
spec=[{'product_id': '101e'}])] spec=[{'product_id': '101e'}])]
free_devs_before = self.pci_stats.get_free_devs() free_devs_before = self.pci_stats.get_free_devs()
devs = self.pci_stats.consume_requests(pci_requests) self.assertRaises(
self.assertIsNone(devs) exception.PciDeviceRequestFailed,
self.pci_stats.consume_requests,
pci_requests,
)
free_devs_after = self.pci_stats.get_free_devs() free_devs_after = self.pci_stats.get_free_devs()
self.assertEqual(free_devs_before, free_devs_after) self.assertEqual(free_devs_before, free_devs_after)

@ -0,0 +1,8 @@
---
fixes:
- |
`Bug #1986838 <https://bugs.launchpad.net/nova/+bug/1986838>`_: Nova now
correctly schedules an instance that requests multiple PCI devices via
multiple PCI aliases in the flavor extra_spec when multiple similar devices
are requested but the compute host has only one such device matching with
each request individually.