nova-manage: heal port allocations

Before I97f06d0ec34cbd75c182caaa686b8de5c777a576 it was possible to
create servers with neutron ports which had resource_request (e.g. a
port with QoS minimum bandwidth policy rule) without allocating the
requested resources in placement. So there could be servers for which
the allocation needs to be healed in placement.

This patch extends the nova-manage heal_allocation CLI to create the
missing port allocations in placement and update the port in neutron
with the resource provider uuid that is used for the allocation.

There are known limiations of this patch. It does not try to reimplement
Placement's allocation candidate functionality. Therefore it cannot
handle the situation when there is more than one RP in the compute
tree which provides the required traits for a port. In this situation
deciding which RP to use would require the in_tree allocation candidate
support from placement which is not available yet and 2) information
about which PCI PF an SRIOV port is allocated from its VF and which RP
represents that PCI device in placement. This information is only
available on the compute hosts.

For the unsupported cases the command will fail gracefully. As soon as
migration support for such servers are implemented in the blueprint
support-move-ops-with-qos-ports the admin can heal the allocation of
such servers by migrating them.

During healing both placement and neutron need to be updated. If any of
those updates fail the code tries to roll back the previous updates for
the instance to make sure that the healing can be re-run later without
issue. However if the rollback fails then the script will terminate with
an error message pointing to documentation that describes how to
recover from such a partially healed situation manually.

Closes-Bug: #1819923
Change-Id: I4b2b1688822eb2f0174df0c8c6c16d554781af85
This commit is contained in:
Balazs Gibizer 2019-03-18 17:24:01 +01:00 committed by Balazs Gibizer
parent e6f0119262
commit 54dea2531c
7 changed files with 1170 additions and 17 deletions

View File

@ -367,13 +367,37 @@ Nova Cells v2
Placement
~~~~~~~~~
``nova-manage placement heal_allocations [--max-count <max_count>] [--verbose] [--dry-run] [--instance <instance_uuid>]``
``nova-manage placement heal_allocations [--max-count <max_count>] [--verbose] [--skip-port-allocations] [--dry-run] [--instance <instance_uuid>]``
Iterates over non-cell0 cells looking for instances which do not have
allocations in the Placement service and which are not undergoing a task
state transition. For each instance found, allocations are created against
the compute node resource provider for that instance based on the flavor
associated with the instance.
Also if the instance has any port attached that has resource request
(e.g. :neutron-doc:`Quality of Service (QoS): Guaranteed Bandwidth
<admin/config-qos-min-bw.html>`) but the corresponding
allocation is not found then the allocation is created against the
network device resource providers according to the resource request of
that port. It is possible that the missing allocation cannot be created
either due to not having enough resource inventory on the host the instance
resides on or because more than one resource provider could fulfill the
request. In this case the instance needs to be manually deleted or the
port needs to be detached. When nova `supports migrating instances
with guaranteed bandwidth ports`_, migration will heal missing allocations
for these instances.
Before the allocations for the ports are persisted in placement nova-manage
tries to update each port in neutron to refer to the resource provider UUID
which provides the requested resources. If any of the port updates fail in
neutron or the allocation update fails in placement the command tries to
roll back the partial updates to the ports. If the roll back fails
then the process stops with exit code ``7`` and the admin needs to do the
rollback in neutron manually according to the description in the exit code
section.
.. _supports migrating instances with guaranteed bandwidth ports: https://specs.openstack.org/openstack/nova-specs/specs/train/approved/support-move-ops-with-qos-ports.html
There is also a special case handled for instances that *do* have
allocations created before Placement API microversion 1.8 where project_id
and user_id values were required. For those types of allocations, the
@ -393,6 +417,13 @@ Placement
specified the ``--max-count`` option has no effect.
*(Since 20.0.0 Train)*
Specify ``--skip-port-allocations`` to skip the healing of the resource
allocations of bound ports, e.g. healing bandwidth resource allocation for
ports having minimum QoS policy rules attached. If your deployment does
not use such a feature then the performance impact of querying neutron
ports for each instance can be avoided with this flag.
*(Since 20.0.0 Train)*
This command requires that the ``[api_database]/connection`` and
``[placement]`` configuration options are set. Placement API >= 1.28 is
required.
@ -405,6 +436,14 @@ Placement
* 3: Unable to create (or update) allocations for an instance against its
compute node resource provider.
* 4: Command completed successfully but no allocations were created.
* 5: Unable to query ports from neutron
* 6: Unable to update ports in neutron
* 7: Cannot roll back neutron port updates. Manual steps needed. The error
message will indicate which neutron ports need to be changed to clean up
``binding:profile`` of the port::
$ openstack port unset <port_uuid> --binding-profile allocation
* 127: Invalid input.
``nova-manage placement sync_aggregates [--verbose]``

View File

@ -23,6 +23,7 @@
from __future__ import print_function
import collections
import functools
import re
import sys
@ -32,6 +33,7 @@ from dateutil import parser as dateutil_parser
import decorator
from keystoneauth1 import exceptions as ks_exc
import netaddr
from neutronclient.common import exceptions as neutron_client_exc
from oslo_config import cfg
from oslo_db import exception as db_exc
from oslo_log import log as logging
@ -54,6 +56,7 @@ from nova.db import migration
from nova.db.sqlalchemy import api as sa_db
from nova import exception
from nova.i18n import _
from nova.network.neutronv2 import api as neutron_api
from nova import objects
from nova.objects import block_device as block_device_obj
from nova.objects import build_request as build_request_obj
@ -1658,6 +1661,290 @@ class PlacementCommands(object):
node_cache[instance.node] = node_uuid
return node_uuid
@staticmethod
def _get_ports(ctxt, instance, neutron):
"""Return the ports that are bound to the instance
:param ctxt: nova.context.RequestContext
:param instance: the instance to return the ports for
:param neutron: nova.network.neutronv2.api.ClientWrapper to
communicate with Neutron
:return: a list of neutron port dict objects
:raise UnableToQueryPorts: If the neutron list ports query fails.
"""
try:
return neutron.list_ports(
ctxt, device_id=instance.uuid,
fields=['id', 'resource_request', 'binding:profile'])['ports']
except neutron_client_exc.NeutronClientException as e:
raise exception.UnableToQueryPorts(
instance_uuid=instance.uuid, error=six.text_type(e))
@staticmethod
def _has_request_but_no_allocation(port):
request = port.get('resource_request')
binding_profile = port.get('binding:profile', {}) or {}
allocation = binding_profile.get('allocation')
# We are defensive here about 'resources' and 'required' in the
# 'resource_request' as neutron API is not clear about those fields
# being optional.
return (request and request.get('resources') and
request.get('required') and
not allocation)
@staticmethod
def _get_rps_in_tree_with_required_traits(
ctxt, rp_uuid, required_traits, placement):
"""Find the RPs that have all the required traits in the given rp tree.
:param ctxt: nova.context.RequestContext
:param rp_uuid: the RP uuid that will be used to query the tree.
:param required_traits: the traits that need to be supported by
the returned resource providers.
:param placement: nova.scheduler.client.report.SchedulerReportClient
to communicate with the Placement service API.
:raise PlacementAPIConnectFailure: if placement API cannot be reached
:raise ResourceProviderRetrievalFailed: if the resource provider does
not exist.
:raise ResourceProviderTraitRetrievalFailed: if resource provider
trait information cannot be read from placement.
:return: A list of RP UUIDs that supports every required traits and
in the tree for the provider rp_uuid.
"""
try:
rps = placement.get_providers_in_tree(ctxt, rp_uuid)
matching_rps = [
rp['uuid']
for rp in rps
if set(required_traits).issubset(
placement.get_provider_traits(ctxt, rp['uuid']).traits)
]
except ks_exc.ClientException:
raise exception.PlacementAPIConnectFailure()
return matching_rps
@staticmethod
def _merge_allocations(alloc1, alloc2):
"""Return a new allocation dict that contains the sum of alloc1 and
alloc2.
:param alloc1: a dict in the form of
{
<rp_uuid>: {'resources': {<resource class>: amount,
<resource class>: amount},
<rp_uuid>: {'resources': {<resource class>: amount},
}
:param alloc2: a dict in the same form as alloc1
:return: the merged allocation of alloc1 and alloc2 in the same format
"""
allocations = collections.defaultdict(
lambda: {'resources': collections.defaultdict(int)})
for alloc in [alloc1, alloc2]:
for rp_uuid in alloc:
for rc, amount in alloc[rp_uuid]['resources'].items():
allocations[rp_uuid]['resources'][rc] += amount
return allocations
def _get_port_allocation(
self, ctxt, node_uuid, port, instance_uuid, placement):
"""Return the extra allocation the instance needs due to the given
port.
:param ctxt: nova.context.RequestContext
:param node_uuid: the ComputeNode uuid the instance is running on.
:param port: the port dict returned from neutron
:param instance_uuid: The uuid of the instance the port is bound to
:param placement: nova.scheduler.client.report.SchedulerReportClient
to communicate with the Placement service API.
:raise PlacementAPIConnectFailure: if placement API cannot be reached
:raise ResourceProviderRetrievalFailed: compute node resource provider
does not exist.
:raise ResourceProviderTraitRetrievalFailed: if resource provider
trait information cannot be read from placement.
:raise MoreThanOneResourceProviderToHealFrom: if it cannot be decided
unambiguously which resource provider to heal from.
:raise NoResourceProviderToHealFrom: if there is no resource provider
found to heal from.
:return: A dict of resources keyed by RP uuid to be included in the
instance allocation dict.
"""
matching_rp_uuids = self._get_rps_in_tree_with_required_traits(
ctxt, node_uuid, port['resource_request']['required'], placement)
if len(matching_rp_uuids) > 1:
# If there is more than one such RP then it is an ambiguous
# situation that we cannot handle here efficiently because that
# would require the reimplementation of most of the allocation
# candidate query functionality of placement. Also if more
# than one such RP exists then selecting the right one might
# need extra information from the compute node. For example
# which PCI PF the VF is allocated from and which RP represents
# that PCI PF in placement. When migration is supported with such
# servers then we can ask the admin to migrate these servers
# instead to heal their allocation.
raise exception.MoreThanOneResourceProviderToHealFrom(
rp_uuids=','.join(matching_rp_uuids),
port_id=port['id'],
instance_uuid=instance_uuid)
if len(matching_rp_uuids) == 0:
raise exception.NoResourceProviderToHealFrom(
port_id=port['id'],
instance_uuid=instance_uuid,
traits=port['resource_request']['required'],
node_uuid=node_uuid)
# We found one RP that matches the traits. Assume that we can allocate
# the resources from it. If there is not enough inventory left on the
# RP then the PUT /allocations placement call will detect that.
rp_uuid = matching_rp_uuids[0]
port_allocation = {
rp_uuid: {
'resources': port['resource_request']['resources']
}
}
return port_allocation
def _get_port_allocations_to_heal(
self, ctxt, instance, node_cache, placement, neutron, output):
"""Return the needed extra allocation for the ports of the instance.
:param ctxt: nova.context.RequestContext
:param instance: instance to get the port allocations for
:param node_cache: dict of Instance.node keys to ComputeNode.uuid
values; this cache is updated if a new node is processed.
:param placement: nova.scheduler.client.report.SchedulerReportClient
to communicate with the Placement service API.
:param neutron: nova.network.neutronv2.api.ClientWrapper to
communicate with Neutron
:param output: function that takes a single message for verbose output
:raise UnableToQueryPorts: If the neutron list ports query fails.
:raise nova.exception.ComputeHostNotFound: if compute node of the
instance not found in the db.
:raise PlacementAPIConnectFailure: if placement API cannot be reached
:raise ResourceProviderRetrievalFailed: if the resource provider
representing the compute node the instance is running on does not
exist.
:raise ResourceProviderTraitRetrievalFailed: if resource provider
trait information cannot be read from placement.
:raise MoreThanOneResourceProviderToHealFrom: if it cannot be decided
unambiguously which resource provider to heal from.
:raise NoResourceProviderToHealFrom: if there is no resource provider
found to heal from.
:return: A two tuple where the first item is a dict of resources keyed
by RP uuid to be included in the instance allocation dict. The
second item is a list of port dicts to be updated in Neutron.
"""
# We need to heal port allocations for ports that have resource_request
# but do not have an RP uuid in the binding:profile.allocation field.
# We cannot use the instance info_cache to check the binding profile
# as this code needs to be able to handle ports that were attached
# before nova in stein started updating the allocation key in the
# binding:profile.
# In theory a port can be assigned to an instance without it being
# bound to any host (e.g. in case of shelve offload) but
# _heal_allocations_for_instance() already filters out instances that
# are not on any host.
ports_to_heal = [
port for port in self._get_ports(ctxt, instance, neutron)
if self._has_request_but_no_allocation(port)]
if not ports_to_heal:
# nothing to do, return early
return {}, []
node_uuid = self._get_compute_node_uuid(
ctxt, instance, node_cache)
allocations = {}
for port in ports_to_heal:
port_allocation = self._get_port_allocation(
ctxt, node_uuid, port, instance.uuid, placement)
rp_uuid = list(port_allocation)[0]
allocations = self._merge_allocations(
allocations, port_allocation)
# We also need to record the RP we are allocated from in the
# port. This will be sent back to Neutron before the allocation
# is updated in placement
binding_profile = port.get('binding:profile', {}) or {}
binding_profile['allocation'] = rp_uuid
port['binding:profile'] = binding_profile
output(_("Found resource provider %(rp_uuid)s having matching "
"traits for port %(port_uuid)s with resource request "
"%(request)s attached to instance %(instance_uuid)s") %
{"rp_uuid": rp_uuid, "port_uuid": port["id"],
"request": port.get("resource_request"),
"instance_uuid": instance.uuid})
return allocations, ports_to_heal
def _update_ports(self, neutron, ports_to_update, output):
succeeded = []
try:
for port in ports_to_update:
body = {
'port': {
'binding:profile': port['binding:profile']
}
}
output(
_('Updating port %(port_uuid)s with attributes '
'%(attributes)s') %
{'port_uuid': port['id'], 'attributes': body['port']})
neutron.update_port(port['id'], body=body)
succeeded.append(port)
except neutron_client_exc.NeutronClientException as e:
output(
_('Updating port %(port_uuid)s failed: %(error)s') %
{'port_uuid': port['id'], 'error': six.text_type(e)})
# one of the port updates failed. We need to roll back the updates
# that succeeded before
self._rollback_port_updates(neutron, succeeded, output)
# we failed to heal so we need to stop but we successfully rolled
# back the partial updates so the admin can retry the healing.
raise exception.UnableToUpdatePorts(error=six.text_type(e))
@staticmethod
def _rollback_port_updates(neutron, ports_to_rollback, output):
# _update_ports() added the allocation key to these ports, so we need
# to remove them during the rollback.
manual_rollback_needed = []
last_exc = None
for port in ports_to_rollback:
profile = port['binding:profile']
profile.pop('allocation')
body = {
'port': {
'binding:profile': profile
}
}
try:
output(_('Rolling back port update for %(port_uuid)s') %
{'port_uuid': port['id']})
neutron.update_port(port['id'], body=body)
except neutron_client_exc.NeutronClientException as e:
output(
_('Rolling back update for port %(port_uuid)s failed: '
'%(error)s') % {'port_uuid': port['id'],
'error': six.text_type(e)})
# TODO(gibi): We could implement a retry mechanism with
# back off.
manual_rollback_needed.append(port['id'])
last_exc = e
if manual_rollback_needed:
# At least one of the port operation failed so we failed to roll
# back. There are partial updates in neutron. Human intervention
# needed.
raise exception.UnableToRollbackPortUpdates(
error=six.text_type(last_exc),
port_uuids=manual_rollback_needed)
def _heal_missing_alloc(self, ctxt, instance, node_cache):
node_uuid = self._get_compute_node_uuid(
ctxt, instance, node_cache)
@ -1683,18 +1970,23 @@ class PlacementCommands(object):
return allocations
def _heal_allocations_for_instance(self, ctxt, instance, node_cache,
output, placement, dry_run):
output, placement, dry_run,
heal_port_allocations, neutron):
"""Checks the given instance to see if it needs allocation healing
:param ctxt: cell-targeted nova.context.RequestContext
:param instance: the instance to check for allocation healing
:param node_cache: dict of Instance.node keys to ComputeNode.uuid
values; this cache is updated if a new node is processed.
:param outout: function that takes a single message for verbose output
:param output: function that takes a single message for verbose output
:param placement: nova.scheduler.client.report.SchedulerReportClient
to communicate with the Placement service API.
:param dry_run: Process instances and print output but do not commit
any changes.
:param heal_port_allocations: True if healing port allocation is
requested, False otherwise.
:param neutron: nova.network.neutronv2.api.ClientWrapper to
communicate with Neutron
:return: True if allocations were created or updated for the instance,
None if nothing needed to be done
:raises: nova.exception.ComputeHostNotFound if a compute node for a
@ -1703,6 +1995,21 @@ class PlacementCommands(object):
a given instance against a given compute node resource provider
:raises: AllocationUpdateFailed if unable to update allocations for
a given instance with consumer project/user information
:raise UnableToQueryPorts: If the neutron list ports query fails.
:raise PlacementAPIConnectFailure: if placement API cannot be reached
:raise ResourceProviderRetrievalFailed: if the resource provider
representing the compute node the instance is running on does not
exist.
:raise ResourceProviderTraitRetrievalFailed: if resource provider
trait information cannot be read from placement.
:raise MoreThanOneResourceProviderToHealFrom: if it cannot be decided
unambiguously which resource provider to heal from.
:raise NoResourceProviderToHealFrom: if there is no resource provider
found to heal from.
:raise UnableToUpdatePorts: if a port update failed in neutron but any
partial update was rolled back successfully.
:raise UnableToRollbackPortUpdates: if a port update failed in neutron
and the rollback of the partial updates also failed.
"""
if instance.task_state is not None:
output(_('Instance %(instance)s is undergoing a task '
@ -1744,6 +2051,19 @@ class PlacementCommands(object):
allocations = self._heal_missing_project_and_user_id(
allocations, instance)
if heal_port_allocations:
to_heal = self._get_port_allocations_to_heal(
ctxt, instance, node_cache, placement, neutron, output)
port_allocations, ports_to_update = to_heal
else:
port_allocations, ports_to_update = {}, []
if port_allocations:
need_healing = need_healing or 'Update'
# Merge in any missing port allocations
allocations['allocations'] = self._merge_allocations(
allocations['allocations'], port_allocations)
if need_healing:
if dry_run:
output(_('[dry-run] %(operation)s allocations for instance '
@ -1752,6 +2072,16 @@ class PlacementCommands(object):
'instance': instance.uuid,
'allocations': allocations})
else:
# First update ports in neutron. If any of those operations
# fail, then roll back the successful part of it and fail the
# healing. We do this first because rolling back the port
# updates is more straight-forward than rolling back allocation
# changes.
self._update_ports(neutron, ports_to_update, output)
# Now that neutron update succeeded we can try to update
# placement. If it fails we need to rollback every neutron port
# update done before.
resp = placement.put_allocations(ctxt, instance.uuid,
allocations)
if resp:
@ -1761,15 +2091,24 @@ class PlacementCommands(object):
'instance': instance.uuid})
return True
else:
# Rollback every neutron update. If we succeed to
# roll back then it is safe to stop here and let the admin
# retry. If the rollback fails then
# _rollback_port_updates() will raise another exception
# that instructs the operator how to clean up manually
# before the healing can be retried
self._rollback_port_updates(
neutron, ports_to_update, output)
raise exception.AllocationUpdateFailed(
consumer_uuid=instance.uuid, error='')
else:
output(_('Instance %s already has allocations with '
'matching consumer project/user.') % instance.uuid)
output(_('The allocation of instance %s is up-to-date. '
'Nothing to be healed.') % instance.uuid)
return
def _heal_instances_in_cell(self, ctxt, max_count, unlimited, output,
placement, dry_run, instance_uuid):
placement, dry_run, instance_uuid,
heal_port_allocations, neutron):
"""Checks for instances to heal in a given cell.
:param ctxt: cell-targeted nova.context.RequestContext
@ -1782,6 +2121,10 @@ class PlacementCommands(object):
:param dry_run: Process instances and print output but do not commit
any changes.
:param instance_uuid: UUID of a specific instance to process.
:param heal_port_allocations: True if healing port allocation is
requested, False otherwise.
:param neutron: nova.network.neutronv2.api.ClientWrapper to
communicate with Neutron
:return: Number of instances that had allocations created.
:raises: nova.exception.ComputeHostNotFound if a compute node for a
given instance cannot be found
@ -1789,6 +2132,21 @@ class PlacementCommands(object):
a given instance against a given compute node resource provider
:raises: AllocationUpdateFailed if unable to update allocations for
a given instance with consumer project/user information
:raise UnableToQueryPorts: If the neutron list ports query fails.
:raise PlacementAPIConnectFailure: if placement API cannot be reached
:raise ResourceProviderRetrievalFailed: if the resource provider
representing the compute node the instance is running on does not
exist.
:raise ResourceProviderTraitRetrievalFailed: if resource provider
trait information cannot be read from placement.
:raise MoreThanOneResourceProviderToHealFrom: if it cannot be decided
unambiguously which resource provider to heal from.
:raise NoResourceProviderToHealFrom: if there is no resource provider
found to heal from.
:raise UnableToUpdatePorts: if a port update failed in neutron but any
partial update was rolled back successfully.
:raise UnableToRollbackPortUpdates: if a port update failed in neutron
and the rollback of the partial updates also failed.
"""
# Keep a cache of instance.node to compute node resource provider UUID.
# This will save some queries for non-ironic instances to the
@ -1820,7 +2178,7 @@ class PlacementCommands(object):
for instance in instances:
if self._heal_allocations_for_instance(
ctxt, instance, node_cache, output, placement,
dry_run):
dry_run, heal_port_allocations, neutron):
num_processed += 1
# Make sure we don't go over the max count. Note that we
@ -1843,7 +2201,8 @@ class PlacementCommands(object):
@action_description(
_("Iterates over non-cell0 cells looking for instances which do "
"not have allocations in the Placement service, or have incomplete "
"consumer project_id/user_id values in existing allocations, and "
"consumer project_id/user_id values in existing allocations or "
"missing allocations for ports having resource request, and "
"which are not undergoing a task state transition. For each "
"instance found, allocations are created (or updated) against the "
"compute node resource provider for that instance based on the "
@ -1864,8 +2223,16 @@ class PlacementCommands(object):
@args('--instance', metavar='<instance_uuid>', dest='instance_uuid',
help='UUID of a specific instance to process. If specified '
'--max-count has no effect.')
@args('--skip-port-allocations', action='store_true',
dest='skip_port_allocations', default=False,
help='Skip the healing of the resource allocations of bound ports. '
'E.g. healing bandwidth resource allocation for ports having '
'minimum QoS policy rules attached. If your deployment does '
'not use such a feature then the performance impact of '
'querying neutron ports for each instance can be avoided with '
'this flag.')
def heal_allocations(self, max_count=None, verbose=False, dry_run=False,
instance_uuid=None):
instance_uuid=None, skip_port_allocations=False):
"""Heals instance allocations in the Placement service
Return codes:
@ -1876,6 +2243,9 @@ class PlacementCommands(object):
* 3: Unable to create (or update) allocations for an instance against
its compute node resource provider.
* 4: Command completed successfully but no allocations were created.
* 5: Unable to query ports from neutron
* 6: Unable to update ports in neutron
* 7: Cannot roll back neutron port updates. Manual steps needed.
* 127: Invalid input.
"""
# NOTE(mriedem): Thoughts on ways to expand this:
@ -1891,6 +2261,8 @@ class PlacementCommands(object):
# would probably only be safe with a specific instance.
# - deal with nested resource providers?
heal_port_allocations = not skip_port_allocations
output = lambda msg: None
if verbose:
output = lambda msg: print(msg)
@ -1937,6 +2309,11 @@ class PlacementCommands(object):
return 4
placement = report.SchedulerReportClient()
neutron = None
if heal_port_allocations:
neutron = neutron_api.get_client(ctxt, admin=True)
num_processed = 0
# TODO(mriedem): Use context.scatter_gather_skip_cell0.
for cell in cells:
@ -1957,14 +2334,28 @@ class PlacementCommands(object):
try:
num_processed += self._heal_instances_in_cell(
cctxt, limit_per_cell, unlimited, output, placement,
dry_run, instance_uuid)
dry_run, instance_uuid, heal_port_allocations, neutron)
except exception.ComputeHostNotFound as e:
print(e.format_message())
return 2
except (exception.AllocationCreateFailed,
exception.AllocationUpdateFailed) as e:
exception.AllocationUpdateFailed,
exception.NoResourceProviderToHealFrom,
exception.MoreThanOneResourceProviderToHealFrom,
exception.PlacementAPIConnectFailure,
exception.ResourceProviderRetrievalFailed,
exception.ResourceProviderTraitRetrievalFailed) as e:
print(e.format_message())
return 3
except exception.UnableToQueryPorts as e:
print(e.format_message())
return 5
except exception.UnableToUpdatePorts as e:
print(e.format_message())
return 6
except exception.UnableToRollbackPortUpdates as e:
print(e.format_message())
return 7
# Make sure we don't go over the max count. Note that we
# don't include instances that already have allocations in the

View File

@ -2440,3 +2440,53 @@ class ReshapeFailed(NovaException):
class ReshapeNeeded(NovaException):
msg_fmt = _("Virt driver indicates that provider inventories need to be "
"moved.")
class HealPortAllocationException(NovaException):
msg_fmt = _("Healing port allocation failed.")
class MoreThanOneResourceProviderToHealFrom(HealPortAllocationException):
msg_fmt = _("More than one matching resource provider %(rp_uuids)s is "
"available for healing the port allocation for port "
"%(port_id)s for instance %(instance_uuid)s. This script "
"does not have enough information to select the proper "
"resource provider from which to heal.")
class NoResourceProviderToHealFrom(HealPortAllocationException):
msg_fmt = _("No matching resource provider is "
"available for healing the port allocation for port "
"%(port_id)s for instance %(instance_uuid)s. There are no "
"resource providers with matching traits %(traits)s in the "
"provider tree of the resource provider %(node_uuid)s ."
"This probably means that the neutron QoS configuration is "
"wrong. Consult with "
"https://docs.openstack.org/neutron/latest/admin/"
"config-qos-min-bw.html for information on how to configure "
"neutron. If the configuration is fixed the script can be run "
"again.")
class UnableToQueryPorts(HealPortAllocationException):
msg_fmt = _("Unable to query ports for instance %(instance_uuid)s: "
"%(error)s")
class UnableToUpdatePorts(HealPortAllocationException):
msg_fmt = _("Unable to update ports with allocations that are about to be "
"created in placement: %(error)s. The healing of the "
"instance is aborted. It is safe to try to heal the instance "
"again.")
class UnableToRollbackPortUpdates(HealPortAllocationException):
msg_fmt = _("Failed to update neutron ports with allocation keys and the "
"automatic rollback of the previously successful port updates "
"also failed: %(error)s. Make sure that the "
"binding:profile.allocation key of the affected ports "
"%(port_uuids)s are manually cleaned in neutron according to "
"document https://docs.openstack.org/nova/latest/cli/"
"nova-manage.html#placement. If you re-run the script without "
"the manual fix then the missing allocation for these ports "
"will not be healed in placement.")

View File

@ -1601,7 +1601,9 @@ class NeutronFixture(fixtures.Fixture):
def update_port(self, port_id, body=None):
port = self._ports[port_id]
port.update(body['port'])
# We need to deepcopy here as well as the body can have a nested dict
# which can be modified by the caller after this update_port call
port.update(copy.deepcopy(body['port']))
return {'port': copy.deepcopy(port)}
def show_quota(self, project_id):

View File

@ -11,18 +11,24 @@
# under the License.
from __future__ import absolute_import
import collections
import mock
import fixtures
from neutronclient.common import exceptions as neutron_client_exc
import os_resource_classes as orc
from oslo_utils.fixture import uuidsentinel
from six.moves import StringIO
from nova.cmd import manage
from nova import config
from nova import context
from nova import exception
from nova import objects
from nova import test
from nova.tests import fixtures as nova_fixtures
from nova.tests.functional import integrated_helpers
from nova.tests.functional import test_servers
CONF = config.CONF
INCOMPLETE_CONSUMER_ID = '00000000-0000-0000-0000-000000000000'
@ -474,9 +480,9 @@ class TestNovaManagePlacementHealAllocations(
self.assertIn('Max count reached. Processed 1 instances.', output)
# If this is the 2nd call, we'll have skipped the first instance.
if x == 0:
self.assertNotIn('already has allocations', output)
self.assertNotIn('is up-to-date', output)
else:
self.assertIn('already has allocations', output)
self.assertIn('is up-to-date', output)
self._assert_healed(server1, rp_uuid1)
self._assert_healed(server2, rp_uuid2)
@ -484,7 +490,7 @@ class TestNovaManagePlacementHealAllocations(
# run it again to make sure nothing was processed
result = self.cli.heal_allocations(verbose=True)
self.assertEqual(4, result, self.output.getvalue())
self.assertIn('already has allocations', self.output.getvalue())
self.assertIn('is up-to-date', self.output.getvalue())
def test_heal_allocations_paging_max_count_more_than_num_instances(self):
"""Sets up 2 instances in cell1 and 1 instance in cell2. Then specify
@ -741,6 +747,547 @@ class TestNovaManagePlacementHealAllocations(
server['id'], output)
class TestNovaManagePlacementHealPortAllocations(
test_servers.PortResourceRequestBasedSchedulingTestBase):
def setUp(self):
super(TestNovaManagePlacementHealPortAllocations, self).setUp()
self.cli = manage.PlacementCommands()
self.flavor = self.api.get_flavors()[0]
self.output = StringIO()
self.useFixture(fixtures.MonkeyPatch('sys.stdout', self.output))
# Make it easier to debug failed test cases
def print_stdout_on_fail(*args, **kwargs):
import sys
sys.stderr.write(self.output.getvalue())
self.addOnException(print_stdout_on_fail)
def _add_resource_request_to_a_bound_port(self, port_id, resource_request):
# NOTE(gibi): self.neutron._ports contains a copy of each neutron port
# defined on class level in the fixture. So modifying what is in the
# _ports list is safe as it is re-created for each Neutron fixture
# instance therefore for each individual test using that fixture.
bound_port = self.neutron._ports[port_id]
bound_port['resource_request'] = resource_request
def _create_server_with_missing_port_alloc(
self, ports, resource_request=None):
if not resource_request:
resource_request = {
"resources": {
orc.NET_BW_IGR_KILOBIT_PER_SEC: 1000,
orc.NET_BW_EGR_KILOBIT_PER_SEC: 1000},
"required": ["CUSTOM_PHYSNET2", "CUSTOM_VNIC_TYPE_NORMAL"]
}
server = self._create_server(
flavor=self.flavor,
networks=[{'port': port['id']} for port in ports])
server = self._wait_for_state_change(self.admin_api, server, 'ACTIVE')
# This is a hack to simulate that we have a server that is missing
# allocation for its port
for port in ports:
self._add_resource_request_to_a_bound_port(
port['id'], resource_request)
updated_ports = [
self.neutron.show_port(port['id'])['port'] for port in ports]
return server, updated_ports
def _assert_placement_updated(self, server, ports):
rsp = self.placement_api.get(
'/allocations/%s' % server['id'],
version=1.28).body
allocations = rsp['allocations']
# we expect one allocation for the compute resources and one for the
# networking resources
self.assertEqual(2, len(allocations))
self.assertEqual(
self._resources_from_flavor(self.flavor),
allocations[self.compute1_rp_uuid]['resources'])
self.assertEqual(server['tenant_id'], rsp['project_id'])
self.assertEqual(server['user_id'], rsp['user_id'])
network_allocations = allocations[
self.ovs_bridge_rp_per_host[self.compute1_rp_uuid]]['resources']
# this code assumes that every port is allocated from the same OVS
# bridge RP
total_request = collections.defaultdict(int)
for port in ports:
port_request = port['resource_request']['resources']
for rc, amount in port_request.items():
total_request[rc] += amount
self.assertEqual(total_request, network_allocations)
def _assert_port_updated(self, port_uuid):
updated_port = self.neutron.show_port(port_uuid)['port']
binding_profile = updated_port.get('binding:profile', {})
self.assertEqual(
self.ovs_bridge_rp_per_host[self.compute1_rp_uuid],
binding_profile['allocation'])
def _assert_ports_updated(self, ports):
for port in ports:
self._assert_port_updated(port['id'])
def _assert_placement_not_updated(self, server):
allocations = self.placement_api.get(
'/allocations/%s' % server['id']).body['allocations']
self.assertEqual(1, len(allocations))
self.assertIn(self.compute1_rp_uuid, allocations)
def _assert_port_not_updated(self, port_uuid):
updated_port = self.neutron.show_port(port_uuid)['port']
binding_profile = updated_port.get('binding:profile', {})
self.assertNotIn('allocation', binding_profile)
def _assert_ports_not_updated(self, ports):
for port in ports:
self._assert_port_not_updated(port['id'])
def test_heal_port_allocation_only(self):
"""Test that only port allocation needs to be healed for an instance.
* boot with a neutron port that does not have resource request
* hack in a resource request for the bound port
* heal the allocation
* check if the port allocation is created in placement and the port
is updated in neutron
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_updated(server, ports)
self._assert_ports_updated(ports)
self.assertIn(
'Successfully updated allocations',
self.output.getvalue())
self.assertEqual(0, result)
def test_no_healing_is_needed(self):
"""Test that the instance has a port that has allocations
so nothing to be healed.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# heal it once
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_updated(server, ports)
self._assert_ports_updated(ports)
self.assertIn(
'Successfully updated allocations',
self.output.getvalue())
self.assertEqual(0, result)
# try to heal it again
result = self.cli.heal_allocations(verbose=True, max_count=2)
# nothing is removed
self._assert_placement_updated(server, ports)
self._assert_ports_updated(ports)
# healing was not needed
self.assertIn(
'Nothing to be healed.',
self.output.getvalue())
self.assertEqual(4, result)
def test_skip_heal_port_allocation(self):
"""Test that only port allocation needs to be healed for an instance
but port healing is skipped on the cli.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# let's trigger a heal
result = self.cli.heal_allocations(
verbose=True, max_count=2, skip_port_allocations=True)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
output = self.output.getvalue()
self.assertNotIn('Updating port', output)
self.assertIn('Nothing to be healed', output)
self.assertEqual(4, result)
def test_skip_heal_port_allocation_but_heal_the_rest(self):
"""Test that the instance doesn't have allocation at all, needs
allocation for ports as well, but only heal the non port related
allocation.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# delete the server allocation in placement to simulate that it needs
# to be healed
# NOTE(gibi): putting empty allocation will delete the consumer in
# placement
allocations = self.placement_api.get(
'/allocations/%s' % server['id'], version=1.28).body
allocations['allocations'] = {}
self.placement_api.put(
'/allocations/%s' % server['id'], allocations, version=1.28)
# let's trigger a heal
result = self.cli.heal_allocations(
verbose=True, max_count=2, skip_port_allocations=True)
# this actually checks that the server has its non port related
# allocation in placement
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
output = self.output.getvalue()
self.assertIn(
'Successfully created allocations for instance', output)
self.assertEqual(0, result)
def test_heal_port_allocation_and_project_id(self):
"""Test that not just port allocation needs to be healed but also the
missing project_id and user_id.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# override allocation with placement microversion <1.8 to simulate
# missing project_id and user_id
alloc_body = {
"allocations": [
{
"resource_provider": {
"uuid": self.compute1_rp_uuid
},
"resources": {
"MEMORY_MB": self.flavor['ram'],
"VCPU": self.flavor['vcpus'],
"DISK_GB": self.flavor['disk']
}
}
]
}
self.placement_api.put('/allocations/%s' % server['id'], alloc_body)
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_updated(server, ports)
self._assert_ports_updated(ports)
output = self.output.getvalue()
self.assertIn(
'Successfully updated allocations for instance', output)
self.assertIn('Processed 1 instances.', output)
self.assertEqual(0, result)
def test_heal_allocation_create_allocation_with_port_allocation(self):
"""Test that the instance doesn't have allocation at all but needs
allocation for the ports as well.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
# delete the server allocation in placement to simulate that it needs
# to be healed
# NOTE(gibi): putting empty allocation will delete the consumer in
# placement
allocations = self.placement_api.get(
'/allocations/%s' % server['id'], version=1.28).body
allocations['allocations'] = {}
self.placement_api.put(
'/allocations/%s' % server['id'], allocations, version=1.28)
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_updated(server, ports)
self._assert_ports_updated(ports)
output = self.output.getvalue()
self.assertIn(
'Successfully created allocations for instance', output)
self.assertEqual(0, result)
def test_heal_port_allocation_not_enough_resources_for_port(self):
"""Test that a port needs allocation but not enough inventory
available.
"""
# The port will request too much NET_BW_IGR_KILOBIT_PER_SEC so there is
# no RP on the host that can provide it.
resource_request = {
"resources": {
orc.NET_BW_IGR_KILOBIT_PER_SEC: 100000000000,
orc.NET_BW_EGR_KILOBIT_PER_SEC: 1000},
"required": ["CUSTOM_PHYSNET2",
"CUSTOM_VNIC_TYPE_NORMAL"]
}
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1], resource_request)
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
# Actually the ports were updated but the update is rolled back when
# the placement update failed
self._assert_ports_not_updated(ports)
output = self.output.getvalue()
self.assertIn(
'Rolling back port update',
output)
self.assertIn(
'Failed to update allocations for consumer',
output)
self.assertEqual(3, result)
def test_heal_port_allocation_no_rp_providing_required_traits(self):
"""Test that a port needs allocation but no rp is providing the
required traits.
"""
# The port will request a trait, CUSTOM_PHYSNET_NONEXISTENT that will
# not be provided by any RP on this host
resource_request = {
"resources": {
orc.NET_BW_IGR_KILOBIT_PER_SEC: 1000,
orc.NET_BW_EGR_KILOBIT_PER_SEC: 1000},
"required": ["CUSTOM_PHYSNET_NONEXISTENT",
"CUSTOM_VNIC_TYPE_NORMAL"]
}
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1], resource_request)
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
self.assertIn(
'No matching resource provider is available for healing the port '
'allocation',
self.output.getvalue())
self.assertEqual(3, result)
def test_heal_port_allocation_ambiguous_rps(self):
"""Test that there are more than one matching RPs are available on the
compute.
"""
# The port will request CUSTOM_VNIC_TYPE_DIRECT trait and there are
# two RPs that supports such trait.
resource_request = {
"resources": {
orc.NET_BW_IGR_KILOBIT_PER_SEC: 1000,
orc.NET_BW_EGR_KILOBIT_PER_SEC: 1000},
"required": ["CUSTOM_PHYSNET2",
"CUSTOM_VNIC_TYPE_DIRECT"]
}
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1], resource_request)
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
self.assertIn(
'More than one matching resource provider',
self.output.getvalue())
self.assertEqual(3, result)
def test_heal_port_allocation_neutron_unavailable_during_port_query(self):
"""Test that Neutron is not available when querying ports.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
with mock.patch.object(
self.neutron, "list_ports",
side_effect=neutron_client_exc.Unauthorized()):
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
self.assertIn(
'Unable to query ports for instance',
self.output.getvalue())
self.assertEqual(5, result)
def test_heal_port_allocation_neutron_unavailable(self):
"""Test that the port cannot be updated in Neutron with RP uuid as
Neutron is unavailable.
"""
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
with mock.patch.object(
self.neutron, "update_port",
side_effect=neutron_client_exc.Forbidden()):
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
self.assertIn(
'Unable to update ports with allocations',
self.output.getvalue())
self.assertEqual(6, result)
def test_heal_multiple_port_allocations_rollback_success(self):
"""Test neutron port update rollback happy case. Try to heal two ports
and make the second port update to fail in neutron. Assert that the
first port update rolled back successfully.
"""
port2 = self.neutron.create_port()['port']
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1, port2])
orig_update_port = self.neutron.update_port
update = []
def fake_update_port(*args, **kwargs):
if len(update) == 0 or len(update) > 1:
update.append(True)
return orig_update_port(*args, **kwargs)
if len(update) == 1:
update.append(True)
raise neutron_client_exc.Forbidden()
with mock.patch.object(
self.neutron, "update_port", side_effect=fake_update_port):
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
# Actually one of the ports were updated but the update is rolled
# back when the second neutron port update failed
self._assert_ports_not_updated(ports)
output = self.output.getvalue()
self.assertIn(
'Rolling back port update',
output)
self.assertIn(
'Unable to update ports with allocations',
output)
self.assertEqual(6, result)
def test_heal_multiple_port_allocations_rollback_fails(self):
"""Test neutron port update rollback error case. Try to heal three
ports and make the last port update to fail in neutron. Also make the
rollback of the second port update to fail.
"""
port2 = self.neutron.create_port()['port']
port3 = self.neutron.create_port(port2)['port']
server, _ = self._create_server_with_missing_port_alloc(
[self.neutron.port_1, port2, port3])
orig_update_port = self.neutron.update_port
port_updates = []
def fake_update_port(port_id, *args, **kwargs):
# 0, 1: the first two update operation succeeds
# 4: the last rollback operation succeeds
if len(port_updates) in [0, 1, 4]:
port_updates.append(port_id)
return orig_update_port(port_id, *args, **kwargs)
# 2 : last update operation fails
# 3 : the first rollback operation also fails
if len(port_updates) in [2, 3]:
port_updates.append(port_id)
raise neutron_client_exc.Forbidden()
with mock.patch.object(
self.neutron, "update_port",
side_effect=fake_update_port) as mock_update_port:
# let's trigger a heal
result = self.cli.heal_allocations(verbose=True, max_count=2)
self.assertEqual(5, mock_update_port.call_count)
self._assert_placement_not_updated(server)
# the order of the ports is random due to usage of dicts so we
# need the info from the fake_update_port that which port update
# failed
# the first port update was successful, this will be the first port to
# rollback too and the rollback will fail
self._assert_port_updated(port_updates[0])
# the second port update was successful, this will be the second port
# to rollback which will succeed
self._assert_port_not_updated(port_updates[1])
# the third port was never updated successfully
self._assert_port_not_updated(port_updates[2])
output = self.output.getvalue()
self.assertIn(
'Rolling back port update',
output)
self.assertIn(
'Failed to update neutron ports with allocation keys and the '
'automatic rollback of the previously successful port updates '
'also failed',
output)
# as we failed to roll back the first port update we instruct the user
# to clean it up manually
self.assertIn(
"Make sure that the binding:profile.allocation key of the "
"affected ports ['%s'] are manually cleaned in neutron"
% port_updates[0],
output)
self.assertEqual(7, result)
def _test_heal_port_allocation_placement_unavailable(
self, server, ports, error):
with mock.patch('nova.cmd.manage.PlacementCommands.'
'_get_rps_in_tree_with_required_traits',
side_effect=error):
result = self.cli.heal_allocations(verbose=True, max_count=2)
self._assert_placement_not_updated(server)
self._assert_ports_not_updated(ports)
self.assertEqual(3, result)
def test_heal_port_allocation_placement_unavailable(self):
server, ports = self._create_server_with_missing_port_alloc(
[self.neutron.port_1])
for error in [
exception.PlacementAPIConnectFailure(),
exception.ResourceProviderRetrievalFailed(uuid=uuidsentinel.rp1),
exception.ResourceProviderTraitRetrievalFailed(
uuid=uuidsentinel.rp1)]:
self._test_heal_port_allocation_placement_unavailable(
server, ports, error)
class TestNovaManagePlacementSyncAggregates(
integrated_helpers.ProviderUsageBaseTestCase):
"""Functional tests for nova-manage placement sync_aggregates"""

View File

@ -2412,6 +2412,8 @@ class TestNovaManagePlacement(test.NoDBTestCase):
self.output = StringIO()
self.useFixture(fixtures.MonkeyPatch('sys.stdout', self.output))
self.cli = manage.PlacementCommands()
self.useFixture(
fixtures.MockPatch('nova.network.neutronv2.api.get_client'))
@ddt.data(-1, 0, "one")
def test_heal_allocations_invalid_max_count(self, max_count):
@ -2469,7 +2471,7 @@ class TestNovaManagePlacement(test.NoDBTestCase):
@mock.patch('nova.objects.ComputeNode.get_by_host_and_nodename',
return_value=objects.ComputeNode(uuid=uuidsentinel.node))
@mock.patch('nova.scheduler.utils.resources_from_flavor',
return_value=mock.sentinel.resources)
return_value={'VCPU': 1})
@mock.patch('nova.scheduler.client.report.SchedulerReportClient.put',
return_value=fake_requests.FakeResponse(
500, content=jsonutils.dumps({"errors": [{"code": ""}]})))
@ -2487,7 +2489,7 @@ class TestNovaManagePlacement(test.NoDBTestCase):
expected_payload = {
'allocations': {
uuidsentinel.node: {
'resources': mock.sentinel.resources
'resources': {'VCPU': 1}
}
},
'user_id': 'fake-user',
@ -2789,6 +2791,119 @@ class TestNovaManagePlacement(test.NoDBTestCase):
self.output.getvalue())
self.assertIn("Conflict!", self.output.getvalue())
def test_has_request_but_no_allocation(self):
# False because there is a full resource_request and allocation set.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.healed,
'resource_request': {
'resources': {
'NET_BW_EGR_KILOBIT_PER_SEC': 1000,
},
'required': [
'CUSTOM_VNIC_TYPE_NORMAL'
]
},
'binding:profile': {'allocation': uuidsentinel.rp1}
}))
# True because there is a full resource_request but no allocation set.
self.assertTrue(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.needs_healing,
'resource_request': {
'resources': {
'NET_BW_EGR_KILOBIT_PER_SEC': 1000,
},
'required': [
'CUSTOM_VNIC_TYPE_NORMAL'
]
},
'binding:profile': {}
}))
# True because there is a full resource_request but no allocation set.
self.assertTrue(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.needs_healing_null_profile,
'resource_request': {
'resources': {
'NET_BW_EGR_KILOBIT_PER_SEC': 1000,
},
'required': [
'CUSTOM_VNIC_TYPE_NORMAL'
]
},
'binding:profile': None,
}))
# False because there are no resources in the resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.empty_resources,
'resource_request': {
'resources': {},
'required': [
'CUSTOM_VNIC_TYPE_NORMAL'
]
},
'binding:profile': {}
}))
# False because there are no resources in the resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.missing_resources,
'resource_request': {
'required': [
'CUSTOM_VNIC_TYPE_NORMAL'
]
},
'binding:profile': {}
}))
# False because there are no required traits in the resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.empty_required,
'resource_request': {
'resources': {
'NET_BW_EGR_KILOBIT_PER_SEC': 1000,
},
'required': []
},
'binding:profile': {}
}))
# False because there are no required traits in the resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.missing_required,
'resource_request': {
'resources': {
'NET_BW_EGR_KILOBIT_PER_SEC': 1000,
},
},
'binding:profile': {}
}))
# False because there are no resources or required traits in the
# resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.empty_resource_request,
'resource_request': {},
'binding:profile': {}
}))
# False because there is no resource_request.
self.assertFalse(
self.cli._has_request_but_no_allocation(
{
'id': uuidsentinel.missing_resource_request,
'binding:profile': {}
}))
class TestNovaManageMain(test.NoDBTestCase):
"""Tests the nova-manage:main() setup code."""

View File

@ -0,0 +1,9 @@
---
other:
- |
The ``nova-manage placement heal_allocations`` `CLI`_ has been extended to
heal missing port allocations which are possible due to `bug 1819923`_ .
.. _bug 1819923: https://bugs.launchpad.net/nova/+bug/1819923
.. _CLI: https://docs.openstack.org/nova/latest/cli/nova-manage.html#placement