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:
parent
c3804efd42
commit
5af28c4de5
@ -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
|
||||
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
|
||||
administrative details.
|
||||
|
@ -175,7 +175,8 @@ class InterfaceAttachmentController(wsgi.Controller):
|
||||
exception.SecurityGroupCannotBeApplied,
|
||||
exception.NetworkInterfaceTaggedAttachNotSupported,
|
||||
exception.NetworksWithQoSPolicyNotSupported,
|
||||
exception.InterfaceAttachPciClaimFailed) as e:
|
||||
exception.InterfaceAttachPciClaimFailed,
|
||||
exception.InterfaceAttachResourceAllocationFailed) as e:
|
||||
raise exc.HTTPBadRequest(explanation=e.format_message())
|
||||
except (exception.InstanceIsLocked,
|
||||
exception.FixedIpAlreadyInUse,
|
||||
|
@ -4976,14 +4976,17 @@ class API(base.Base):
|
||||
context, instance, instance_actions.ATTACH_INTERFACE)
|
||||
|
||||
# NOTE(gibi): Checking if the requested port has resource request as
|
||||
# such ports are currently not supported as they would at least
|
||||
# need resource allocation manipulation in placement but might also
|
||||
# need a new scheduling if resource on this host is not available.
|
||||
# such ports are only supported if the compute service version is >= 55
|
||||
# TODO(gibi): Remove this check in X as there we can be sure that all
|
||||
# computes are new enough
|
||||
if port_id:
|
||||
port = self.network_api.show_port(context, port_id)
|
||||
if port['port'].get(constants.RESOURCE_REQUEST):
|
||||
raise exception.AttachInterfaceWithQoSPolicyNotSupported(
|
||||
instance_uuid=instance.uuid)
|
||||
svc = objects.Service.get_by_host_and_binary(
|
||||
context, instance.host, 'nova-compute')
|
||||
if svc.version < 55:
|
||||
raise exception.AttachInterfaceWithQoSPolicyNotSupported(
|
||||
instance_uuid=instance.uuid)
|
||||
|
||||
return self.compute_rpcapi.attach_interface(context,
|
||||
instance=instance, network_id=network_id, port_id=port_id,
|
||||
|
@ -7569,6 +7569,96 @@ class ComputeManager(manager.Manager):
|
||||
|
||||
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
|
||||
# NotFound and QuotaError exceptions getting traced as well.
|
||||
@messaging.expected_exceptions(
|
||||
@ -7582,6 +7672,7 @@ class ComputeManager(manager.Manager):
|
||||
# PortNotUsableDNS
|
||||
# AttachSRIOVPortNotSupported
|
||||
# NetworksWithQoSPolicyNotSupported
|
||||
# InterfaceAttachResourceAllocationFailed
|
||||
exception.Invalid)
|
||||
@wrap_exception()
|
||||
@wrap_instance_event(prefix='compute')
|
||||
@ -7639,31 +7730,39 @@ class ComputeManager(manager.Manager):
|
||||
instance.flavor, instance.image_meta)
|
||||
pci_reqs = objects.InstancePCIRequests(
|
||||
requests=[], instance_uuid=instance.uuid)
|
||||
self.network_api.create_resource_requests(
|
||||
_, request_groups = self.network_api.create_resource_requests(
|
||||
context, requested_networks, pci_reqs,
|
||||
affinity_policy=pci_numa_affinity_policy)
|
||||
|
||||
# We only support one port per attach request so we at most have one
|
||||
# pci request
|
||||
pci_req = None
|
||||
if pci_reqs.requests:
|
||||
pci_req = pci_reqs.requests[0]
|
||||
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:
|
||||
pci_device = self._claim_pci_device_for_interface_attach(
|
||||
context, instance, pci_reqs)
|
||||
except exception.InterfaceAttachPciClaimFailed:
|
||||
with excutils.save_and_reraise_exception():
|
||||
if pci_req:
|
||||
instance.pci_requests.requests.remove(pci_req)
|
||||
if resources:
|
||||
# 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(
|
||||
context,
|
||||
instance,
|
||||
requested_networks,
|
||||
bind_host_id=bind_host_id,
|
||||
resource_provider_mapping=provider_mappings,
|
||||
)
|
||||
|
||||
if len(network_info) != 1:
|
||||
|
@ -1339,6 +1339,11 @@ class InterfaceAttachPciClaimFailed(Invalid):
|
||||
"interface attach")
|
||||
|
||||
|
||||
class InterfaceAttachResourceAllocationFailed(Invalid):
|
||||
msg_fmt = _("Failed to allocate additional resources to %(instance_uuid)s "
|
||||
"during interface attach")
|
||||
|
||||
|
||||
class InterfaceDetachFailed(Invalid):
|
||||
msg_fmt = _("Failed to detach network adapter device from "
|
||||
"%(instance_uuid)s")
|
||||
|
@ -31,7 +31,7 @@ LOG = logging.getLogger(__name__)
|
||||
|
||||
|
||||
# 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
|
||||
@ -194,6 +194,9 @@ SERVICE_VERSION_HISTORY = (
|
||||
# Add accel_uuids (accelerator requests) param to shelve_instance and
|
||||
# shelve_offload_instance and unshelve_instance
|
||||
{'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
|
||||
|
@ -192,6 +192,17 @@ class ResourceRequest(object):
|
||||
|
||||
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):
|
||||
requested_resources = (request_spec.requested_resources
|
||||
if 'requested_resources' in request_spec and
|
||||
|
@ -5898,21 +5898,34 @@ class UnsupportedPortResourceRequestBasedSchedulingTest(
|
||||
self.neutron.port_with_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
|
||||
server = self._create_server(
|
||||
flavor=self.flavor,
|
||||
networks=[{'port': self.neutron.port_1['id']}])
|
||||
self._wait_for_state_change(server, 'ACTIVE')
|
||||
|
||||
# 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)
|
||||
# simulate that the compute the instance is running on is older than
|
||||
# when support is added for attach, older than service version 55
|
||||
orig_get_service = objects.Service.get_by_host_and_binary
|
||||
|
||||
def fake_get_service(context, host, binary):
|
||||
service = orig_get_service(context, host, binary)
|
||||
service.version = 54
|
||||
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.assertIn('Attaching interfaces with QoS policy is '
|
||||
'not supported for instance',
|
||||
@ -6131,6 +6144,301 @@ class PortResourceRequestBasedSchedulingTest(
|
||||
self.sriov_dev_rp_per_host[self.compute1_rp_uuid][self.PF2],
|
||||
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):
|
||||
port = self.neutron.port_with_resource_request
|
||||
|
||||
|
@ -7094,9 +7094,15 @@ class ComputeAPIUnitTestCase(_ComputeAPIUnitTestMixIn, test.NoDBTestCase):
|
||||
mock_record.assert_called_once_with(
|
||||
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')
|
||||
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()
|
||||
service = objects.Service()
|
||||
service.version = 54
|
||||
mock_get_service.return_value = service
|
||||
with mock.patch.object(
|
||||
self.compute_api.network_api, 'show_port',
|
||||
return_value={'port': {
|
||||
@ -7110,6 +7116,42 @@ class ComputeAPIUnitTestCase(_ComputeAPIUnitTestMixIn, test.NoDBTestCase):
|
||||
'foo_net_id', 'foo_port_id', None
|
||||
)
|
||||
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.object(compute_rpcapi.ComputeAPI, 'detach_interface')
|
||||
|
@ -65,6 +65,7 @@ from nova.objects import instance as instance_obj
|
||||
from nova.objects import migrate_data as migrate_data_obj
|
||||
from nova.policies import base as base_policy
|
||||
from nova.policies import servers as servers_policy
|
||||
from nova.scheduler import utils as scheduler_utils
|
||||
from nova import test
|
||||
from nova.tests import fixtures
|
||||
from nova.tests.unit.compute import eventlet_utils
|
||||
@ -10151,6 +10152,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
"_claim_pci_device_for_interface_attach",
|
||||
return_value=None)
|
||||
) 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,
|
||||
instance,
|
||||
network_id,
|
||||
@ -10160,7 +10162,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
mock_allocate.assert_called_once_with(
|
||||
self.context, instance,
|
||||
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]
|
||||
self.assertEqual(1, len(network_requests.objects))
|
||||
network_request = network_requests[0]
|
||||
@ -10184,6 +10186,79 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
|
||||
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')
|
||||
def test_attach_sriov_interface(self, mock_notify):
|
||||
instance = self._create_fake_instance_obj()
|
||||
@ -10218,6 +10293,8 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
pci_requests=None, affinity_policy=None):
|
||||
# Simulate that the requested port is an SRIOV port
|
||||
pci_requests.requests.append(pci_req)
|
||||
# without resource request
|
||||
return None, []
|
||||
|
||||
mock_create_resource_req.side_effect = create_resource_req
|
||||
|
||||
@ -10228,7 +10305,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
mock_allocate.assert_called_once_with(
|
||||
self.context, instance,
|
||||
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]
|
||||
self.assertEqual(1, len(network_requests.objects))
|
||||
network_request = network_requests[0]
|
||||
@ -10254,6 +10331,100 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
|
||||
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')
|
||||
def test_interface_tagged_attach(self, mock_notify):
|
||||
instance = self._create_fake_instance_obj()
|
||||
@ -10273,6 +10444,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
'_claim_pci_device_for_interface_attach',
|
||||
return_value=None)
|
||||
) as (mock_capabilities, mock_create_resource_req, mock_claim_pci):
|
||||
mock_create_resource_req.return_value = (None, [])
|
||||
vif = self.compute.attach_interface(self.context,
|
||||
instance,
|
||||
network_id,
|
||||
@ -10282,7 +10454,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
mock_allocate.assert_called_once_with(
|
||||
self.context, instance,
|
||||
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]
|
||||
self.assertEqual(1, len(network_requests.objects))
|
||||
network_request = network_requests[0]
|
||||
@ -10330,6 +10502,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
network_id = nwinfo[0]['network']['id']
|
||||
port_id = nwinfo[0]['id']
|
||||
req_ip = '1.2.3.4'
|
||||
instance.pci_requests = objects.InstancePCIRequests(requests=[])
|
||||
|
||||
with test.nested(
|
||||
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,
|
||||
mock_dict, mock_create_resource_req, mock_claim_pci):
|
||||
|
||||
mock_create_resource_req.return_value = (None, [])
|
||||
mock_allocate.return_value = nwinfo
|
||||
mock_attach.side_effect = exception.NovaException("attach_failed")
|
||||
self.assertRaises(exception.InterfaceAttachFailed,
|
||||
@ -10355,7 +10529,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
mock_allocate.assert_called_once_with(
|
||||
self.context, instance,
|
||||
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]
|
||||
self.assertEqual(1, len(network_requests.objects))
|
||||
network_request = network_requests[0]
|
||||
@ -10424,6 +10598,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
pci_requests=None, affinity_policy=None):
|
||||
# Simulate that the requested port is an SRIOV port
|
||||
pci_requests.requests.append(pci_req)
|
||||
return None, []
|
||||
|
||||
mock_create_resource_req.side_effect = create_resource_req
|
||||
mock_allocate.return_value = nwinfo
|
||||
@ -10436,7 +10611,7 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
mock_allocate.assert_called_once_with(
|
||||
self.context, instance,
|
||||
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]
|
||||
self.assertEqual(1, len(network_requests.objects))
|
||||
network_request = network_requests[0]
|
||||
@ -10480,17 +10655,29 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
'create_resource_requests'),
|
||||
mock.patch.object(self.compute.rt, 'claim_pci_devices',
|
||||
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 (
|
||||
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)
|
||||
|
||||
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)
|
||||
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)
|
||||
|
||||
ex = self.assertRaises(
|
||||
messaging.ExpectedException, self.compute.attach_interface,
|
||||
@ -10510,6 +10697,228 @@ class ComputeAPITestCase(BaseTestCase):
|
||||
|
||||
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')
|
||||
def test_detach_interface(self, mock_notify):
|
||||
nwinfo, port_id = self.test_attach_interface()
|
||||
|
@ -2545,6 +2545,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase,
|
||||
db_instance)
|
||||
f_instance.flavor = objects.Flavor()
|
||||
f_instance.system_metadata = {}
|
||||
f_instance.pci_requests = objects.InstancePCIRequests(requests=[])
|
||||
e = exception.InterfaceAttachFailed(instance_uuid=f_instance.uuid)
|
||||
|
||||
@mock.patch('nova.network.neutron.API.create_resource_requests')
|
||||
@ -2561,6 +2562,7 @@ class ComputeManagerUnitTestCase(test.NoDBTestCase,
|
||||
def do_test(
|
||||
update, meth, add_fault, notify, event, mock_claim_pci,
|
||||
mock_create_resource_req):
|
||||
mock_create_resource_req.return_value = None, []
|
||||
self.assertRaises(exception.InterfaceAttachFailed,
|
||||
self.compute.attach_interface,
|
||||
self.context, f_instance, uuids.network_id,
|
||||
|
@ -1274,6 +1274,31 @@ class TestUtils(TestUtilsBase):
|
||||
rr = utils.ResourceRequest.from_request_spec(rs)
|
||||
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):
|
||||
flavor = objects.Flavor(
|
||||
vcpus=1, memory_mb=1024, root_gb=10, ephemeral_gb=5, swap=0)
|
||||
|
@ -0,0 +1,5 @@
|
||||
---
|
||||
features:
|
||||
- |
|
||||
The ``POST /servers/{server_id}/os-interface`` API now supports
|
||||
attaching neutron ports with QoS minimum bandwidth rules attached.
|
Loading…
Reference in New Issue
Block a user