Replace blind retry with libvirt event waiting in detach

Nova so far applied a retry loop that tried to periodically detach the
device from libvirt while the device was visible in the domain xml. This
could lead to an issue where an already progressing detach on the
libvirt side is interrupted by nova re-sending the detach request for
the same device. See bug #1882521 for more information.

Also if there was both a persistent and a live domain the nova tried the
detach from both at the same call. This lead to confusion about the
result when such call failed. Was the detach failed partially?

We can do better, at least for the live detach case. Based on the
libvirt developers detaching from the persistent domain always
succeeds and it is a synchronous process. Detaching from the live
domain can be both synchronous or asynchronous depending on the guest
OS and the load on the hypervisor. But for live detach libvirt always
sends an event [1] nova can wait for.

So this patch does two things.

1) Separates the detach from the persistent domain from the detach from
   the live domain to make the error cases clearer.

2) Changes the retry mechanism.

   Detaching from the persistent domain is not retried. If libvirt
   reports device not found, while both persistent and live detach
   is needed, the error is ignored, and the process continues with
   the live detach. In any other case the error considered as fatal.

   Detaching from the live domain is changed to always wait for the
   libvirt event. In case of timeout, the live detach is retried.
   But a failure event from libvirt considered fatal, based on the
   information from the libvirt developers, so in this case the
   detach is not retried.

Related-Bug: #1882521

[1]https://libvirt.org/html/libvirt-libvirt-domain.html#virConnectDomainEventDeviceRemovedCallback

Change-Id: I7f2b6330decb92e2838aa7cee47fb228f00f47da
This commit is contained in:
Balazs Gibizer 2021-01-12 08:23:17 +01:00
parent 74d6fc4c28
commit e56cc4f439
8 changed files with 1144 additions and 385 deletions

View File

@ -875,6 +875,30 @@ Related options:
* It's recommended to consider including ``x86_64=q35`` in
:oslo.config:option:`libvirt.hw_machine_type`; see
:ref:`deploying-sev-capable-infrastructure` for more on this.
"""),
cfg.IntOpt('device_detach_attempts',
default=8,
min=1,
help="""
Maximum number of attempts the driver tries to detach a device in libvirt.
Related options:
* :oslo.config:option:`libvirt.device_detach_timeout`
"""),
cfg.IntOpt('device_detach_timeout',
default=20,
min=1,
help="""
Maximum number of seconds the driver waits for the success or the failure
event from libvirt for a given device detach attempt before it re-trigger the
detach.
Related options:
* :oslo.config:option:`libvirt.device_detach_attempts`
"""),
]

View File

@ -1278,9 +1278,17 @@ class Domain(object):
self.attachDevice(xml)
def detachDevice(self, xml):
# TODO(gibi): this should handle nics similarly to attachDevice()
disk_info = _parse_disk_info(etree.fromstring(xml))
disk_info['_attached'] = True
return disk_info in self._def['devices']['disks']
attached_disk_info = None
for attached_disk in self._def['devices']['disks']:
if attached_disk['target_dev'] == disk_info.get('target_dev'):
attached_disk_info = attached_disk
break
if attached_disk_info:
self._def['devices']['disks'].remove(attached_disk_info)
return attached_disk_info is not None
def detachDeviceFlags(self, xml, flags):
self.detachDevice(xml)

File diff suppressed because it is too large Load Diff

View File

@ -21,7 +21,6 @@ from oslo_service import fixture as service_fixture
from oslo_utils import encodeutils
from nova import context
from nova import exception
from nova import test
from nova.tests.unit.virt.libvirt import fakelibvirt
from nova.virt.libvirt import config as vconfig
@ -213,212 +212,6 @@ class GuestTestCase(test.NoDBTestCase):
"</xml>", flags=(fakelibvirt.VIR_DOMAIN_AFFECT_CONFIG |
fakelibvirt.VIR_DOMAIN_AFFECT_LIVE))
def test_detach_device_with_retry_from_transient_domain(self):
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
get_config = mock.Mock()
get_config.side_effect = [conf, conf, conf, None, None]
dev_path = "/dev/vdb"
self.domain.isPersistent.return_value = False
retry_detach = self.guest.detach_device_with_retry(
get_config, dev_path, live=True, inc_sleep_time=.01)
self.domain.detachDeviceFlags.assert_called_once_with(
"</xml>", flags=fakelibvirt.VIR_DOMAIN_AFFECT_LIVE)
self.domain.detachDeviceFlags.reset_mock()
retry_detach()
self.assertEqual(1, self.domain.detachDeviceFlags.call_count)
def test_detach_device_with_retry_detach_success(self):
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
get_config = mock.Mock()
# Force multiple retries of detach
get_config.side_effect = [conf, conf, conf, conf, conf, None, None]
dev_path = "/dev/vdb"
self.domain.isPersistent.return_value = True
retry_detach = self.guest.detach_device_with_retry(
get_config, dev_path, live=True, inc_sleep_time=.01)
# Ensure we've only done the initial detach call
self.domain.detachDeviceFlags.assert_called_once_with(
"</xml>", flags=(fakelibvirt.VIR_DOMAIN_AFFECT_CONFIG |
fakelibvirt.VIR_DOMAIN_AFFECT_LIVE))
get_config.assert_called_with(dev_path)
# Some time later, we can do the wait/retry to ensure detach succeeds
self.domain.detachDeviceFlags.reset_mock()
retry_detach()
# Should have two retries before we pretend device is detached
self.assertEqual(2, self.domain.detachDeviceFlags.call_count)
def test_detach_device_with_retry_detach_failure(self):
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
# Continue to return some value for the disk config
get_config = mock.Mock(return_value=conf)
self.domain.isPersistent.return_value = True
retry_detach = self.guest.detach_device_with_retry(
get_config, "/dev/vdb", live=True, inc_sleep_time=.01,
max_retry_count=3)
# Ensure we've only done the initial detach call
self.domain.detachDeviceFlags.assert_called_once_with(
"</xml>", flags=(fakelibvirt.VIR_DOMAIN_AFFECT_CONFIG |
fakelibvirt.VIR_DOMAIN_AFFECT_LIVE))
# Some time later, we can do the wait/retry to ensure detach
self.domain.detachDeviceFlags.reset_mock()
# Should hit max # of retries
self.assertRaises(exception.DeviceDetachFailed, retry_detach)
self.assertEqual(4, self.domain.detachDeviceFlags.call_count)
def test_detach_device_with_retry_device_not_found(self):
get_config = mock.Mock(return_value=None)
self.domain.isPersistent.return_value = True
ex = self.assertRaises(
exception.DeviceNotFound, self.guest.detach_device_with_retry,
get_config, "/dev/vdb", live=True)
self.assertIn("/dev/vdb", str(ex))
def test_detach_device_with_retry_device_not_found_alt_name(self):
"""Tests to make sure we use the alternative name in errors."""
get_config = mock.Mock(return_value=None)
self.domain.isPersistent.return_value = True
ex = self.assertRaises(
exception.DeviceNotFound, self.guest.detach_device_with_retry,
get_config, mock.sentinel.device, live=True,
alternative_device_name='foo')
self.assertIn('foo', str(ex))
@mock.patch.object(libvirt_guest.Guest, "detach_device")
def _test_detach_device_with_retry_second_detach_failure(
self, mock_detach, error_code=None, error_message=None,
supports_device_missing=False):
# This simulates a retry of the transient/live domain detach
# failing because the device is not found
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
self.domain.isPersistent.return_value = True
get_config = mock.Mock(return_value=conf)
fake_device = "vdb"
fake_exc = fakelibvirt.make_libvirtError(
fakelibvirt.libvirtError, "",
error_message=error_message,
error_code=error_code,
error_domain=fakelibvirt.VIR_FROM_DOMAIN)
mock_detach.side_effect = [None, fake_exc]
retry_detach = self.guest.detach_device_with_retry(
get_config, fake_device, live=True,
inc_sleep_time=.01, max_retry_count=3)
# Some time later, we can do the wait/retry to ensure detach
self.assertRaises(exception.DeviceNotFound, retry_detach)
# Check that the save_and_reraise_exception context manager didn't log
# a traceback when the libvirtError was caught and DeviceNotFound was
# raised.
self.assertNotIn('Original exception being dropped',
self.stdlog.logger.output)
def test_detach_device_with_retry_second_detach_device_missing(self):
self._test_detach_device_with_retry_second_detach_failure(
error_code=fakelibvirt.VIR_ERR_DEVICE_MISSING,
error_message="device not found: disk vdb not found",
supports_device_missing=True)
def _test_detach_device_with_retry_first_detach_failure(
self, error_code=None, error_message=None,
supports_device_missing=False):
# This simulates a persistent or live domain detach failing because the
# device is not found during the first attempt to detach the device.
# We should still attempt to detach the device from the live config if
# the detach from persistent failed OR we should retry the detach from
# the live config if the first detach from live config failed.
# Note that the side effects in this test case [fake_exc, None] could
# not happen in real life if the first detach failed because the detach
# from live raised not found. In real life, the second attempt to
# detach from live would raise not found again because the device is
# not present. The purpose of this test is to verify that we try to
# detach a second time if the first detach fails, so we are OK with the
# unrealistic side effects for detach from live failing the first time.
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
self.domain.isPersistent.return_value = True
get_config = mock.Mock()
# Simulate an inactive or live detach attempt which fails (not found)
# followed by a live config detach attempt that is successful
get_config.side_effect = [conf, conf, conf, None, None]
fake_device = "vdb"
fake_exc = fakelibvirt.make_libvirtError(
fakelibvirt.libvirtError, "",
error_message=error_message,
error_code=error_code,
error_domain=fakelibvirt.VIR_FROM_DOMAIN)
# Detach from persistent or live raises not found, detach from live
# succeeds afterward
self.domain.detachDeviceFlags.side_effect = [fake_exc, None]
retry_detach = self.guest.detach_device_with_retry(get_config,
fake_device, live=True, inc_sleep_time=.01, max_retry_count=3)
# We should have tried to detach from the persistent domain
self.domain.detachDeviceFlags.assert_called_once_with(
"</xml>", flags=(fakelibvirt.VIR_DOMAIN_AFFECT_CONFIG |
fakelibvirt.VIR_DOMAIN_AFFECT_LIVE))
# During the retry detach, should detach from the live domain
self.domain.detachDeviceFlags.reset_mock()
retry_detach()
# We should have tried to detach from the live domain
self.domain.detachDeviceFlags.assert_called_once_with(
"</xml>", flags=fakelibvirt.VIR_DOMAIN_AFFECT_LIVE)
def test_detach_device_with_retry_first_detach_device_missing(self):
self._test_detach_device_with_retry_first_detach_failure(
error_code=fakelibvirt.VIR_ERR_DEVICE_MISSING,
error_message="device not found: disk vdb not found",
supports_device_missing=True)
def test_detach_device_with_already_in_process_of_unplug_error(self):
# Assert that DeviceNotFound is raised when encountering
# https://bugzilla.redhat.com/show_bug.cgi?id=1878659
# This is raised as QEMU returns a VIR_ERR_INTERNAL_ERROR when
# a request to device_del is made while another is about to complete.
self.domain.isPersistent.return_value = True
conf = mock.Mock(spec=vconfig.LibvirtConfigGuestDevice)
conf.to_xml.return_value = "</xml>"
existing_unplug_exc = fakelibvirt.make_libvirtError(
fakelibvirt.libvirtError, "",
error_message='device vdb is already in the process of unplug',
error_code=fakelibvirt.VIR_ERR_INTERNAL_ERROR,
error_domain=fakelibvirt.VIR_FROM_DOMAIN
)
device_missing_exc = fakelibvirt.make_libvirtError(
fakelibvirt.libvirtError, "",
error_message='device not found: disk vdb not found',
error_code=fakelibvirt.VIR_ERR_DEVICE_MISSING,
error_domain=fakelibvirt.VIR_FROM_DOMAIN
)
# Raise VIR_ERR_INTERNAL_ERROR on the second call before raising
# VIR_ERR_DEVICE_MISSING to mock the first call successfully detaching
# the device asynchronously.
self.domain.detachDeviceFlags.side_effect = [
None,
existing_unplug_exc,
device_missing_exc
]
retry_detach = self.guest.detach_device_with_retry(
mock.Mock(return_value=conf),
'vdb',
live=True,
inc_sleep_time=.01
)
# Assert that we raise exception.DeviceNotFound
self.assertRaises(exception.DeviceNotFound, retry_detach)
def test_get_xml_desc(self):
self.guest.get_xml_desc()
self.domain.XMLDesc.assert_called_once_with(flags=0)

View File

@ -416,6 +416,7 @@ class _VirtDriverTestCase(_FakeDriverBackendTestCase):
self.assertEqual(storage_ip, result['ip'])
@catch_notimplementederror
@mock.patch('threading.Event.wait', new=mock.Mock())
@mock.patch.object(libvirt.driver.LibvirtDriver, '_build_device_metadata',
return_value=objects.InstanceDeviceMetadata())
def test_attach_detach_volume(self, _):
@ -452,6 +453,7 @@ class _VirtDriverTestCase(_FakeDriverBackendTestCase):
'/dev/sda', 2))
@catch_notimplementederror
@mock.patch('threading.Event.wait', new=mock.Mock())
@mock.patch.object(libvirt.driver.LibvirtDriver, '_build_device_metadata',
return_value=objects.InstanceDeviceMetadata())
def test_attach_detach_different_power_states(self, _):

View File

@ -504,6 +504,10 @@ class LibvirtDriver(driver.ComputeDriver):
self.pgpu_type_mapping = collections.defaultdict(str)
self.supported_vgpu_types = self._get_supported_vgpu_types()
# Handles ongoing device manipultion in libvirt where we wait for the
# events about success or failure.
self._device_event_handler = AsyncDeviceEventsHandler()
def _discover_vpmems(self, vpmem_conf=None):
"""Discover vpmems on host and configuration.
@ -1400,6 +1404,9 @@ class LibvirtDriver(driver.ComputeDriver):
def destroy(self, context, instance, network_info, block_device_info=None,
destroy_disks=True):
self._destroy(instance)
# NOTE(gibi): if there was device detach in progress then we need to
# unblock the waiting threads and clean up.
self._device_event_handler.cleanup_waiters(instance.uuid)
self.cleanup(context, instance, network_info, block_device_info,
destroy_disks)
@ -2189,8 +2196,20 @@ class LibvirtDriver(driver.ComputeDriver):
# These are libvirt specific events handled here on the driver
# level instead of propagating them to the compute manager level
if isinstance(event, libvirtevent.DeviceEvent):
# TODO(gibi): handle it
pass
had_clients = self._device_event_handler.notify_waiters(event)
if had_clients:
LOG.debug(
"Received event %s from libvirt while the driver is "
"waiting for it; dispatched.",
event,
)
else:
LOG.warning(
"Received event %s from libvirt but the driver is not "
"waiting for it; ignored.",
event,
)
else:
LOG.debug(
"Received event %s from libvirt but no handler is "
@ -2201,6 +2220,321 @@ class LibvirtDriver(driver.ComputeDriver):
# manager
super().emit_event(event)
def _detach_with_retry(
self,
guest: libvirt_guest.Guest,
instance_uuid: str,
# to properly typehint this param we would need typing.Protocol but
# that is only available since python 3.8
get_device_conf_func: ty.Callable,
device_name: str,
live: bool,
) -> None:
"""Detaches a device from the guest
If live detach is requested then this call will wait for the libvirt
event signalling the end of the detach process.
If the live detach times out then it will retry the detach. Detach from
the persistent config is not retried as it is:
* synchronous and no event is sent from libvirt
* it is always expected to succeed if the device is in the domain
config
:param guest: the guest we are detach the device from
:param instance_uuid: the UUID of the instance we are detaching the
device from
:param get_device_conf_func: function which returns the configuration
for device from the domain, having one optional boolean parameter
`from_persistent_config` to select which domain config to query
:param device_name: This is the name of the device used solely for
error messages. Note that it is not the same as the device alias
used by libvirt to identify the device.
:param live: bool to indicate whether it affects the guest in running
state. If live is True then the device is detached from both the
persistent and the live config. If live is False the device only
detached from the persistent config.
:raises exception.DeviceNotFound: if the device does not exist in the
domain even before we try to detach or if libvirt reported that the
device is missing from the domain synchronously.
:raises exception.DeviceDetachFailed: if libvirt reported error during
detaching from the live domain or we timed out waiting for libvirt
events and run out of retries
:raises libvirt.libvirtError: for any other errors reported by libvirt
synchronously.
"""
persistent = guest.has_persistent_configuration()
if not persistent and not live:
# nothing to do
return
persistent_dev = None
if persistent:
persistent_dev = get_device_conf_func(from_persistent_config=True)
live_dev = None
if live:
live_dev = get_device_conf_func()
if live and live_dev is None:
# caller requested a live detach but device is not present
raise exception.DeviceNotFound(device=device_name)
if not live and persistent_dev is None:
# caller requested detach from the persistent domain but device is
# not present
raise exception.DeviceNotFound(device=device_name)
if persistent_dev:
try:
self._detach_from_persistent(
guest, instance_uuid, persistent_dev, get_device_conf_func,
device_name)
except exception.DeviceNotFound:
if live:
# ignore the error so that we can do the live detach
LOG.warning(
'Libvirt reported sync error while detaching '
'device %s from instance %s from the persistent '
'domain config. Ignoring the error to proceed with '
'live detach as that was also requested.',
device_name, instance_uuid)
else:
# if only persistent detach was requested then give up
raise
if live and live_dev:
self._detach_from_live_with_retry(
guest, instance_uuid, live_dev, get_device_conf_func,
device_name)
def _detach_from_persistent(
self,
guest: libvirt_guest.Guest,
instance_uuid: str,
persistent_dev: ty.Union[
vconfig.LibvirtConfigGuestDisk,
vconfig.LibvirtConfigGuestInterface],
get_device_conf_func,
device_name: str,
):
LOG.debug(
'Attempting to detach device %s from instance %s from '
'the persistent domain config.', device_name, instance_uuid)
self._detach_sync(
persistent_dev, guest, instance_uuid, device_name,
persistent=True, live=False)
# make sure the dev is really gone
persistent_dev = get_device_conf_func(
from_persistent_config=True)
if not persistent_dev:
LOG.info(
'Successfully detached device %s from instance %s '
'from the persistent domain config.',
device_name, instance_uuid)
else:
# Based on the libvirt devs this should never happen
LOG.warning(
'Failed to detach device %s from instance %s '
'from the persistent domain config. Libvirt did not '
'report any error but the device is still in the '
'config.', device_name, instance_uuid)
def _detach_from_live_with_retry(
self,
guest: libvirt_guest.Guest,
instance_uuid: str,
live_dev: ty.Union[
vconfig.LibvirtConfigGuestDisk,
vconfig.LibvirtConfigGuestInterface],
get_device_conf_func,
device_name: str,
):
max_attempts = CONF.libvirt.device_detach_attempts
for attempt in range(max_attempts):
LOG.debug(
'(%s/%s): Attempting to detach device %s with device '
'alias %s from instance %s from the live domain config.',
attempt + 1, max_attempts, device_name, live_dev.alias,
instance_uuid)
self._detach_from_live_and_wait_for_event(
live_dev, guest, instance_uuid, device_name)
# make sure the dev is really gone
live_dev = get_device_conf_func()
if not live_dev:
LOG.info(
'Successfully detached device %s from instance %s '
'from the live domain config.', device_name, instance_uuid)
# we are done
return
LOG.debug(
'Failed to detach device %s with device alias %s from '
'instance %s from the live domain config. Libvirt did not '
'report any error but the device is still in the config.',
device_name, live_dev.alias, instance_uuid)
msg = (
'Run out of retry while detaching device %s with device '
'alias %s from instance %s from the live domain config. '
'Device is still attached to the guest.')
LOG.error(msg, device_name, live_dev.alias, instance_uuid)
raise exception.DeviceDetachFailed(
device=device_name,
reason=msg % (device_name, live_dev.alias, instance_uuid))
def _detach_from_live_and_wait_for_event(
self,
dev: ty.Union[
vconfig.LibvirtConfigGuestDisk,
vconfig.LibvirtConfigGuestInterface],
guest: libvirt_guest.Guest,
instance_uuid: str,
device_name: str,
) -> None:
"""Detaches a device from the live config of the guest and waits for
the libvirt event singling the finish of the detach.
:param dev: the device configuration to be detached
:param guest: the guest we are detach the device from
:param instance_uuid: the UUID of the instance we are detaching the
device from
:param device_name: This is the name of the device used solely for
error messages.
:raises exception.DeviceNotFound: if libvirt reported that the device
is missing from the domain synchronously.
:raises libvirt.libvirtError: for any other errors reported by libvirt
synchronously.
:raises DeviceDetachFailed: if libvirt sent DeviceRemovalFailedEvent
"""
# So we will issue an detach to libvirt and we will wait for an
# event from libvirt about the result. We need to set up the event
# handling before the detach to avoid missing the event if libvirt
# is really fast
# NOTE(gibi): we need to use the alias name of the device as that
# is what libvirt will send back to us in the event
waiter = self._device_event_handler.create_waiter(
instance_uuid, dev.alias,
{libvirtevent.DeviceRemovedEvent,
libvirtevent.DeviceRemovalFailedEvent})
try:
self._detach_sync(
dev, guest, instance_uuid, device_name, persistent=False,
live=True)
except Exception:
# clean up the libvirt event handler as we failed synchronously
self._device_event_handler.delete_waiter(waiter)
raise
LOG.debug(
'Start waiting for the detach event from libvirt for '
'device %s with device alias %s for instance %s',
device_name, dev.alias, instance_uuid)
# We issued the detach without any exception so we can wait for
# a libvirt event to arrive to notify us about the result
# NOTE(gibi): we expect that this call will be unblocked by an
# incoming libvirt DeviceRemovedEvent or DeviceRemovalFailedEvent
event = self._device_event_handler.wait(
waiter, timeout=CONF.libvirt.device_detach_timeout)
if not event:
# This should not happen based on information from the libvirt
# developers. But it does at least during the cleanup of the
# tempest test case
# ServerRescueNegativeTestJSON.test_rescued_vm_detach_volume
# Log a warning and let the upper layer detect that the device is
# still attached and retry
LOG.error(
'Waiting for libvirt event about the detach of '
'device %s with device alias %s from instance %s is timed '
'out.', device_name, dev.alias, instance_uuid)
if isinstance(event, libvirtevent.DeviceRemovalFailedEvent):
# Based on the libvirt developers this signals a permanent failure
LOG.error(
'Received DeviceRemovalFailedEvent from libvirt for the '
'detach of device %s with device alias %s from instance %s ',
device_name, dev.alias, instance_uuid)
raise exception.DeviceDetachFailed(
device=device_name,
reason="DeviceRemovalFailedEvent received from libvirt")
@staticmethod
def _detach_sync(
dev: ty.Union[
vconfig.LibvirtConfigGuestDisk,
vconfig.LibvirtConfigGuestInterface],
guest: libvirt_guest.Guest,
instance_uuid: str,
device_name: str,
persistent: bool,
live: bool,
):
"""Detaches a device from the guest without waiting for libvirt events
It only handles synchronous errors (i.e. exceptions) but does not wait
for any event from libvirt.
:param dev: the device configuration to be detached
:param guest: the guest we are detach the device from
:param instance_uuid: the UUID of the instance we are detaching the
device from
:param device_name: This is the name of the device used solely for
error messages.
:param live: detach the device from the live domain config only
:param persistent: detach the device from the persistent domain config
only
:raises exception.DeviceNotFound: if libvirt reported that the device
is missing from the domain synchronously.
:raises libvirt.libvirtError: for any other errors reported by libvirt
synchronously.
"""
try:
guest.detach_device(dev, persistent=persistent, live=live)
except libvirt.libvirtError as ex:
code = ex.get_error_code()
msg = ex.get_error_message()
if code == libvirt.VIR_ERR_DEVICE_MISSING:
LOG.debug(
'Libvirt failed to detach device %s from instance %s '
'synchronously (persistent=%s, live=%s) with error: %s.',
device_name, instance_uuid, persistent, live, str(ex))
raise exception.DeviceNotFound(device=device_name) from ex
# NOTE(lyarwood): https://bugzilla.redhat.com/1878659
# Ignore this known QEMU bug for the time being allowing
# our retry logic to handle it.
# NOTE(gibi): This can only happen in case of detaching from the
# live domain as we never retry a detach from the persistent
# domain so we cannot hit an already running detach there.
# In case of detaching from the live domain this error can happen
# if the caller timed out during the first detach attempt then saw
# that the device is still attached and therefore looped over and
# and retried the detach. In this case the previous attempt stopped
# waiting for the libvirt event. Also libvirt reports that there is
# a detach ongoing, so the current attempt expects that a
# libvirt event will be still emitted. Therefore we simply return
# from here. Then the caller will wait for such event.
if (code == libvirt.VIR_ERR_INTERNAL_ERROR and msg and
'already in the process of unplug' in msg
):
LOG.debug(
'Ignoring QEMU rejecting our request to detach device %s '
'from instance %s as it is caused by a previous request '
'still being in progress.', device_name, instance_uuid)
return
LOG.warning(
'Unexpected libvirt error while detaching device %s from '
'instance %s: %s', device_name, instance_uuid, str(ex))
raise
def detach_volume(self, context, connection_info, instance, mountpoint,
encryption=None):
disk_dev = mountpoint.rpartition("/")[2]
@ -2213,10 +2547,14 @@ class LibvirtDriver(driver.ComputeDriver):
# detaching any attached encryptors or disconnecting the underlying
# volume in _disconnect_volume. Otherwise, the encryptor or volume
# driver may report that the volume is still in use.
wait_for_detach = guest.detach_device_with_retry(
guest.get_disk, disk_dev, live=live)
wait_for_detach()
get_dev = functools.partial(guest.get_disk, disk_dev)
self._detach_with_retry(
guest,
instance.uuid,
get_dev,
device_name=disk_dev,
live=live
)
except exception.InstanceNotFound:
# NOTE(zhaoqin): If the instance does not exist, _lookup_by_name()
# will throw InstanceNotFound exception. Need to
@ -2423,12 +2761,14 @@ class LibvirtDriver(driver.ComputeDriver):
state = guest.get_power_state(self._host)
live = state in (power_state.RUNNING, power_state.PAUSED)
# Now we are going to loop until the interface is detached or we
# timeout.
wait_for_detach = guest.detach_device_with_retry(
guest.get_interface_by_cfg, cfg, live=live,
alternative_device_name=self.vif_driver.get_vif_devname(vif))
wait_for_detach()
get_dev = functools.partial(guest.get_interface_by_cfg, cfg)
self._detach_with_retry(
guest,
instance.uuid,
get_dev,
device_name=self.vif_driver.get_vif_devname(vif),
live=live,
)
except exception.DeviceDetachFailed:
# We failed to detach the device even with the retry loop, so let's
# dump some debug information to the logs before raising back up.

View File

@ -31,7 +31,6 @@ import time
from lxml import etree
from oslo_log import log as logging
from oslo_service import loopingcall
from oslo_utils import encodeutils
from oslo_utils import excutils
from oslo_utils import importutils
@ -399,109 +398,6 @@ class Guest(object):
devs.append(dev)
return devs
def detach_device_with_retry(self, get_device_conf_func, device, live,
max_retry_count=7, inc_sleep_time=10,
max_sleep_time=60,
alternative_device_name=None):
"""Detaches a device from the guest. After the initial detach request,
a function is returned which can be used to ensure the device is
successfully removed from the guest domain (retrying the removal as
necessary).
:param get_device_conf_func: function which takes device as a parameter
and returns the configuration for device
:param device: device to detach
:param live: bool to indicate whether it affects the guest in running
state
:param max_retry_count: number of times the returned function will
retry a detach before failing
:param inc_sleep_time: incremental time to sleep in seconds between
detach retries
:param max_sleep_time: max sleep time in seconds beyond which the sleep
time will not be incremented using param
inc_sleep_time. On reaching this threshold,
max_sleep_time will be used as the sleep time.
:param alternative_device_name: This is an alternative identifier for
the device if device is not an ID, used solely for error messages.
"""
alternative_device_name = alternative_device_name or device
def _try_detach_device(conf, persistent=False, live=False):
# Raise DeviceNotFound if the device isn't found during detach
try:
self.detach_device(conf, persistent=persistent, live=live)
if get_device_conf_func(device) is None:
LOG.debug('Successfully detached device %s from guest. '
'Persistent? %s. Live? %s',
device, persistent, live)
except libvirt.libvirtError as ex:
with excutils.save_and_reraise_exception(reraise=False) as ctx:
code = ex.get_error_code()
msg = ex.get_error_message()
if code == libvirt.VIR_ERR_DEVICE_MISSING:
raise exception.DeviceNotFound(
device=alternative_device_name)
# NOTE(lyarwood): https://bugzilla.redhat.com/1878659
# Ignore this known QEMU bug for the time being allowing
# our retry logic to fire again and hopefully see that
# the device has been removed asynchronously by QEMU
# in the meantime when the next call to detach raises
# VIR_ERR_DEVICE_MISSING.
if (code == libvirt.VIR_ERR_INTERNAL_ERROR and
msg and 'already in the process of unplug' in msg
):
LOG.debug('Ignoring QEMU rejecting our request to '
'detach as it is caused by a previous '
'request still being in progress.')
return
# Re-raise the original exception if we're not raising
# DeviceNotFound instead. This will avoid logging of a
# "Original exception being dropped" traceback.
ctx.reraise = True
conf = get_device_conf_func(device)
if conf is None:
raise exception.DeviceNotFound(device=alternative_device_name)
persistent = self.has_persistent_configuration()
LOG.debug('Attempting initial detach for device %s',
alternative_device_name)
try:
_try_detach_device(conf, persistent, live)
except exception.DeviceNotFound:
# NOTE(melwitt): There are effectively two configs for an instance.
# The persistent config (affects instance upon next boot) and the
# live config (affects running instance). When we detach a device,
# we need to detach it from both configs if the instance has a
# persistent config and a live config. If we tried to detach the
# device with persistent=True and live=True and it was not found,
# we should still try to detach from the live config, so continue.
if persistent and live:
pass
else:
raise
LOG.debug('Start retrying detach until device %s is gone.',
alternative_device_name)
@loopingcall.RetryDecorator(max_retry_count=max_retry_count,
inc_sleep_time=inc_sleep_time,
max_sleep_time=max_sleep_time,
exceptions=exception.DeviceDetachFailed)
def _do_wait_and_retry_detach():
config = get_device_conf_func(device)
if config is not None:
# Device is already detached from persistent config
# and only the live config needs to be updated.
_try_detach_device(config, persistent=False, live=live)
reason = _("Unable to detach the device from the live config.")
raise exception.DeviceDetachFailed(
device=alternative_device_name, reason=reason)
return _do_wait_and_retry_detach
def detach_device(self, conf, persistent=False, live=False):
"""Detaches device to the guest.

View File

@ -0,0 +1,11 @@
---
fixes:
- |
To fix `device detach issues`__ in the libvirt driver the detach logic has
been changed from a sleep based retry loop to waiting for libvirt domain
events. During this change we also introduced two new config options to
allow fine tuning the retry logic. For details see the description of the
new ``[libvirt]device_detach_attempts`` and
``[libvirt]device_detach_timeout`` config options.
.. __: https://bugs.launchpad.net/nova/+bug/1882521