Import and convert to oslo loopingcall.

Import the oslo looping call implementation (which is a copy of
nova's), delete nova's local copy, convert all users to the new
location.

It should be noted that the oslo implementation of
FixedIntervalLoopingCall measures time from the start of the
periodic task, not the end, so periodic tasks will run with a
constant frequency instead of the frequency changing depending on
how long the periodic task takes to run.

Change-Id: Ia62ce1988f5373c09146efa6b3b1d1dc094d50c4
This commit is contained in:
Michael Still
2013-04-09 13:25:38 +10:00
parent 36b1038472
commit 8c53d87ad9
14 changed files with 208 additions and 156 deletions

View File

@@ -0,0 +1,147 @@
# vim: tabstop=4 shiftwidth=4 softtabstop=4
# Copyright 2010 United States Government as represented by the
# Administrator of the National Aeronautics and Space Administration.
# Copyright 2011 Justin Santa Barbara
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import sys
from eventlet import event
from eventlet import greenthread
from nova.openstack.common.gettextutils import _
from nova.openstack.common import log as logging
from nova.openstack.common import timeutils
LOG = logging.getLogger(__name__)
class LoopingCallDone(Exception):
"""Exception to break out and stop a LoopingCall.
The poll-function passed to LoopingCall can raise this exception to
break out of the loop normally. This is somewhat analogous to
StopIteration.
An optional return-value can be included as the argument to the exception;
this return-value will be returned by LoopingCall.wait()
"""
def __init__(self, retvalue=True):
""":param retvalue: Value that LoopingCall.wait() should return."""
self.retvalue = retvalue
class LoopingCallBase(object):
def __init__(self, f=None, *args, **kw):
self.args = args
self.kw = kw
self.f = f
self._running = False
self.done = None
def stop(self):
self._running = False
def wait(self):
return self.done.wait()
class FixedIntervalLoopingCall(LoopingCallBase):
"""A fixed interval looping call."""
def start(self, interval, initial_delay=None):
self._running = True
done = event.Event()
def _inner():
if initial_delay:
greenthread.sleep(initial_delay)
try:
while self._running:
start = timeutils.utcnow()
self.f(*self.args, **self.kw)
end = timeutils.utcnow()
if not self._running:
break
delay = interval - timeutils.delta_seconds(start, end)
if delay <= 0:
LOG.warn(_('task run outlasted interval by %s sec') %
-delay)
greenthread.sleep(delay if delay > 0 else 0)
except LoopingCallDone, e:
self.stop()
done.send(e.retvalue)
except Exception:
LOG.exception(_('in fixed duration looping call'))
done.send_exception(*sys.exc_info())
return
else:
done.send(True)
self.done = done
greenthread.spawn_n(_inner)
return self.done
# TODO(mikal): this class name is deprecated in Havana and should be removed
# in the I release
LoopingCall = FixedIntervalLoopingCall
class DynamicLoopingCall(LoopingCallBase):
"""A looping call which sleeps until the next known event.
The function called should return how long to sleep for before being
called again.
"""
def start(self, initial_delay=None, periodic_interval_max=None):
self._running = True
done = event.Event()
def _inner():
if initial_delay:
greenthread.sleep(initial_delay)
try:
while self._running:
idle = self.f(*self.args, **self.kw)
if not self._running:
break
if periodic_interval_max is not None:
idle = min(idle, periodic_interval_max)
LOG.debug(_('Dynamic looping call sleeping for %.02f '
'seconds'), idle)
greenthread.sleep(idle)
except LoopingCallDone, e:
self.stop()
done.send(e.retvalue)
except Exception:
LOG.exception(_('in dynamic looping call'))
done.send_exception(*sys.exc_info())
return
else:
done.send(True)
self.done = done
greenthread.spawn(_inner)
return self.done

View File

@@ -37,6 +37,7 @@ from nova import exception
from nova.openstack.common import eventlet_backdoor
from nova.openstack.common import importutils
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.openstack.common import rpc
from nova import servicegroup
from nova import utils
@@ -473,7 +474,7 @@ class Service(object):
else:
initial_delay = None
periodic = utils.DynamicLoopingCall(self.periodic_tasks)
periodic = loopingcall.DynamicLoopingCall(self.periodic_tasks)
periodic.start(initial_delay=initial_delay,
periodic_interval_max=self.periodic_interval_max)
self.timers.append(periodic)

View File

@@ -18,6 +18,7 @@ from oslo.config import cfg
from nova import conductor
from nova import context
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.openstack.common import timeutils
from nova.servicegroup import api
from nova import utils
@@ -46,7 +47,8 @@ class DbDriver(api.ServiceGroupDriver):
' ServiceGroup driver'))
report_interval = service.report_interval
if report_interval:
pulse = utils.FixedIntervalLoopingCall(self._report_state, service)
pulse = loopingcall.FixedIntervalLoopingCall(self._report_state,
service)
pulse.start(interval=report_interval,
initial_delay=report_interval)
return pulse

View File

@@ -22,10 +22,10 @@ from oslo.config import cfg
from nova import conductor
from nova import context
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.openstack.common import memorycache
from nova.openstack.common import timeutils
from nova.servicegroup import api
from nova import utils
CONF = cfg.CONF
@@ -58,7 +58,8 @@ class MemcachedDriver(api.ServiceGroupDriver):
'Memcached based ServiceGroup driver'))
report_interval = service.report_interval
if report_interval:
pulse = utils.FixedIntervalLoopingCall(self._report_state, service)
pulse = loopingcall.FixedIntervalLoopingCall(self._report_state,
service)
pulse.start(interval=report_interval,
initial_delay=report_interval)
return pulse

View File

@@ -22,8 +22,8 @@ from oslo.config import cfg
from nova import exception
from nova.openstack.common import importutils
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.servicegroup import api
from nova import utils
evzookeeper = importutils.try_import('evzookeeper')
membership = importutils.try_import('evzookeeper.membersip')
@@ -139,7 +139,7 @@ class ZooKeeperDriver(api.ServiceGroupDriver):
return all_members
class FakeLoopingCall(utils.LoopingCallBase):
class FakeLoopingCall(loopingcall.LoopingCallBase):
"""The fake Looping Call implementation, created for backward
compatibility with a membership based on DB.
"""

View File

@@ -18,6 +18,7 @@
from nova import exception
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova import utils
LOG = logging.getLogger(__name__)
@@ -77,12 +78,13 @@ def _wait_for_remove(device, tries):
devices = get_device_list()
if device["device"] not in devices:
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
def remove_device(device):
tries = 0
timer = utils.FixedIntervalLoopingCall(_wait_for_remove, device, tries)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_remove, device,
tries)
timer.start(interval=2).wait()
timer.stop()

View File

@@ -42,6 +42,7 @@ from nova import exception
from nova.openstack.common import fileutils
from nova.openstack.common import importutils
from nova.openstack.common import jsonutils
from nova.openstack.common import loopingcall
from nova.openstack.common import uuidutils
from nova import test
from nova.tests import fake_libvirt_utils
@@ -4844,7 +4845,7 @@ class LibvirtDriverTestCase(test.TestCase):
'uuid': 'not_found_uuid'})
# instance is running case
self.assertRaises(utils.LoopingCallDone,
self.assertRaises(loopingcall.LoopingCallDone,
self.libvirtconnection._wait_for_running,
{'name': 'running',
'uuid': 'running_uuid'})
@@ -4984,7 +4985,7 @@ class LibvirtDriverTestCase(test.TestCase):
self.stubs.Set(self.libvirtconnection, 'to_xml', lambda *a, **k: None)
self.stubs.Set(self.libvirtconnection, '_create_domain_and_network',
lambda *a: None)
self.stubs.Set(utils, 'FixedIntervalLoopingCall',
self.stubs.Set(loopingcall, 'FixedIntervalLoopingCall',
lambda *a, **k: FakeLoopingCall())
libvirt_utils.get_instance_path({}).AndReturn('/fake/foo')

View File

@@ -38,7 +38,6 @@ import tempfile
import time
from xml.sax import saxutils
from eventlet import event
from eventlet.green import subprocess
from eventlet import greenthread
import netaddr
@@ -547,113 +546,6 @@ class LazyPluggable(object):
return getattr(backend, key)
class LoopingCallDone(Exception):
"""Exception to break out and stop a LoopingCall.
The poll-function passed to LoopingCall can raise this exception to
break out of the loop normally. This is somewhat analogous to
StopIteration.
An optional return-value can be included as the argument to the exception;
this return-value will be returned by LoopingCall.wait()
"""
def __init__(self, retvalue=True):
""":param retvalue: Value that LoopingCall.wait() should return."""
self.retvalue = retvalue
class LoopingCallBase(object):
def __init__(self, f=None, *args, **kw):
self.args = args
self.kw = kw
self.f = f
self._running = False
self.done = None
def stop(self):
self._running = False
def wait(self):
return self.done.wait()
class FixedIntervalLoopingCall(LoopingCallBase):
"""A looping call which happens at a fixed interval."""
def start(self, interval, initial_delay=None):
self._running = True
done = event.Event()
def _inner():
if initial_delay:
greenthread.sleep(initial_delay)
try:
while self._running:
self.f(*self.args, **self.kw)
if not self._running:
break
greenthread.sleep(interval)
except LoopingCallDone, e:
self.stop()
done.send(e.retvalue)
except Exception:
LOG.exception(_('in fixed duration looping call'))
done.send_exception(*sys.exc_info())
return
else:
done.send(True)
self.done = done
greenthread.spawn(_inner)
return self.done
class DynamicLoopingCall(LoopingCallBase):
"""A looping call which happens sleeps until the next known event.
The function called should return how long to sleep for before being
called again.
"""
def start(self, initial_delay=None, periodic_interval_max=None):
self._running = True
done = event.Event()
def _inner():
if initial_delay:
greenthread.sleep(initial_delay)
try:
while self._running:
idle = self.f(*self.args, **self.kw)
if not self._running:
break
if periodic_interval_max is not None:
idle = min(idle, periodic_interval_max)
LOG.debug(_('Periodic task processor sleeping for %.02f '
'seconds'), idle)
greenthread.sleep(idle)
except LoopingCallDone, e:
self.stop()
done.send(e.retvalue)
except Exception:
LOG.exception(_('in dynamic looping call'))
done.send_exception(*sys.exc_info())
return
else:
done.send(True)
self.done = done
greenthread.spawn(_inner)
return self.done
def xhtml_escape(value):
"""Escapes a string so it is valid within XML or XHTML.

View File

@@ -29,6 +29,7 @@ from oslo.config import cfg
from nova import exception
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova import paths
from nova import utils
from nova.virt.baremetal import baremetal_states
@@ -149,10 +150,10 @@ class IPMI(base.PowerManager):
if self._is_power("on"):
self.state = baremetal_states.ACTIVE
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
if self.retries > CONF.baremetal.ipmi_power_retry:
self.state = baremetal_states.ERROR
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
try:
self.retries += 1
self._exec_ipmitool("power on")
@@ -160,7 +161,7 @@ class IPMI(base.PowerManager):
LOG.exception(_("IPMI power on failed"))
self.retries = 0
timer = utils.FixedIntervalLoopingCall(_wait_for_power_on)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_power_on)
timer.start(interval=0.5).wait()
def _power_off(self):
@@ -171,10 +172,10 @@ class IPMI(base.PowerManager):
if self._is_power("off"):
self.state = baremetal_states.DELETED
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
if self.retries > CONF.baremetal.ipmi_power_retry:
self.state = baremetal_states.ERROR
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
try:
self.retries += 1
self._exec_ipmitool("power off")
@@ -182,7 +183,7 @@ class IPMI(base.PowerManager):
LOG.exception(_("IPMI power off failed"))
self.retries = 0
timer = utils.FixedIntervalLoopingCall(_wait_for_power_off)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_power_off)
timer.start(interval=0.5).wait()
def _set_pxe_for_next_boot(self):

View File

@@ -30,8 +30,8 @@ from nova import exception
from nova.openstack.common.db import exception as db_exc
from nova.openstack.common import fileutils
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.openstack.common import timeutils
from nova import utils
from nova.virt.baremetal import baremetal_states
from nova.virt.baremetal import base
from nova.virt.baremetal import db
@@ -458,7 +458,7 @@ class PXE(base.NodeDriver):
if instance['uuid'] != row.get('instance_uuid'):
locals['error'] = _("Node associated with another instance"
" while waiting for deploy of %s")
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
status = row.get('task_state')
if (status == baremetal_states.DEPLOYING
@@ -470,7 +470,7 @@ class PXE(base.NodeDriver):
baremetal_states.ACTIVE):
LOG.info(_("PXE deploy completed for instance %s")
% instance['uuid'])
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
elif status == baremetal_states.DEPLOYFAIL:
locals['error'] = _("PXE deploy failed for instance %s")
except exception.NodeNotFound:
@@ -482,11 +482,11 @@ class PXE(base.NodeDriver):
locals['error'] = _("Timeout reached while waiting for "
"PXE deploy of instance %s")
if locals['error']:
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
expiration = timeutils.utcnow() + datetime.timedelta(
seconds=CONF.baremetal.pxe_deploy_timeout)
timer = utils.FixedIntervalLoopingCall(_wait_for_deploy)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_deploy)
timer.start(interval=1).wait()
if locals['error']:

View File

@@ -75,6 +75,7 @@ from nova.openstack.common import fileutils
from nova.openstack.common import importutils
from nova.openstack.common import jsonutils
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova.openstack.common.notifier import api as notifier
from nova import utils
from nova import version
@@ -734,12 +735,12 @@ class LibvirtDriver(driver.ComputeDriver):
except exception.NotFound:
LOG.error(_("During wait destroy, instance disappeared."),
instance=instance)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
if state == power_state.SHUTDOWN:
LOG.info(_("Instance destroyed successfully."),
instance=instance)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
# NOTE(wangpan): If the instance was booted again after destroy,
# this may be a endless loop, so check the id of
@@ -750,10 +751,11 @@ class LibvirtDriver(driver.ComputeDriver):
LOG.info(_("Instance may be started again."),
instance=instance)
kwargs['is_running'] = True
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
kwargs = {'is_running': False}
timer = utils.FixedIntervalLoopingCall(_wait_for_destroy, old_domid)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_destroy,
old_domid)
timer.start(interval=0.5).wait()
if kwargs['is_running']:
LOG.info(_("Going to destroy instance again."), instance=instance)
@@ -1327,8 +1329,8 @@ class LibvirtDriver(driver.ComputeDriver):
LOG.info(_("Instance shutdown successfully."),
instance=instance)
self._create_domain(domain=dom)
timer = utils.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer = loopingcall.FixedIntervalLoopingCall(
self._wait_for_running, instance)
timer.start(interval=0.5).wait()
return True
elif old_domid != new_domid:
@@ -1382,9 +1384,9 @@ class LibvirtDriver(driver.ComputeDriver):
if state == power_state.RUNNING:
LOG.info(_("Instance rebooted successfully."),
instance=instance)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
timer = utils.FixedIntervalLoopingCall(_wait_for_reboot)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_reboot)
timer.start(interval=0.5).wait()
def pause(self, instance):
@@ -1405,8 +1407,8 @@ class LibvirtDriver(driver.ComputeDriver):
"""Power on the specified instance."""
dom = self._lookup_by_name(instance['name'])
self._create_domain(domain=dom, instance=instance)
timer = utils.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer = loopingcall.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer.start(interval=0.5).wait()
def suspend(self, instance):
@@ -1534,9 +1536,9 @@ class LibvirtDriver(driver.ComputeDriver):
if state == power_state.RUNNING:
LOG.info(_("Instance spawned successfully."),
instance=instance)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
timer = utils.FixedIntervalLoopingCall(_wait_for_boot)
timer = loopingcall.FixedIntervalLoopingCall(_wait_for_boot)
timer.start(interval=0.5).wait()
def _flush_libvirt_console(self, pty):
@@ -3139,7 +3141,7 @@ class LibvirtDriver(driver.ComputeDriver):
recover_method(ctxt, instance_ref, dest, block_migration)
# Waiting for completion of live_migration.
timer = utils.FixedIntervalLoopingCall(f=None)
timer = loopingcall.FixedIntervalLoopingCall(f=None)
def wait_for_live_migration():
"""waiting for live migration completion."""
@@ -3506,7 +3508,7 @@ class LibvirtDriver(driver.ComputeDriver):
if state == power_state.RUNNING:
LOG.info(_("Instance running successfully."), instance=instance)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
def finish_migration(self, context, migration, instance, disk_info,
network_info, image_meta, resize_instance,
@@ -3562,8 +3564,8 @@ class LibvirtDriver(driver.ComputeDriver):
write_to_disk=True)
self._create_domain_and_network(xml, instance, network_info,
block_device_info)
timer = utils.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer = loopingcall.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer.start(interval=0.5).wait()
def _cleanup_failed_migration(self, inst_base):
@@ -3596,8 +3598,8 @@ class LibvirtDriver(driver.ComputeDriver):
self._create_domain_and_network(xml, instance, network_info,
block_device_info)
timer = utils.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer = loopingcall.FixedIntervalLoopingCall(self._wait_for_running,
instance)
timer.start(interval=0.5).wait()
def confirm_migration(self, migration, instance, network_info):

View File

@@ -29,6 +29,7 @@ from oslo.config import cfg
from nova import exception
from nova.openstack.common import lockutils
from nova.openstack.common import log as logging
from nova.openstack.common import loopingcall
from nova import paths
from nova.storage import linuxscsi
from nova import utils
@@ -538,7 +539,7 @@ class LibvirtAOEVolumeDriver(LibvirtBaseVolumeDriver):
def _wait_for_device_discovery(aoedevpath, mount_device):
tries = self.tries
if os.path.exists(aoedevpath):
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
if self.tries >= CONF.num_aoe_discover_tries:
raise exception.NovaException(_("AoE device not found at %s") %
@@ -551,8 +552,8 @@ class LibvirtAOEVolumeDriver(LibvirtBaseVolumeDriver):
self.tries = self.tries + 1
self.tries = 0
timer = utils.FixedIntervalLoopingCall(_wait_for_device_discovery,
aoedevpath, mount_device)
timer = loopingcall.FixedIntervalLoopingCall(
_wait_for_device_discovery, aoedevpath, mount_device)
timer.start(interval=2).wait()
tries = self.tries
@@ -701,7 +702,7 @@ class LibvirtFibreChannelVolumeDriver(LibvirtBaseVolumeDriver):
# get the /dev/sdX device. This is used
# to find the multipath device.
self.device_name = os.path.realpath(device)
raise utils.LoopingCallDone()
raise loopingcall.LoopingCallDone()
if self.tries >= CONF.num_iscsi_scan_tries:
msg = _("Fibre Channel device not found.")
@@ -717,8 +718,8 @@ class LibvirtFibreChannelVolumeDriver(LibvirtBaseVolumeDriver):
self.host_device = None
self.device_name = None
self.tries = 0
timer = utils.FixedIntervalLoopingCall(_wait_for_device_discovery,
host_devices, mount_device)
timer = loopingcall.FixedIntervalLoopingCall(
_wait_for_device_discovery, host_devices, mount_device)
timer.start(interval=2).wait()
tries = self.tries

View File

@@ -46,7 +46,7 @@ from oslo.config import cfg
from nova import exception
from nova.openstack.common import jsonutils
from nova.openstack.common import log as logging
from nova import utils
from nova.openstack.common import loopingcall
from nova.virt import driver
from nova.virt.vmwareapi import error_util
from nova.virt.vmwareapi import host
@@ -552,8 +552,9 @@ class VMwareAPISession(object):
The task is polled until it completes.
"""
done = event.Event()
loop = utils.FixedIntervalLoopingCall(self._poll_task, instance_uuid,
task_ref, done)
loop = loopingcall.FixedIntervalLoopingCall(self._poll_task,
instance_uuid,
task_ref, done)
loop.start(CONF.vmwareapi_task_poll_interval)
ret_val = done.wait()
loop.stop()

View File

@@ -15,6 +15,7 @@ module=jsonutils
module=local
module=lockutils
module=log
module=loopingcall
module=network_utils
module=notifier
module=plugin