Support interface attach with QoS ports

The following logic is added to the ComputeManager attach_interface
flow:

* gather the resource request of the port from neutron
* query allocation candidates restricted to the current compute node
  from placement
* extend the existing allocation of the instance with one of the
  allocation candidates in placement
* update the InstancePCIRequest (if any) to ensure that the PCI claim
  only allocates VF from the PF the placement resources are allocated from
* ensure that during port binding neutron gets the RP UUID, the resources
  are allocated from, in the allocation key of the binding profile

This patch bumps the compute service version to indicate that QoS
interface attach is supported. Also the check that was so far rejected
such attach is now updated to only reject it if the compute service
version is too old.

The "scheduling" during interface attach for PCI backed ports has the
same limitation as normal scheduling for such ports. It always selects
the first allocation candidate returned by placement even if later in
the process it turns out that such allocation candidate points to a PCI
PF that has no free VFs left.

Blueprint: support-interface-attach-with-qos-ports

Change-Id: Id4684093e8bdf3b61667490443e3d2f6ed65f4b3
This commit is contained in:
Balazs Gibizer 2020-10-09 15:56:56 +02:00
parent c3804efd42
commit 5af28c4de5
13 changed files with 944 additions and 28 deletions

View File

@ -36,5 +36,8 @@ same-cell resize if the server has such ports.
As of 21.0.0 (Ussuri), nova supports evacuating, live migrating and unshelving As of 21.0.0 (Ussuri), nova supports evacuating, live migrating and unshelving
servers with neutron ports having resource requests. servers with neutron ports having resource requests.
As of 23.0.0 (Wallaby), nova supports attaching neutron ports having QoS
minimum bandwidth rules.
See :nova-doc:`the admin guide <admin/port_with_resource_request.html>` for See :nova-doc:`the admin guide <admin/port_with_resource_request.html>` for
administrative details. administrative details.

View File

@ -175,7 +175,8 @@ class InterfaceAttachmentController(wsgi.Controller):
exception.SecurityGroupCannotBeApplied, exception.SecurityGroupCannotBeApplied,
exception.NetworkInterfaceTaggedAttachNotSupported, exception.NetworkInterfaceTaggedAttachNotSupported,
exception.NetworksWithQoSPolicyNotSupported, exception.NetworksWithQoSPolicyNotSupported,
exception.InterfaceAttachPciClaimFailed) as e: exception.InterfaceAttachPciClaimFailed,
exception.InterfaceAttachResourceAllocationFailed) as e:
raise exc.HTTPBadRequest(explanation=e.format_message()) raise exc.HTTPBadRequest(explanation=e.format_message())
except (exception.InstanceIsLocked, except (exception.InstanceIsLocked,
exception.FixedIpAlreadyInUse, exception.FixedIpAlreadyInUse,

View File

@ -4976,14 +4976,17 @@ class API(base.Base):
context, instance, instance_actions.ATTACH_INTERFACE) context, instance, instance_actions.ATTACH_INTERFACE)
# NOTE(gibi): Checking if the requested port has resource request as # NOTE(gibi): Checking if the requested port has resource request as
# such ports are currently not supported as they would at least # such ports are only supported if the compute service version is >= 55
# need resource allocation manipulation in placement but might also # TODO(gibi): Remove this check in X as there we can be sure that all
# need a new scheduling if resource on this host is not available. # computes are new enough
if port_id: if port_id:
port = self.network_api.show_port(context, port_id) port = self.network_api.show_port(context, port_id)
if port['port'].get(constants.RESOURCE_REQUEST): if port['port'].get(constants.RESOURCE_REQUEST):
raise exception.AttachInterfaceWithQoSPolicyNotSupported( svc = objects.Service.get_by_host_and_binary(
instance_uuid=instance.uuid) context, instance.host, 'nova-compute')
if svc.version < 55:
raise exception.AttachInterfaceWithQoSPolicyNotSupported(
instance_uuid=instance.uuid)
return self.compute_rpcapi.attach_interface(context, return self.compute_rpcapi.attach_interface(context,
instance=instance, network_id=network_id, port_id=port_id, instance=instance, network_id=network_id, port_id=port_id,

View File

@ -7569,6 +7569,96 @@ class ComputeManager(manager.Manager):
return device return device
def _allocate_port_resource_for_instance(
self,
context: nova.context.RequestContext,
instance: 'objects.Instance',
pci_reqs: 'objects.InstancePCIRequests',
request_groups: ty.List['objects.RequestGroup'],
) -> ty.Tuple[ty.Optional[ty.Dict[str, ty.List[str]]],
ty.Optional[ty.Dict[str, ty.Dict[str, ty.Dict[str, int]]]]]:
"""Allocate resources for the request in placement
:param context: nova.context.RequestContext
:param instance: the objects.Instance to where the interface is being
attached
:param pci_reqs: A list of InstancePCIRequest objects describing the
needed PCI devices
:param request_groups: A list of RequestGroup objects describing the
resources the port requests from placement
:raises InterfaceAttachResourceAllocationFailed: if we failed to
allocate resource in placement for the request
:returns: A tuple of provider mappings and allocated resources or
(None, None) if no resource allocation was needed for the request
"""
if not request_groups:
return None, None
request_group = request_groups[0]
# restrict the resource request to the current compute node. The
# compute node uuid is the uuid of the root provider of the node in
# placement
compute_node_uuid = objects.ComputeNode.get_by_nodename(
context, instance.node).uuid
request_group.in_tree = compute_node_uuid
# NOTE(gibi): when support is added for attaching a cyborg based
# smart NIC the ResourceRequest could be extended to handle multiple
# request groups.
rr = scheduler_utils.ResourceRequest.from_request_group(request_group)
res = self.reportclient.get_allocation_candidates(context, rr)
alloc_reqs, provider_sums, version = res
if not alloc_reqs:
# no allocation candidates available, we run out of free resources
raise exception.InterfaceAttachResourceAllocationFailed(
instance_uuid=instance.uuid)
# select one of the candidates and update the instance
# allocation
# TODO(gibi): We could loop over all possible candidates
# if the first one selected here does not work due to race or due
# to not having free PCI devices. However the latter is only
# detected later in the interface attach code path.
alloc_req = alloc_reqs[0]
resources = alloc_req['allocations']
provider_mappings = alloc_req['mappings']
try:
self.reportclient.add_resources_to_instance_allocation(
context, instance.uuid, resources)
except exception.AllocationUpdateFailed as e:
# We lost a race. We could retry another candidate
raise exception.InterfaceAttachResourceAllocationFailed(
instance_uuid=instance.uuid) from e
except (
exception.ConsumerAllocationRetrievalFailed,
keystone_exception.ClientException,
) as e:
# These are non-recoverable errors so we should not retry
raise exception.InterfaceAttachResourceAllocationFailed(
instance_uuid=instance.uuid) from e
try:
update = (
compute_utils.
update_pci_request_spec_with_allocated_interface_name)
update(
context, self.reportclient, pci_reqs.requests,
provider_mappings)
except (
exception.AmbiguousResourceProviderForPCIRequest,
exception.UnexpectedResourceProviderNameForPCIRequest
):
# These are programing errors. So we clean up an re-raise to let
# the request fail
with excutils.save_and_reraise_exception():
self.reportclient.remove_resources_from_instance_allocation(
context, instance.uuid, resources)
return provider_mappings, resources
# TODO(mriedem): There are likely race failures which can result in # TODO(mriedem): There are likely race failures which can result in
# NotFound and QuotaError exceptions getting traced as well. # NotFound and QuotaError exceptions getting traced as well.
@messaging.expected_exceptions( @messaging.expected_exceptions(
@ -7582,6 +7672,7 @@ class ComputeManager(manager.Manager):
# PortNotUsableDNS # PortNotUsableDNS
# AttachSRIOVPortNotSupported # AttachSRIOVPortNotSupported
# NetworksWithQoSPolicyNotSupported # NetworksWithQoSPolicyNotSupported
# InterfaceAttachResourceAllocationFailed
exception.Invalid) exception.Invalid)
@wrap_exception() @wrap_exception()
@wrap_instance_event(prefix='compute') @wrap_instance_event(prefix='compute')
@ -7639,31 +7730,39 @@ class ComputeManager(manager.Manager):
instance.flavor, instance.image_meta) instance.flavor, instance.image_meta)
pci_reqs = objects.InstancePCIRequests( pci_reqs = objects.InstancePCIRequests(
requests=[], instance_uuid=instance.uuid) requests=[], instance_uuid=instance.uuid)
self.network_api.create_resource_requests( _, request_groups = self.network_api.create_resource_requests(
context, requested_networks, pci_reqs, context, requested_networks, pci_reqs,
affinity_policy=pci_numa_affinity_policy) affinity_policy=pci_numa_affinity_policy)
# We only support one port per attach request so we at most have one # We only support one port per attach request so we at most have one
# pci request # pci request
pci_req = None
if pci_reqs.requests: if pci_reqs.requests:
pci_req = pci_reqs.requests[0] pci_req = pci_reqs.requests[0]
requested_networks[0].pci_request_id = pci_req.request_id requested_networks[0].pci_request_id = pci_req.request_id
instance.pci_requests.requests.append(pci_req)
result = self._allocate_port_resource_for_instance(
context, instance, pci_reqs, request_groups)
provider_mappings, resources = result
try: try:
pci_device = self._claim_pci_device_for_interface_attach( pci_device = self._claim_pci_device_for_interface_attach(
context, instance, pci_reqs) context, instance, pci_reqs)
except exception.InterfaceAttachPciClaimFailed: except exception.InterfaceAttachPciClaimFailed:
with excutils.save_and_reraise_exception(): with excutils.save_and_reraise_exception():
if pci_req: if resources:
instance.pci_requests.requests.remove(pci_req) # TODO(gibi): Instead of giving up we could try another
# allocation candidate from _allocate_resources() if any
self._deallocate_port_resource_for_instance(
context, instance, port_id, resources)
instance.pci_requests.requests.extend(pci_reqs.requests)
network_info = self.network_api.allocate_for_instance( network_info = self.network_api.allocate_for_instance(
context, context,
instance, instance,
requested_networks, requested_networks,
bind_host_id=bind_host_id, bind_host_id=bind_host_id,
resource_provider_mapping=provider_mappings,
) )
if len(network_info) != 1: if len(network_info) != 1:

View File

@ -1339,6 +1339,11 @@ class InterfaceAttachPciClaimFailed(Invalid):
"interface attach") "interface attach")
class InterfaceAttachResourceAllocationFailed(Invalid):
msg_fmt = _("Failed to allocate additional resources to %(instance_uuid)s "
"during interface attach")
class InterfaceDetachFailed(Invalid): class InterfaceDetachFailed(Invalid):
msg_fmt = _("Failed to detach network adapter device from " msg_fmt = _("Failed to detach network adapter device from "
"%(instance_uuid)s") "%(instance_uuid)s")

View File

@ -31,7 +31,7 @@ LOG = logging.getLogger(__name__)
# NOTE(danms): This is the global service version counter # NOTE(danms): This is the global service version counter
SERVICE_VERSION = 54 SERVICE_VERSION = 55
# NOTE(danms): This is our SERVICE_VERSION history. The idea is that any # NOTE(danms): This is our SERVICE_VERSION history. The idea is that any
@ -194,6 +194,9 @@ SERVICE_VERSION_HISTORY = (
# Add accel_uuids (accelerator requests) param to shelve_instance and # Add accel_uuids (accelerator requests) param to shelve_instance and
# shelve_offload_instance and unshelve_instance # shelve_offload_instance and unshelve_instance
{'compute_rpc': '5.13'}, {'compute_rpc': '5.13'},
# Version 55: Compute RPC v5.13:
# Add support for qos interface attach
{'compute_rpc': '5.13'},
) )
# This is used to raise an error at service startup if older than N-1 computes # This is used to raise an error at service startup if older than N-1 computes

View File

@ -192,6 +192,17 @@ class ResourceRequest(object):
return res_req return res_req
@classmethod
def from_request_group(
cls,
request_group: 'objects.RequestGroup',
) -> 'ResourceRequest':
"""Create a new instance of ResourceRequest from a RequestGroup."""
res_req = cls()
res_req._add_request_group(request_group)
res_req.strip_zeros()
return res_req
def _process_requested_resources(self, request_spec): def _process_requested_resources(self, request_spec):
requested_resources = (request_spec.requested_resources requested_resources = (request_spec.requested_resources
if 'requested_resources' in request_spec and if 'requested_resources' in request_spec and

View File

@ -5898,21 +5898,34 @@ class UnsupportedPortResourceRequestBasedSchedulingTest(
self.neutron.port_with_resource_request[ self.neutron.port_with_resource_request[
constants.RESOURCE_REQUEST]) constants.RESOURCE_REQUEST])
def test_interface_attach_with_port_resource_request(self): def test_interface_attach_with_resource_request_old_compute(self):
# create a server # create a server
server = self._create_server( server = self._create_server(
flavor=self.flavor, flavor=self.flavor,
networks=[{'port': self.neutron.port_1['id']}]) networks=[{'port': self.neutron.port_1['id']}])
self._wait_for_state_change(server, 'ACTIVE') self._wait_for_state_change(server, 'ACTIVE')
# try to add a port with resource request # simulate that the compute the instance is running on is older than
post = { # when support is added for attach, older than service version 55
'interfaceAttachment': { orig_get_service = objects.Service.get_by_host_and_binary
'port_id': self.neutron.port_with_resource_request['id']
}} def fake_get_service(context, host, binary):
ex = self.assertRaises(client.OpenStackApiException, service = orig_get_service(context, host, binary)
self.api.attach_interface, service.version = 54
server['id'], post) return service
with mock.patch(
'nova.objects.Service.get_by_host_and_binary',
side_effect=fake_get_service
):
# try to add a port with resource request
post = {
'interfaceAttachment': {
'port_id': self.neutron.port_with_resource_request['id']
}}
ex = self.assertRaises(
client.OpenStackApiException, self.api.attach_interface,
server['id'], post)
self.assertEqual(400, ex.response.status_code) self.assertEqual(400, ex.response.status_code)
self.assertIn('Attaching interfaces with QoS policy is ' self.assertIn('Attaching interfaces with QoS policy is '
'not supported for instance', 'not supported for instance',
@ -6131,6 +6144,301 @@ class PortResourceRequestBasedSchedulingTest(
self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2], self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2],
sriov_binding['allocation']) sriov_binding['allocation'])
def test_interface_attach_with_resource_request(self):
server = self._create_server(
flavor=self.flavor,
networks=[{'port': self.neutron.port_1['id']}])
self._wait_for_state_change(server, 'ACTIVE')
# start a second compute to show that resources are only allocated from
# the compute the instance currently runs on
self.compute2 = self._start_compute('host2')
self.compute2_rp_uuid = self._get_provider_uuid_by_host('host2')
self._create_networking_rp_tree('host2', self.compute2_rp_uuid)
self.compute2_service_id = self.admin_api.get_services(
host='host2', binary='nova-compute')[0]['id']
# attach an OVS port with resource request
ovs_port = self.neutron.port_with_resource_request
post = {
'interfaceAttachment': {
'port_id': ovs_port['id']
}}
self.api.attach_interface(server['id'], post)
ovs_port = self.neutron.show_port(ovs_port['id'])['port']
allocations = self.placement.get(
'/allocations/%s' % server['id']).body['allocations']
# We expect one set of allocations for the compute resources on the
# compute RP and one set for the networking resources on the OVS
# bridge RP.
self.assertEqual(2, len(allocations))
self.assertComputeAllocationMatchesFlavor(
allocations, self.compute1_rp_uuid, self.flavor)
ovs_allocations = allocations[
self.ovs_bridge_rp_per_host[self.compute1_rp_uuid]]['resources']
self.assertPortMatchesAllocation(ovs_port, ovs_allocations)
# We expect that only the RP uuid of the networking RP having the port
# allocation is sent in the port binding for the port having resource
# request
ovs_binding = ovs_port['binding:profile']
self.assertEqual(self.ovs_bridge_rp_per_host[self.compute1_rp_uuid],
ovs_binding['allocation'])
# now attach an SRIOV port
sriov_port = self.neutron.port_with_sriov_resource_request
post = {
'interfaceAttachment': {
'port_id': sriov_port['id']
}}
self.api.attach_interface(server['id'], post)
ovs_port = self.neutron.show_port(ovs_port['id'])['port']
sriov_port = self.neutron.show_port(sriov_port['id'])['port']
allocations = self.placement.get(
'/allocations/%s' % server['id']).body['allocations']
# We expect one set of allocations for the compute resources on the
# compute RP and one set each for the networking resources on the OVS
# bridge RP and on the SRIOV PF RP.
self.assertEqual(3, len(allocations))
self.assertComputeAllocationMatchesFlavor(
allocations, self.compute1_rp_uuid, self.flavor)
ovs_allocations = allocations[
self.ovs_bridge_rp_per_host[self.compute1_rp_uuid]]['resources']
sriov_allocations = allocations[
self.sriov_dev_rp_per_host[
self.compute1_rp_uuid][self.PF2]]['resources']
self.assertPortMatchesAllocation(ovs_port, ovs_allocations)
self.assertPortMatchesAllocation(sriov_port, sriov_allocations)
# We expect that only the RP uuid of the networking RP having the port
# allocation is sent in the port binding for the port having resource
# request
ovs_binding = ovs_port['binding:profile']
self.assertEqual(self.ovs_bridge_rp_per_host[self.compute1_rp_uuid],
ovs_binding['allocation'])
sriov_binding = sriov_port['binding:profile']
self.assertEqual(
self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2],
sriov_binding['allocation'])
def test_interface_attach_with_resource_request_no_candidates(self):
server = self._create_server(
flavor=self.flavor,
networks=[{'port': self.neutron.port_1['id']}])
self._wait_for_state_change(server, 'ACTIVE')
# attach an OVS port with too big resource request
ovs_port = self.neutron.port_with_resource_request
resources = self.neutron._ports[
ovs_port['id']]['resource_request']['resources']
resources['NET_BW_IGR_KILOBIT_PER_SEC'] = 1000000
post = {
'interfaceAttachment': {
'port_id': ovs_port['id']
}}
ex = self.assertRaises(
client.OpenStackApiException, self.api.attach_interface,
server['id'], post)
self.assertEqual(400, ex.response.status_code)
self.assertIn('Failed to allocate additional resources', str(ex))
def test_interface_attach_with_resource_request_pci_claim_fails(self):
# boot a server with a single SRIOV port that has no resource request
sriov_port = self.neutron.sriov_port
server = self._create_server(
flavor=self.flavor,
networks=[{'port': sriov_port['id']}])
self._wait_for_state_change(server, 'ACTIVE')
sriov_port = self.neutron.show_port(sriov_port['id'])['port']
sriov_binding = sriov_port['binding:profile']
# We expect that this consume the last available VF from the PF2
self.assertEqual(
fake.FakeDriverWithPciResources.PCI_ADDR_PF2_VF1,
sriov_binding['pci_slot'])
# Now attach a second port to this server that has resource request
# At this point PF2 has available bandwidth but no available VF
# and PF3 has available VF but no available bandwidth so we expect
# the attach to fail.
sriov_port_with_res_req = self.neutron.port_with_sriov_resource_request
post = {
'interfaceAttachment': {
'port_id': sriov_port_with_res_req['id']
}}
ex = self.assertRaises(
client.OpenStackApiException, self.api.attach_interface,
server['id'], post)
self.assertEqual(400, ex.response.status_code)
self.assertIn('Failed to claim PCI device', str(ex))
sriov_port_with_res_req = self.neutron.show_port(
sriov_port_with_res_req['id'])['port']
allocations = self.placement.get(
'/allocations/%s' % server['id']).body['allocations']
# We expect only one allocations that is on the compute RP as the
# allocation made towards the PF2 RP has been rolled back when the PCI
# claim failed
self.assertEqual([self.compute1_rp_uuid], list(allocations))
self.assertComputeAllocationMatchesFlavor(
allocations, self.compute1_rp_uuid, self.flavor)
# We expect that the port binding is not updated with any RP uuid as
# the attach failed.
sriov_binding = sriov_port_with_res_req['binding:profile']
self.assertNotIn('allocation', sriov_binding)
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
# when the sriov interface is attached.
rsp = self.placement.put(
'/resource_providers/%s'
% self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2],
{"name": "invalid-device-rp-name"})
self.assertEqual(200, rsp.status)
server = self._create_server(
flavor=self.flavor,
networks=[{'port': self.neutron.port_1['id']}])
self._wait_for_state_change(server, 'ACTIVE')
sriov_port = self.neutron.port_with_sriov_resource_request
post = {
'interfaceAttachment': {
'port_id': sriov_port['id']
}}
ex = self.assertRaises(
client.OpenStackApiException, self.api.attach_interface,
server['id'], post)
self.assertEqual(500, ex.response.status_code)
self.assertIn('UnexpectedResourceProviderNameForPCIRequest', str(ex))
sriov_port = self.neutron.show_port(sriov_port['id'])['port']
allocations = self.placement.get(
'/allocations/%s' % server['id']).body['allocations']
# We expect only one allocations that is on the compute RP as the
# allocation made towards the PF2 RP has been rolled back when the PCI
# update failed
self.assertEqual([self.compute1_rp_uuid], list(allocations))
self.assertComputeAllocationMatchesFlavor(
allocations, self.compute1_rp_uuid, self.flavor)
# We expect that the port binding is not updated with any RP uuid as
# the attach failed.
sriov_binding = sriov_port['binding:profile']
self.assertNotIn('allocation', sriov_binding)
def test_interface_attach_sriov_with_qos_pci_update_fails_cleanup_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
# when the sriov interface is attached.
rsp = self.placement.put(
'/resource_providers/%s'
% self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2],
{"name": "invalid-device-rp-name"})
self.assertEqual(200, rsp.status)
server = self._create_server(
flavor=self.flavor,
networks=[{'port': self.neutron.port_1['id']}])
self._wait_for_state_change(server, 'ACTIVE')
sriov_port = self.neutron.port_with_sriov_resource_request
post = {
'interfaceAttachment': {
'port_id': sriov_port['id']
}}
orig_put = adapter.Adapter.put
conflict_rsp = fake_requests.FakeResponse(
409,
jsonutils.dumps(
{'errors': [
{'code': 'placement.concurrent_update',
'detail': 'consumer generation conflict'}]}))
self.adapter_put_call_count = 0
def fake_put(_self, url, **kwargs):
self.adapter_put_call_count += 1
if self.adapter_put_call_count == 1:
# allocation update to add the port resource request
return orig_put(_self, url, **kwargs)
else:
# cleanup calls to remove the port resource allocation
return conflict_rsp
# this mock makes sure that the placement cleanup will fail with
# conflict
with mock.patch('keystoneauth1.adapter.Adapter.put', new=fake_put):
ex = self.assertRaises(
client.OpenStackApiException, self.api.attach_interface,
server['id'], post)
self.assertEqual(500, ex.response.status_code)
self.assertIn('AllocationUpdateFailed', str(ex))
# we have a proper log about the leak
PF_rp_uuid = self.sriov_dev_rp_per_host[
self.compute1_rp_uuid][self.PF2]
self.assertIn(
"nova.exception.AllocationUpdateFailed: Failed to update "
"allocations for consumer %s. Error: Cannot remove "
"resources {'%s': "
"{'resources': {'NET_BW_EGR_KILOBIT_PER_SEC': 10000, "
"'NET_BW_IGR_KILOBIT_PER_SEC': 10000}}} from the allocation "
"due to multiple successive generation conflicts in "
"placement." % (server['id'], PF_rp_uuid),
self.stdlog.logger.output)
# assert that we retried the cleanup multiple times
self.assertEqual(5, self.adapter_put_call_count)
sriov_port = self.neutron.show_port(sriov_port['id'])['port']
allocations = self.placement.get(
'/allocations/%s' % server['id']).body['allocations']
# As the cleanup failed we leaked allocation in placement
self.assertEqual(2, len(allocations))
self.assertComputeAllocationMatchesFlavor(
allocations, self.compute1_rp_uuid, self.flavor)
sriov_allocations = allocations[
self.sriov_dev_rp_per_host[
self.compute1_rp_uuid][self.PF2]]['resources']
# this is the leaked allocation in placement
self.assertPortMatchesAllocation(sriov_port, sriov_allocations)
# We expect that the port binding is not updated with any RP uuid as
# the attach failed.
sriov_binding = sriov_port['binding:profile']
self.assertNotIn('allocation', sriov_binding)
def test_interface_detach_with_port_with_bandwidth_request(self): def test_interface_detach_with_port_with_bandwidth_request(self):
port = self.neutron.port_with_resource_request port = self.neutron.port_with_resource_request

View File

@ -7094,9 +7094,15 @@ class ComputeAPIUnitTestCase(_ComputeAPIUnitTestMixIn, test.NoDBTestCase):
mock_record.assert_called_once_with( mock_record.assert_called_once_with(
self.context, instance, instance_actions.ATTACH_INTERFACE) self.context, instance, instance_actions.ATTACH_INTERFACE)
@mock.patch('nova.objects.service.Service.get_by_host_and_binary')
@mock.patch('nova.compute.api.API._record_action_start') @mock.patch('nova.compute.api.API._record_action_start')
def test_attach_interface_qos_aware_port(self, mock_record): def test_attach_interface_qos_aware_port_old_compute(
self, mock_record, mock_get_service
):
instance = self._create_instance_obj() instance = self._create_instance_obj()
service = objects.Service()
service.version = 54
mock_get_service.return_value = service
with mock.patch.object( with mock.patch.object(
self.compute_api.network_api, 'show_port', self.compute_api.network_api, 'show_port',
return_value={'port': { return_value={'port': {
@ -7110,6 +7116,42 @@ class ComputeAPIUnitTestCase(_ComputeAPIUnitTestMixIn, test.NoDBTestCase):
'foo_net_id', 'foo_port_id', None 'foo_net_id', 'foo_port_id', None
) )
mock_show_port.assert_called_once_with(self.context, 'foo_port_id') mock_show_port.assert_called_once_with(self.context, 'foo_port_id')
mock_get_service.assert_called_once_with(
self.context, instance.host, 'nova-compute')
@mock.patch('nova.compute.rpcapi.ComputeAPI.attach_interface')
@mock.patch('nova.objects.service.Service.get_by_host_and_binary')
@mock.patch('nova.compute.api.API._record_action_start')
def test_attach_interface_qos_aware_port(
self, mock_record, mock_get_service, mock_attach):
instance = self._create_instance_obj()
service = objects.Service()
service.version = 55
mock_get_service.return_value = service
with mock.patch.object(
self.compute_api.network_api,
"show_port",
return_value={
"port": {
constants.RESOURCE_REQUEST: {
"resources": {"CUSTOM_RESOURCE_CLASS": 42}
}
}
},
) as mock_show_port:
self.compute_api.attach_interface(
self.context, instance, mock.sentinel.net_id,
mock.sentinel.port_id, mock.sentinel.ip, mock.sentinel.tag)
mock_show_port.assert_called_once_with(
self.context, mock.sentinel.port_id)
mock_get_service.assert_called_once_with(
self.context, instance.host, 'nova-compute')
mock_attach.assert_called_once_with(
self.context, instance=instance, network_id=mock.sentinel.net_id,
port_id=mock.sentinel.port_id, requested_ip=mock.sentinel.ip,
tag=mock.sentinel.tag)
@mock.patch('nova.compute.api.API._record_action_start') @mock.patch('nova.compute.api.API._record_action_start')
@mock.patch.object(compute_rpcapi.ComputeAPI, 'detach_interface') @mock.patch.object(compute_rpcapi.ComputeAPI, 'detach_interface')

View File

@ -65,6 +65,7 @@ from nova.objects import instance as instance_obj
from nova.objects import migrate_data as migrate_data_obj from nova.objects import migrate_data as migrate_data_obj
from nova.policies import base as base_policy from nova.policies import base as base_policy
from nova.policies import servers as servers_policy from nova.policies import servers as servers_policy
from nova.scheduler import utils as scheduler_utils
from nova import test from nova import test
from nova.tests import fixtures from nova.tests import fixtures
from nova.tests.unit.compute import eventlet_utils from nova.tests.unit.compute import eventlet_utils
@ -10151,6 +10152,7 @@ class ComputeAPITestCase(BaseTestCase):
"_claim_pci_device_for_interface_attach", "_claim_pci_device_for_interface_attach",
return_value=None) return_value=None)
) as (cap, mock_lock, mock_create_resource_req, mock_claim_pci): ) as (cap, mock_lock, mock_create_resource_req, mock_claim_pci):
mock_create_resource_req.return_value = (None, [])
vif = self.compute.attach_interface(self.context, vif = self.compute.attach_interface(self.context,
instance, instance,
network_id, network_id,
@ -10160,7 +10162,7 @@ class ComputeAPITestCase(BaseTestCase):
mock_allocate.assert_called_once_with( mock_allocate.assert_called_once_with(
self.context, instance, self.context, instance,
test.MatchType(objects.NetworkRequestList), test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-mini') bind_host_id='fake-mini', resource_provider_mapping=None)
network_requests = mock_allocate.mock_calls[0][1][2] network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects)) self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0] network_request = network_requests[0]
@ -10184,6 +10186,79 @@ class ComputeAPITestCase(BaseTestCase):
return nwinfo, port_id return nwinfo, port_id
@mock.patch.object(compute_utils, 'notify_about_instance_action')
def test_attach_interface_with_qos(self, mock_notify):
instance = self._create_fake_instance_obj()
nwinfo = [fake_network_cache_model.new_vif()]
network_id = nwinfo[0]['network']['id']
port_id = nwinfo[0]['id']
req_ip = '1.2.3.4'
lock_name = 'interface-%s-%s' % (instance.uuid, port_id)
mock_allocate = mock.Mock(return_value=nwinfo)
self.compute.network_api.allocate_for_instance = mock_allocate
with test.nested(
mock.patch.dict(self.compute.driver.capabilities,
supports_attach_interface=True),
mock.patch('oslo_concurrency.lockutils.lock'),
mock.patch("nova.network.neutron.API.create_resource_requests"),
mock.patch.object(
self.compute,
"_claim_pci_device_for_interface_attach",
return_value=None),
mock.patch.object(
self.compute, '_allocate_port_resource_for_instance'),
) as (cap, mock_lock, mock_create_resource_req, mock_claim_pci,
mock_allocate_res
):
request_groups = [objects.RequestGroup]
mock_create_resource_req.return_value = (None, request_groups)
mock_allocate_res.return_value = (
mock.sentinel.provider_mappings, mock.sentinel.resources)
vif = self.compute.attach_interface(
self.context, instance, network_id, port_id, req_ip, None)
# check that the vif created based on the port we are attaching
self.assertEqual(vif['id'], port_id)
# ensure that we are passing the proper network request including the
# provider mapping to the neutron code path
mock_allocate.assert_called_once_with(
self.context, instance,
test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-mini',
resource_provider_mapping=mock.sentinel.provider_mappings)
network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0]
self.assertEqual(network_id, network_request.network_id)
self.assertEqual(port_id, network_request.port_id)
self.assertEqual(req_ip, str(network_request.address))
mock_notify.assert_has_calls([
mock.call(self.context, instance, self.compute.host,
action='interface_attach', phase='start'),
mock.call(self.context, instance, self.compute.host,
action='interface_attach', phase='end')])
mock_lock.assert_called_once_with(lock_name, mock.ANY, mock.ANY,
mock.ANY, delay=mock.ANY, do_log=mock.ANY, fair=mock.ANY,
semaphores=mock.ANY)
# as this is an OVS port we don't call the pci claim but with an empty
# request
mock_claim_pci.assert_called_once_with(
self.context, instance,
test.MatchType(objects.InstancePCIRequests))
pci_reqs = mock_claim_pci.mock_calls[0][1][2]
self.assertEqual(instance.uuid, pci_reqs.instance_uuid)
self.assertEqual([], pci_reqs.requests)
# as this port has resource request we need to call
# _allocate_port_resource_for_instance for it
mock_allocate_res.assert_called_once_with(
self.context, instance, pci_reqs, request_groups)
@mock.patch.object(compute_utils, 'notify_about_instance_action') @mock.patch.object(compute_utils, 'notify_about_instance_action')
def test_attach_sriov_interface(self, mock_notify): def test_attach_sriov_interface(self, mock_notify):
instance = self._create_fake_instance_obj() instance = self._create_fake_instance_obj()
@ -10218,6 +10293,8 @@ class ComputeAPITestCase(BaseTestCase):
pci_requests=None, affinity_policy=None): pci_requests=None, affinity_policy=None):
# Simulate that the requested port is an SRIOV port # Simulate that the requested port is an SRIOV port
pci_requests.requests.append(pci_req) pci_requests.requests.append(pci_req)
# without resource request
return None, []
mock_create_resource_req.side_effect = create_resource_req mock_create_resource_req.side_effect = create_resource_req
@ -10228,7 +10305,7 @@ class ComputeAPITestCase(BaseTestCase):
mock_allocate.assert_called_once_with( mock_allocate.assert_called_once_with(
self.context, instance, self.context, instance,
test.MatchType(objects.NetworkRequestList), test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-mini') bind_host_id='fake-mini', resource_provider_mapping=None)
network_requests = mock_allocate.mock_calls[0][1][2] network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects)) self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0] network_request = network_requests[0]
@ -10254,6 +10331,100 @@ class ComputeAPITestCase(BaseTestCase):
return nwinfo, port_id return nwinfo, port_id
@mock.patch.object(compute_utils, 'notify_about_instance_action')
def test_attach_sriov_interface_with_qos(self, mock_notify):
instance = self._create_fake_instance_obj()
instance.pci_requests = objects.InstancePCIRequests(requests=[])
instance.pci_devices = objects.PciDeviceList(objects=[])
instance.numa_topology = objects.InstanceNUMATopology()
nwinfo = [fake_network_cache_model.new_vif()]
network_id = nwinfo[0]['network']['id']
port_id = nwinfo[0]['id']
req_ip = '1.2.3.4'
mock_allocate = mock.Mock(return_value=nwinfo)
self.compute.network_api.allocate_for_instance = mock_allocate
with test.nested(
mock.patch.dict(self.compute.driver.capabilities,
supports_attach_interface=True),
mock.patch.object(self.compute.network_api,
'create_resource_requests'),
mock.patch.object(self.compute.rt, 'claim_pci_devices'),
mock.patch.object(self.compute.rt,
'allocate_pci_devices_for_instance'),
mock.patch.object(instance, 'save'),
mock.patch.object(
self.compute, '_allocate_port_resource_for_instance')
) as (mock_capabilities, mock_create_resource_req, mock_claim_pci,
mock_allocate_pci, mock_save, mock_allocate_res):
pci_req = objects.InstancePCIRequest(request_id=uuids.pci_req)
pci_device = objects.PciDevice(request_id=pci_req.request_id)
mock_claim_pci.return_value = [pci_device]
request_groups = [objects.RequestGroup()]
def create_resource_req(context, requested_networks,
pci_requests=None, affinity_policy=None):
# Simulate that the requested port is an SRIOV port
pci_requests.requests.append(pci_req)
# with resource request
return None, request_groups
mock_create_resource_req.side_effect = create_resource_req
mock_allocate_res.return_value = (
mock.sentinel.provider_mappings, mock.sentinel.resources)
vif = self.compute.attach_interface(
self.context, instance, network_id, port_id, req_ip, None)
self.assertEqual(vif['id'], port_id)
# ensure that we are passing the proper network request including the
# provider mapping to the neutron code path
mock_allocate.assert_called_once_with(
self.context, instance,
test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-mini',
resource_provider_mapping=mock.sentinel.provider_mappings)
network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0]
self.assertEqual(network_id, network_request.network_id)
self.assertEqual(port_id, network_request.port_id)
self.assertEqual(req_ip, str(network_request.address))
# ensure we gathered the resource request from neutron
mock_create_resource_req.assert_called_once_with(
self.context, network_requests,
test.MatchType(objects.InstancePCIRequests),
affinity_policy=None)
# this is an SR-IOV port so we need to call pci claim with a
# non empty PCI request
mock_claim_pci.assert_called_once_with(
self.context, test.MatchType(objects.InstancePCIRequests),
instance.numa_topology)
pci_reqs = mock_claim_pci.mock_calls[0][1][1]
self.assertEqual([pci_req], pci_reqs.requests)
# after the pci claim we also need to allocate that pci to the instace
mock_allocate_pci.assert_called_once_with(self.context, instance)
# and as this changes the instance we have to save it.
mock_save.assert_called_once_with()
# both the pci request and the pci device should be up to date in the
# instance after the allocation
self.assertIn(pci_req, instance.pci_requests.requests)
self.assertIn(pci_device, instance.pci_devices.objects)
# ensure that we called _allocate_port_resource_for_instance as it has
# resource reques
mock_allocate_res.assert_called_once_with(
self.context, instance, pci_reqs, request_groups)
@mock.patch.object(compute_utils, 'notify_about_instance_action') @mock.patch.object(compute_utils, 'notify_about_instance_action')
def test_interface_tagged_attach(self, mock_notify): def test_interface_tagged_attach(self, mock_notify):
instance = self._create_fake_instance_obj() instance = self._create_fake_instance_obj()
@ -10273,6 +10444,7 @@ class ComputeAPITestCase(BaseTestCase):
'_claim_pci_device_for_interface_attach', '_claim_pci_device_for_interface_attach',
return_value=None) return_value=None)
) as (mock_capabilities, mock_create_resource_req, mock_claim_pci): ) as (mock_capabilities, mock_create_resource_req, mock_claim_pci):
mock_create_resource_req.return_value = (None, [])
vif = self.compute.attach_interface(self.context, vif = self.compute.attach_interface(self.context,
instance, instance,
network_id, network_id,
@ -10282,7 +10454,7 @@ class ComputeAPITestCase(BaseTestCase):
mock_allocate.assert_called_once_with( mock_allocate.assert_called_once_with(
self.context, instance, self.context, instance,
test.MatchType(objects.NetworkRequestList), test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-mini') bind_host_id='fake-mini', resource_provider_mapping=None)
network_requests = mock_allocate.mock_calls[0][1][2] network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects)) self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0] network_request = network_requests[0]
@ -10330,6 +10502,7 @@ class ComputeAPITestCase(BaseTestCase):
network_id = nwinfo[0]['network']['id'] network_id = nwinfo[0]['network']['id']
port_id = nwinfo[0]['id'] port_id = nwinfo[0]['id']
req_ip = '1.2.3.4' req_ip = '1.2.3.4'
instance.pci_requests = objects.InstancePCIRequests(requests=[])
with test.nested( with test.nested(
mock.patch.object(compute_utils, 'notify_about_instance_action'), mock.patch.object(compute_utils, 'notify_about_instance_action'),
@ -10347,6 +10520,7 @@ class ComputeAPITestCase(BaseTestCase):
) as (mock_notify, mock_attach, mock_allocate, mock_deallocate, ) as (mock_notify, mock_attach, mock_allocate, mock_deallocate,
mock_dict, mock_create_resource_req, mock_claim_pci): mock_dict, mock_create_resource_req, mock_claim_pci):
mock_create_resource_req.return_value = (None, [])
mock_allocate.return_value = nwinfo mock_allocate.return_value = nwinfo
mock_attach.side_effect = exception.NovaException("attach_failed") mock_attach.side_effect = exception.NovaException("attach_failed")
self.assertRaises(exception.InterfaceAttachFailed, self.assertRaises(exception.InterfaceAttachFailed,
@ -10355,7 +10529,7 @@ class ComputeAPITestCase(BaseTestCase):
mock_allocate.assert_called_once_with( mock_allocate.assert_called_once_with(
self.context, instance, self.context, instance,
test.MatchType(objects.NetworkRequestList), test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-host') bind_host_id='fake-host', resource_provider_mapping=None)
network_requests = mock_allocate.mock_calls[0][1][2] network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects)) self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0] network_request = network_requests[0]
@ -10424,6 +10598,7 @@ class ComputeAPITestCase(BaseTestCase):
pci_requests=None, affinity_policy=None): pci_requests=None, affinity_policy=None):
# Simulate that the requested port is an SRIOV port # Simulate that the requested port is an SRIOV port
pci_requests.requests.append(pci_req) pci_requests.requests.append(pci_req)
return None, []
mock_create_resource_req.side_effect = create_resource_req mock_create_resource_req.side_effect = create_resource_req
mock_allocate.return_value = nwinfo mock_allocate.return_value = nwinfo
@ -10436,7 +10611,7 @@ class ComputeAPITestCase(BaseTestCase):
mock_allocate.assert_called_once_with( mock_allocate.assert_called_once_with(
self.context, instance, self.context, instance,
test.MatchType(objects.NetworkRequestList), test.MatchType(objects.NetworkRequestList),
bind_host_id='fake-host') bind_host_id='fake-host', resource_provider_mapping=None)
network_requests = mock_allocate.mock_calls[0][1][2] network_requests = mock_allocate.mock_calls[0][1][2]
self.assertEqual(1, len(network_requests.objects)) self.assertEqual(1, len(network_requests.objects))
network_request = network_requests[0] network_request = network_requests[0]
@ -10480,17 +10655,29 @@ class ComputeAPITestCase(BaseTestCase):
'create_resource_requests'), 'create_resource_requests'),
mock.patch.object(self.compute.rt, 'claim_pci_devices', mock.patch.object(self.compute.rt, 'claim_pci_devices',
return_value=[]), return_value=[]),
mock.patch.object(
self.compute, '_allocate_port_resource_for_instance'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'remove_resources_from_instance_allocation')
) as ( ) as (
mock_capabilities, mock_create_resource_req, mock_claim_pci): mock_capabilities, mock_create_resource_req, mock_claim_pci,
mock_allocate_res, mock_remove_res
):
pci_req = objects.InstancePCIRequest(request_id=uuids.pci_req) pci_req = objects.InstancePCIRequest(request_id=uuids.pci_req)
request_groups = [objects.RequestGroup()]
def create_resource_req(context, requested_networks, def create_resource_req(context, requested_networks,
pci_requests=None, affinity_policy=None): pci_requests=None, affinity_policy=None):
# Simulate that the requested port is an SRIOV port # Simulate that the requested port is an SRIOV port
pci_requests.requests.append(pci_req) pci_requests.requests.append(pci_req)
return None, request_groups
mock_create_resource_req.side_effect = create_resource_req mock_create_resource_req.side_effect = create_resource_req
mock_allocate_res.return_value = (
mock.sentinel.provider_mappings, mock.sentinel.resources)
ex = self.assertRaises( ex = self.assertRaises(
messaging.ExpectedException, self.compute.attach_interface, messaging.ExpectedException, self.compute.attach_interface,
@ -10510,6 +10697,228 @@ class ComputeAPITestCase(BaseTestCase):
self.assertNotIn(pci_req, instance.pci_requests.requests) self.assertNotIn(pci_req, instance.pci_requests.requests)
mock_allocate_res.assert_called_once_with(
self.context, instance, pci_reqs, request_groups)
mock_remove_res.assert_called_once_with(
self.context, instance.uuid, mock.sentinel.resources)
def test__allocate_port_resource_for_instance(self):
instance = self._create_fake_instance_obj()
pci_reqs = objects.InstancePCIRequests(requests=[])
request_groups = [
objects.RequestGroup(
resources={"CUSTOM_FOO": 13},
requester_id=uuids.requester_id)
]
with test.nested(
mock.patch.object(objects.ComputeNode, 'get_by_nodename'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'get_allocation_candidates'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name'),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
mock_update_pci
):
mock_get_nodename.return_value = objects.ComputeNode(
uuid=uuids.compute_node)
alloc_reqs = [
{
'allocations': mock.sentinel.resources,
'mappings': mock.sentinel.provider_mappings,
}
]
mock_get_alloc_candidates.return_value = (
alloc_reqs, mock.sentinel.provider_sums, mock.sentinel.version)
res = self.compute._allocate_port_resource_for_instance(
self.context, instance, pci_reqs, request_groups)
provider_mappings, resources = res
self.assertEqual(
mock.sentinel.provider_mappings, provider_mappings)
self.assertEqual(
mock.sentinel.resources, resources)
mock_get_nodename.assert_called_once_with(
self.context, instance.node)
mock_get_alloc_candidates.assert_called_once_with(
self.context, test.MatchType(scheduler_utils.ResourceRequest))
resource_request = mock_get_alloc_candidates.mock_calls[0][1][1]
actual_rg = resource_request.get_request_group(
request_groups[0].requester_id)
self.assertEqual(request_groups[0], actual_rg)
self.assertEqual(uuids.compute_node, actual_rg.in_tree)
mock_add_res.assert_called_once_with(
self.context, instance.uuid, mock.sentinel.resources)
mock_update_pci.assert_called_once_with(
self.context, self.compute.reportclient,
pci_reqs.requests, mock.sentinel.provider_mappings)
def test__allocate_port_resource_for_instance_no_candidate(self):
instance = self._create_fake_instance_obj()
pci_reqs = objects.InstancePCIRequests(requests=[])
request_groups = [
objects.RequestGroup(
resources={"CUSTOM_FOO": 13},
requester_id=uuids.requester_id)
]
with test.nested(
mock.patch.object(objects.ComputeNode, 'get_by_nodename'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'get_allocation_candidates'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'add_resources_to_instance_allocation',
new=mock.NonCallableMock()),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name',
new=mock.NonCallableMock()),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
mock_update_pci
):
mock_get_nodename.return_value = objects.ComputeNode(
uuid=uuids.compute_node)
mock_get_alloc_candidates.return_value = (None, None, None)
self.assertRaises(
exception.InterfaceAttachResourceAllocationFailed,
self.compute._allocate_port_resource_for_instance,
self.context, instance, pci_reqs, request_groups)
mock_get_nodename.assert_called_once_with(
self.context, instance.node)
mock_get_alloc_candidates.assert_called_once_with(
self.context, test.MatchType(scheduler_utils.ResourceRequest))
def test__allocate_port_resource_for_instance_fails_to_extend_alloc(self):
instance = self._create_fake_instance_obj()
pci_reqs = objects.InstancePCIRequests(requests=[])
request_groups = [
objects.RequestGroup(
resources={"CUSTOM_FOO": 13},
requester_id=uuids.requester_id)
]
with test.nested(
mock.patch.object(objects.ComputeNode, 'get_by_nodename'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'get_allocation_candidates'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name',
new=mock.NonCallableMock),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
mock_update_pci
):
mock_get_nodename.return_value = objects.ComputeNode(
uuid=uuids.compute_node)
alloc_reqs = [
{
'allocations': mock.sentinel.resources,
'mappings': mock.sentinel.provider_mappings,
}
]
mock_get_alloc_candidates.return_value = (
alloc_reqs, mock.sentinel.provider_sums, mock.sentinel.version)
mock_add_res.side_effect = exception.AllocationUpdateFailed(
consumer_uuid=instance.uuid, error='test')
self.assertRaises(
exception.InterfaceAttachResourceAllocationFailed,
self.compute._allocate_port_resource_for_instance,
self.context, instance, pci_reqs, request_groups)
mock_get_nodename.assert_called_once_with(
self.context, instance.node)
mock_get_alloc_candidates.assert_called_once_with(
self.context, test.MatchType(scheduler_utils.ResourceRequest))
resource_request = mock_get_alloc_candidates.mock_calls[0][1][1]
actual_rg = resource_request.get_request_group(
request_groups[0].requester_id)
self.assertEqual(request_groups[0], actual_rg)
self.assertEqual(uuids.compute_node, actual_rg.in_tree)
mock_add_res.assert_called_once_with(
self.context, instance.uuid, mock.sentinel.resources)
def test__allocate_port_resource_for_instance_fails_to_update_pci(self):
instance = self._create_fake_instance_obj()
pci_reqs = objects.InstancePCIRequests(requests=[])
request_groups = [
objects.RequestGroup(
resources={"CUSTOM_FOO": 13},
requester_id=uuids.requester_id)
]
with test.nested(
mock.patch.object(objects.ComputeNode, 'get_by_nodename'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'get_allocation_candidates'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'add_resources_to_instance_allocation'),
mock.patch(
'nova.compute.utils.'
'update_pci_request_spec_with_allocated_interface_name'),
mock.patch(
'nova.scheduler.client.report.SchedulerReportClient.'
'remove_resources_from_instance_allocation'),
) as (
mock_get_nodename, mock_get_alloc_candidates, mock_add_res,
mock_update_pci, mock_remove_res
):
mock_get_nodename.return_value = objects.ComputeNode(
uuid=uuids.compute_node)
alloc_reqs = [
{
'allocations': mock.sentinel.resources,
'mappings': mock.sentinel.provider_mappings,
}
]
mock_get_alloc_candidates.return_value = (
alloc_reqs, mock.sentinel.provider_sums, mock.sentinel.version)
mock_update_pci.side_effect = (
exception.AmbiguousResourceProviderForPCIRequest(
providers=[], requester="requester"))
self.assertRaises(
exception.AmbiguousResourceProviderForPCIRequest,
self.compute._allocate_port_resource_for_instance,
self.context, instance, pci_reqs, request_groups)
mock_get_nodename.assert_called_once_with(
self.context, instance.node)
mock_get_alloc_candidates.assert_called_once_with(
self.context, test.MatchType(scheduler_utils.ResourceRequest))
resource_request = mock_get_alloc_candidates.mock_calls[0][1][1]
actual_rg = resource_request.get_request_group(
request_groups[0].requester_id)
self.assertEqual(request_groups[0], actual_rg)
self.assertEqual(uuids.compute_node, actual_rg.in_tree)
mock_add_res.assert_called_once_with(
self.context, instance.uuid, mock.sentinel.resources)
mock_update_pci.assert_called_once_with(
self.context, self.compute.reportclient,
pci_reqs.requests, mock.sentinel.provider_mappings)
mock_remove_res.assert_called_once_with(
self.context, instance.uuid, mock.sentinel.resources)
@mock.patch.object(compute_utils, 'notify_about_instance_action') @mock.patch.object(compute_utils, 'notify_about_instance_action')
def test_detach_interface(self, mock_notify): def test_detach_interface(self, mock_notify):
nwinfo, port_id = self.test_attach_interface() nwinfo, port_id = self.test_attach_interface()

View File

@ -2545,6 +2545,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase,
db_instance) db_instance)
f_instance.flavor = objects.Flavor() f_instance.flavor = objects.Flavor()
f_instance.system_metadata = {} f_instance.system_metadata = {}
f_instance.pci_requests = objects.InstancePCIRequests(requests=[])
e = exception.InterfaceAttachFailed(instance_uuid=f_instance.uuid) e = exception.InterfaceAttachFailed(instance_uuid=f_instance.uuid)
@mock.patch('nova.network.neutron.API.create_resource_requests') @mock.patch('nova.network.neutron.API.create_resource_requests')
@ -2561,6 +2562,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase,
def do_test( def do_test(
update, meth, add_fault, notify, event, mock_claim_pci, update, meth, add_fault, notify, event, mock_claim_pci,
mock_create_resource_req): mock_create_resource_req):
mock_create_resource_req.return_value = None, []
self.assertRaises(exception.InterfaceAttachFailed, self.assertRaises(exception.InterfaceAttachFailed,
self.compute.attach_interface, self.compute.attach_interface,
self.context, f_instance, uuids.network_id, self.context, f_instance, uuids.network_id,

View File

@ -1274,6 +1274,31 @@ class TestUtils(TestUtilsBase):
rr = utils.ResourceRequest.from_request_spec(rs) rr = utils.ResourceRequest.from_request_spec(rs)
self.assertResourceRequestsEqual(expected, rr) self.assertResourceRequestsEqual(expected, rr)
def test_resource_request_from_request_group(self):
rg = objects.RequestGroup.from_port_request(
self.context,
uuids.port_id,
port_resource_request={
"resources": {
"NET_BW_IGR_KILOBIT_PER_SEC": 1000,
"NET_BW_EGR_KILOBIT_PER_SEC": 1000},
"required": ["CUSTOM_PHYSNET_2",
"CUSTOM_VNIC_TYPE_NORMAL"]
}
)
rr = utils.ResourceRequest.from_request_group(rg)
self.assertEqual(
f'limit=1000&'
f'required{uuids.port_id}='
f'CUSTOM_PHYSNET_2%2C'
f'CUSTOM_VNIC_TYPE_NORMAL&'
f'resources{uuids.port_id}='
f'NET_BW_EGR_KILOBIT_PER_SEC%3A1000%2C'
f'NET_BW_IGR_KILOBIT_PER_SEC%3A1000',
rr.to_querystring())
def test_resource_request_add_group_inserts_the_group(self): def test_resource_request_add_group_inserts_the_group(self):
flavor = objects.Flavor( flavor = objects.Flavor(
vcpus=1, memory_mb=1024, root_gb=10, ephemeral_gb=5, swap=0) vcpus=1, memory_mb=1024, root_gb=10, ephemeral_gb=5, swap=0)

View File

@ -0,0 +1,5 @@
---
features:
- |
The ``POST /servers/{server_id}/os-interface`` API now supports
attaching neutron ports with QoS minimum bandwidth rules attached.