Immediate caching of an image

Till now glance was dependent on periodic job to get image
cached locally and also has dependency on ``cachemanage``
middleware for the same.

This patch removes the periodic job and dependency of
``cachemanage`` middleware and initiates immediate caching
of an image via ``PUT /cache/image_id`` API call.

Co-Author: Dan Smith <dms@danplanet.com>

Implements: blueprint instant-caching
Change-Id: I9ab3f1b7595e22dbb03af95168314352a44eb930
This commit is contained in:
Abhishek Kekane 2022-05-17 09:41:00 +00:00
parent 95a02faf9c
commit c1a2b9e221
12 changed files with 369 additions and 249 deletions

View File

@ -74,8 +74,6 @@ correctly.
store, points to where the data is kept.
- ``filesystem_store_datadirs`` This is used to point to multiple
filesystem stores.
- ``cache_prefetcher_interval`` The interval in seconds to run periodic
job 'cache_images'.
Controlling the Growth of the Image Cache
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
@ -107,37 +105,76 @@ executable.
The recommended practice is to use ``cron`` to fire ``glance-cache-cleaner``
at a semi-regular interval.
Prefetching Images into the Image Cache
---------------------------------------
Controlling Image Cache using V2 API
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
In Train, Glance API has added a new periodic job ``cache_images`` which will
run after every predefined time interval to fetch the queued images into cache.
The default time interval for the ``cache_images`` periodic job is 300
seconds. Admin/Operator can configure this interval in glance-api.conf file or
glance-cache.conf file using ``cache_prefetcher_interval`` configuration
option. The ``cache_images`` periodic job will only run if cache middleware
is enabled in your cloud.
In Yoga, Glance API has added new APIs for managing cache
related operations. In Zed, Glance has removed support of ``cache_images``
periodic job which was used to prefetch all queued images concurrently,
logging the results of the fetch for each image. Instead the image can be
immediately cached once it is queued for caching. You can use below API
calls to control the cache related operations.
To queue an image for prefetching, you can use one of the following methods:
To queue an image for immediate caching, you can use one of the following
methods:
* If the ``cache_manage`` middleware is enabled in the application pipeline,
you may call ``PUT /queued/<IMAGE_ID>`` to queue the image with
identifier ``<IMAGE_ID>``
* You can call ``PUT /cache/<IMAGE_ID>`` to queue the image for immediate
caching with identifier ``<IMAGE_ID>``
* Alternately, you can use the ``glance-cache-manage`` program to queue the
image. This program may be run from a different host than the host
containing the image cache. Example usage::
* Alternately, you can use the ``cache-queue`` command of glance client to
queue the image for immediate caching.
$ glance-cache-manage --host=<HOST> queue-image <IMAGE_ID>
$ glance cache-queue <IMAGE_ID>
This will queue the image with identifier ``<IMAGE_ID>`` for prefetching
This will queue the image with identifier ``<IMAGE_ID>`` for immediate
caching.
Once you have queued the images you wish to prefetch, the ``cache_images``
periodic job will prefetch all queued images concurrently, logging the
results of the fetch for each image.
To find out which images are in the image cache use one of the
following methods:
Finding Which Images are in the Image Cache
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
* You can call ``GET /cache`` to see a JSON-serialized list of
mappings that show cached images, the number of cache hits on each image,
the size of the image, and the times they were last accessed as well as
images which are queued for caching.
* Alternately, you can use the ``cache-list`` command of glance
client. Example usage::
$ glance cache-list
To delete images which are already cached or queued for caching use one of
the following methods:
* You can call ``DELETE /cache/<IMAGE_ID>`` to remove the image file for image
with identifier ``<IMAGE_ID>`` from the cache or queued state.
* Alternately, you can use the ``cache-delete`` command of glance
client. Example usage::
$ glance cache-delete <IMAGE_ID>
* You can also call ``DELETE /cache`` with header
``x-image-cache-clear-target`` to delete either only cached images or
only queued images or both. Possible values for header are ``cache``,
``queue``, ``both``.
* Alternately, you can use the ``cache-clear`` command of glance client
to delete only cached images or only queued images or both. Example usage::
$ glance cache-clear (default target is ``both``)
$ glance cache-clear --target cached
$ glance cache-clear --target queued
* In Glance, image cache is local to each node, hence cache operations
must be performed on each node locally. If OpenStack cloud is deployed with
HA (3/5/7 controllers) then while running the cache related operations it is
necessary to specify the HOST address using -H option.
Example usage::
$ glance --host=<HOST> cache-list
Finding Which Images are in the Image Cache with glance-cache-manage
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
You can find out which images are in the image cache using one of the
following methods:
@ -171,8 +208,8 @@ following methods:
Note that the image's cache hit is not shown using this method.
Manually Removing Images from the Image Cache
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Manually Removing Images from the Image Cache with glance-cache-manage
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
If the ``cachemanage`` middleware is enabled, you may call
``DELETE /cached-images/<IMAGE_ID>`` to remove the image file for image

View File

@ -17,6 +17,9 @@
Controller for Image Cache Management API
"""
import queue
import threading
import glance_store
from oslo_config import cfg
from oslo_log import log as logging
@ -35,6 +38,7 @@ import glance.notifier
CONF = cfg.CONF
LOG = logging.getLogger(__name__)
WORKER = None
class CacheController(object):
@ -44,6 +48,7 @@ class CacheController(object):
def __init__(self, db_api=None, policy_enforcer=None, notifier=None,
store_api=None):
global WORKER
if not CONF.image_cache_dir:
self.cache = None
else:
@ -56,6 +61,13 @@ class CacheController(object):
self.gateway = glance.gateway.Gateway(self.db_api, self.store_api,
self.notifier, self.policy)
# Initialize the worker only if cache is enabled
if CONF.image_cache_dir and not WORKER:
# If we're the first, start the thread
WORKER = CacheWorker()
WORKER.start()
LOG.debug('Started cache worker thread')
def _enforce(self, req, image=None, new_policy=None):
"""Authorize request against given policy"""
if not new_policy:
@ -229,6 +241,47 @@ class CacheController(object):
raise webob.exc.HTTPBadRequest(explanation=msg)
self.cache.queue_image(image_id)
WORKER.submit(image_id)
class CacheWorker(threading.Thread):
EXIT_SENTINEL = object()
def __init__(self, *args, **kwargs):
self.q = queue.Queue(maxsize=-1)
# NOTE(abhishekk): Importing the prefetcher just in time to avoid
# import loop during initialization
from glance.image_cache import prefetcher # noqa
self.prefetcher = prefetcher.Prefetcher()
super().__init__(*args, **kwargs)
# NOTE(abhishekk): Setting daemon to True because if `atexit` event
# handler is not called due to some reason the main process will
# not hang for the thread which will never exit.
self.setDaemon(True)
def submit(self, job):
self.q.put(job)
def terminate(self):
# NOTE(danms): Make the API workers call this before we exit
# to make sure any cache operations finish.
LOG.info('Signaling cache worker thread to exit')
self.q.put(self.EXIT_SENTINEL)
self.join()
LOG.info('Cache worker thread exited')
def run(self):
while True:
task = self.q.get()
if task == self.EXIT_SENTINEL:
LOG.debug("CacheWorker thread exiting")
break
LOG.debug("Processing image '%s' for caching", task)
self.prefetcher.fetch_image_into_cache(task)
# do whatever work you have to do on task
self.q.task_done()
LOG.debug("Caching of an image '%s' is complete", task)
class CachedImageDeserializer(wsgi.JSONRequestDeserializer):

View File

@ -119,15 +119,7 @@ def main():
# NOTE(danms): Configure system-wide threading model to use eventlet
glance.async_.set_threadpool_model('eventlet')
# NOTE(abhishekk): Added initialize_prefetcher KW argument to Server
# object so that prefetcher object should only be initialized in case
# of API service and ignored in case of registry. Once registry is
# removed this parameter should be removed as well.
initialize_prefetcher = False
if CONF.paste_deploy.flavor == 'keystone+cachemanagement':
initialize_prefetcher = True
server = wsgi.Server(initialize_glance_store=True,
initialize_prefetcher=initialize_prefetcher)
server = wsgi.Server(initialize_glance_store=True)
server.start(config.load_paste_app('glance-api'), default_port=9292)
server.wait()
except Exception as e:

View File

@ -29,7 +29,6 @@ import signal
import struct
import subprocess
import sys
import threading
import time
from eventlet.green import socket
@ -270,22 +269,6 @@ cli_opts = [
'used for inter-process communication.'),
]
cache_opts = [
cfg.FloatOpt('cache_prefetcher_interval',
default=300,
help=_("""
The interval in seconds to run periodic job cache_images.
The cache_images method will fetch all images which are in queued state
for caching in cache directory. The default value is 300.
Possible values:
* Positive integer
Related options:
* None
"""))
]
LOG = logging.getLogger(__name__)
@ -295,7 +278,6 @@ CONF.register_opts(socket_opts)
CONF.register_opts(eventlet_opts)
CONF.register_opts(wsgi_opts)
CONF.register_opts(store_opts)
CONF.register_opts(cache_opts)
profiler_opts.set_defaults(CONF)
ASYNC_EVENTLET_THREAD_POOL_LIST = []
@ -434,8 +416,7 @@ class BaseServer(metaclass=abc.ABCMeta):
This class requires initialize_glance_store set to True if
glance store needs to be initialized.
"""
def __init__(self, threads=1000, initialize_glance_store=False,
initialize_prefetcher=False):
def __init__(self, threads=1000, initialize_glance_store=False):
os.umask(0o27) # ensure files are created with the correct privileges
self._logger = logging.getLogger("eventlet.wsgi.server")
self.threads = threads
@ -445,21 +426,6 @@ class BaseServer(metaclass=abc.ABCMeta):
# NOTE(abhishek): Allows us to only re-initialize glance_store when
# the API's configuration reloads.
self.initialize_glance_store = initialize_glance_store
self.initialize_prefetcher = initialize_prefetcher
if self.initialize_prefetcher:
# NOTE(abhishekk): Importing the prefetcher just in time to avoid
# import loop during initialization
from glance.image_cache import prefetcher # noqa
self.prefetcher = prefetcher.Prefetcher()
def cache_images(self):
# After every 'cache_prefetcher_interval' this call will run and fetch
# all queued images into cache if there are any
cache_thread = threading.Timer(CONF.cache_prefetcher_interval,
self.cache_images)
cache_thread.daemon = True
cache_thread.start()
self.prefetcher.run()
@staticmethod
def set_signal_handler(signal_name, handler):
@ -505,9 +471,6 @@ class BaseServer(metaclass=abc.ABCMeta):
cleaner = housekeeping.StagingStoreCleaner(glance.db.get_api())
self.pool.spawn_n(cleaner.clean_orphaned_staging_residue)
if self.initialize_prefetcher:
self.cache_images()
def start_wsgi(self):
workers = get_num_workers()
self.pool = self.create_pool()
@ -582,6 +545,14 @@ class BaseServer(metaclass=abc.ABCMeta):
for pool in ASYNC_EVENTLET_THREAD_POOL_LIST:
pool.waitall()
# NOTE(abhishekk): Importing the cache_images API module just
# in time to avoid partial initialization of wsgi module
from glance.api.v2 import cached_images # noqa
if cached_images.WORKER:
# If we started a cache worker, signal it to exit
# and wait until it does.
cached_images.WORKER.terminate()
def _single_run(self, application, sock):
"""Start a WSGI server in a new green thread."""
LOG.info(_LI("Starting single process server"))

View File

@ -34,6 +34,14 @@ CONF.import_opt("enabled_backends", "glance.common.wsgi")
logging.register_options(CONF)
LOG = logging.getLogger(__name__)
# Detect if we're running under the uwsgi server
try:
import uwsgi
LOG.debug('Detected running under uwsgi')
except ImportError:
LOG.debug('Detected not running under uwsgi')
uwsgi = None
CONFIG_FILES = ['glance-api-paste.ini',
'glance-image-import.conf',
'glance-api.conf']
@ -82,7 +90,7 @@ def _validate_policy_enforcement_configuration():
raise exception.ServerError(fail_message)
def drain_threadpools():
def drain_workers():
# NOTE(danms): If there are any other named pools that we need to
# drain before exit, they should be in this list.
pools_to_drain = ['tasks_pool']
@ -91,6 +99,12 @@ def drain_threadpools():
LOG.info('Waiting for remaining threads in pool %r', pool_name)
pool_model.pool.shutdown()
from glance.api.v2 import cached_images # noqa
if cached_images.WORKER:
# If we started a cache worker, signal it to exit
# and wait until it does.
cached_images.WORKER.terminate()
def run_staging_cleanup():
cleaner = housekeeping.StagingStoreCleaner(glance.db.get_api())
@ -103,28 +117,6 @@ def run_staging_cleanup():
cleanup_thread.start()
def cache_images(cache_prefetcher):
# After every 'cache_prefetcher_interval' this call will run and fetch
# all queued images into cache if there are any
cache_thread = threading.Timer(CONF.cache_prefetcher_interval,
cache_images, (cache_prefetcher,))
cache_thread.daemon = True
cache_thread.start()
cache_prefetcher.run()
def run_cache_prefetcher():
if not CONF.paste_deploy.flavor == 'keystone+cachemanagement':
LOG.debug('Cache not enabled, skipping prefetching images in cache!!!')
return
# NOTE(abhishekk): Importing the prefetcher just in time to avoid
# import loop during initialization
from glance.image_cache import prefetcher # noqa
cache_prefetcher = prefetcher.Prefetcher()
cache_images(cache_prefetcher)
def init_app():
config.set_config_defaults()
config_files = _get_config_files()
@ -134,7 +126,10 @@ def init_app():
# NOTE(danms): We are running inside uwsgi or mod_wsgi, so no eventlet;
# use native threading instead.
glance.async_.set_threadpool_model('native')
atexit.register(drain_threadpools)
if uwsgi:
uwsgi.atexit = drain_workers
else:
atexit.register(drain_workers)
# NOTE(danms): Change the default threadpool size since we
# are dealing with native threads and not greenthreads.
@ -157,7 +152,6 @@ def init_app():
glance_store.create_stores(CONF)
glance_store.verify_default_store()
run_cache_prefetcher()
run_staging_cleanup()
_setup_os_profiler()

View File

@ -55,7 +55,6 @@ _api_opts = [
glance.common.wsgi.socket_opts,
glance.common.wsgi.wsgi_opts,
glance.common.wsgi.store_opts,
glance.common.wsgi.cache_opts,
glance.common.wsgi.cli_opts,
glance.image_cache.drivers.sqlite.sqlite_opts,
glance.image_cache.image_cache_opts,

View File

@ -1600,7 +1600,7 @@ class SynchronousAPIBase(test_utils.BaseTestCase):
self.setup_simple_paste()
self.setup_stores()
def start_server(self, enable_cache=False):
def start_server(self, enable_cache=True):
"""Builds and "starts" the API server.
Note that this doesn't actually "start" anything like

View File

@ -12,12 +12,12 @@
# 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 time
from unittest import mock
import oslo_policy.policy
from glance.api import policy
from glance.image_cache import prefetcher
from glance.tests import functional
@ -52,7 +52,7 @@ class TestImageCache(functional.SynchronousAPIBase):
image_id = response.json['id']
output['queued'] = image_id
for visibility in ['public', 'private', 'community', 'shared']:
for visibility in ['public', 'private']:
data = {
'name': '%s-image' % visibility,
'visibility': visibility,
@ -101,14 +101,37 @@ class TestImageCache(functional.SynchronousAPIBase):
else:
self.assertEqual(expected_code, response.status_code)
def cache_image(self):
# NOTE(abhishekk): Here we are not running periodic job which caches
# queued images as precaching is not part of this patch, so to test
# all caching operations we are using this way to cache images for us
cache_prefetcher = prefetcher.Prefetcher()
cache_prefetcher.run()
def wait_for_caching(self, image_id, max_sec=10, delay_sec=0.2,
start_delay_sec=None):
start_time = time.time()
done_time = start_time + max_sec
if start_delay_sec:
time.sleep(start_delay_sec)
while time.time() <= done_time:
output = self.list_cache()['cached_images']
output = [image['image_id'] for image in output]
if output and image_id in output:
return
time.sleep(delay_sec)
def test_cache_api_lifecycle(self):
msg = "Image {0} failed to cached within {1} sec"
raise Exception(msg.format(image_id, max_sec))
def test_cache_list(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Ensure that nothing is cached and nothing is queued for caching
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Queue 1 image for caching
self.cache_queue(images['public'])
output = self.list_cache()
self.assertEqual(1, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
def test_cache_queue(self):
self.start_server(enable_cache=True)
images = self.load_data()
@ -117,128 +140,121 @@ class TestImageCache(functional.SynchronousAPIBase):
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Queue 1 image for caching
self.cache_queue(images['public'])
# NOTE(abhishekk): As queue call will immediately start caching
# the image, lets wait for completion.
self.wait_for_caching(images['public'])
# Now verify that we have 1 cached image
output = self.list_cache()
self.assertEqual(1, len(output['cached_images']))
# Verify same image is cached
self.assertIn(images['public'], output['cached_images'][0]['image_id'])
def test_cache_delete(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Queue 1 image for caching
self.cache_queue(images['public'])
self.wait_for_caching(images['public'])
# Now verify that we have 1 cached image
output = self.list_cache()
self.assertEqual(1, len(output['cached_images']))
# Verify same image is cached
self.assertIn(images['public'], output['cached_images'][0]['image_id'])
# Delete cached image
self.cache_delete(images['public'])
# Now verify that we have 0 cached image
output = self.list_cache()
self.assertEqual(0, len(output['cached_images']))
def test_cache_clear_queued_images(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Queue 2 images for caching
self.cache_queue(images['public'])
self.cache_queue(images['private'])
# Now verify that we have 2 queued images
# NOTE(abhishekk): We might fail with race here as queue call
# will immediately start caching of an image, so we may not find
# all images in queued state.
output = self.list_cache()
self.assertEqual(2, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Clear all images from cache
self.cache_clear(target='queue')
# Now verify that we have 0 queued images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
def test_cache_clear_cached_images(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Queue 2 images for caching
self.cache_queue(images['public'])
self.cache_queue(images['private'])
self.wait_for_caching(images['public'])
self.wait_for_caching(images['private'])
# Now verify that we have 2 cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(2, len(output['cached_images']))
# Clear all images from cache
self.cache_clear(target='cache')
# Now verify that we have 0 cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
def test_cache_clear(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Queue 2 images for caching
self.cache_queue(images['public'])
self.wait_for_caching(images['public'])
# Now verify that we have 1 cached images
output = self.list_cache()
self.assertEqual(1, len(output['cached_images']))
self.cache_queue(images['private'])
# Now verify that we have 1 queued and 1 cached images
output = self.list_cache()
# NOTE(abhishekk): We might fail with race here as queue call
# will immediately start caching of an image, so we may not find
# image in queued state.
self.assertEqual(1, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Clear all images from cache
self.cache_clear()
# Now verify that we have 0 queued and cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
def test_cache_api_negative_scenarios(self):
self.start_server(enable_cache=True)
images = self.load_data()
# Try non-existing image to queue for caching
self.cache_queue('non-existing-image-id', expected_code=404)
# Verify that you can not queue non-active image
self.cache_queue(images['queued'], expected_code=400)
# Queue 1 image for caching
self.cache_queue(images['public'])
# Now verify that we have 1 image queued for caching and 0
# cached images
output = self.list_cache()
self.assertEqual(1, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Verify same image is queued for caching
self.assertIn(images['public'], output['queued_images'])
# Cache the image
self.cache_image()
# Now verify that we have 0 queued image and 1 cached image
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Verify same image is queued for caching
self.assertIn(images['public'], output['cached_images'][0]['image_id'])
# Queue 2nd image for caching
self.cache_queue(images['community'])
# Now verify that we have 1 image queued for caching and 1
# cached images
output = self.list_cache()
self.assertEqual(1, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Verify same image is queued for caching
self.assertIn(images['community'], output['queued_images'])
self.assertIn(images['public'], output['cached_images'][0]['image_id'])
# Queue 3rd image for caching
self.cache_queue(images['private'])
# Now verify that we have 2 images queued for caching and 1
# cached images
output = self.list_cache()
self.assertEqual(2, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Verify same image is queued for caching
self.assertIn(images['private'], output['queued_images'])
# Try to delete non-existing image from cache
self.cache_delete('non-existing-image-id', expected_code=404)
# Delete public image from cache
self.cache_delete(images['public'])
# Now verify that we have 2 image queued for caching and no
# cached images
output = self.list_cache()
self.assertEqual(2, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Verify clearing cache fails with 400 if invalid header is passed
self.cache_clear(target='both', expected_code=400)
# Delete all queued images
self.cache_clear(target='queue')
# Now verify that we have 0 image queued for caching and 0
# cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Queue and cache image so we have something to clear
self.cache_queue(images['public'])
# Now verify that we have 1 queued image
output = self.list_cache()
self.assertEqual(1, len(output['queued_images']))
self.cache_image()
# Now verify that we have 0 queued image and 1 cached image
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Delete all cached images
self.cache_clear(target='cache')
# Now verify that we have 0 image queued for caching and 0
# cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Now we need 2 queued images and 2 cached images in order
# to delete both of them together
self.cache_queue(images['public'])
self.cache_queue(images['private'])
# Now verify that we have 2 queued images
output = self.list_cache()
self.assertEqual(2, len(output['queued_images']))
self.cache_image()
# Now verify that we have 0 queued images and 2 cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(2, len(output['cached_images']))
self.cache_queue(images['community'])
self.cache_queue(images['shared'])
# Verify we have 2 queued and 2 cached images
output = self.list_cache()
self.assertEqual(2, len(output['queued_images']))
self.assertEqual(2, len(output['cached_images']))
# Now delete all queued and all cached images at once
self.cache_clear()
# Now verify that we have 0 image queued for caching and 0
# cached images
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Try to cache image again to validate nothing will be cached
self.cache_image()
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
def test_cache_image_queue_delete(self):
# This test verifies that if image is queued for caching
# and user deletes the original image, but it is still
@ -290,21 +306,14 @@ class TestImageCache(functional.SynchronousAPIBase):
self.assertEqual(0, len(output['cached_images']))
self.cache_queue(images['public'])
# Now verify that we have 1 image queued for caching and 0
# cached images
output = self.list_cache()
self.assertEqual(1, len(output['queued_images']))
self.assertEqual(0, len(output['cached_images']))
# Verify same image is queued for caching
self.assertIn(images['public'], output['queued_images'])
# Cache the image
self.cache_image()
# wait for caching the image
self.wait_for_caching(images['public'])
# Now verify that we have 0 queued image and 1 cached image
output = self.list_cache()
self.assertEqual(0, len(output['queued_images']))
self.assertEqual(1, len(output['cached_images']))
# Verify same image is queued for caching
# Verify same image cached
self.assertIn(images['public'], output['cached_images'][0]['image_id'])
# Delete image and verify that it is deleted from

View File

@ -17,6 +17,7 @@
from unittest import mock
from glance.api import common
from glance.api.v2 import cached_images
import glance.async_
from glance.common import exception
from glance.common import wsgi_app
@ -49,10 +50,22 @@ class TestWsgiAppInit(test_utils.BaseTestCase):
mock_load, mock_exit):
mock_config_files.return_value = []
wsgi_app.init_app()
mock_exit.assert_called_once_with(wsgi_app.drain_threadpools)
mock_exit.assert_called_once_with(wsgi_app.drain_workers)
@mock.patch('glance.common.config.load_paste_app')
@mock.patch('glance.async_.set_threadpool_model')
@mock.patch('glance.common.wsgi_app._get_config_files')
def test_uwsgi_init_registers_exit_handler(self, mock_config_files,
mock_set_model,
mock_load):
mock_config_files.return_value = []
with mock.patch.object(wsgi_app, 'uwsgi') as mock_u:
wsgi_app.init_app()
self.assertEqual(mock_u.atexit, wsgi_app.drain_workers)
@mock.patch('glance.api.v2.cached_images.WORKER')
@mock.patch('glance.async_._THREADPOOL_MODEL', new=None)
def test_drain_threadpools(self):
def test_drain_workers(self, mock_cache_worker):
# Initialize the thread pool model and tasks_pool, like API
# under WSGI would, and so we have a pointer to that exact
# pool object in the cache
@ -60,11 +73,24 @@ class TestWsgiAppInit(test_utils.BaseTestCase):
model = common.get_thread_pool('tasks_pool')
with mock.patch.object(model.pool, 'shutdown') as mock_shutdown:
wsgi_app.drain_threadpools()
wsgi_app.drain_workers()
# Make sure that shutdown() was called on the tasks_pool
# ThreadPoolExecutor
mock_shutdown.assert_called_once_with()
# Make sure we terminated the cache worker, if present.
mock_cache_worker.terminate.assert_called_once_with()
@mock.patch('glance.async_._THREADPOOL_MODEL', new=None)
def test_drain_workers_no_cache(self):
glance.async_.set_threadpool_model('native')
model = common.get_thread_pool('tasks_pool')
with mock.patch.object(model.pool, 'shutdown'):
# Make sure that with no WORKER initialized, we do not fail
wsgi_app.drain_workers()
self.assertIsNone(cached_images.WORKER)
@mock.patch('glance.common.config.load_paste_app')
@mock.patch('glance.async_.set_threadpool_model')
@mock.patch('glance.common.wsgi_app._get_config_files')
@ -113,21 +139,6 @@ class TestWsgiAppInit(test_utils.BaseTestCase):
daemon=True)
mock_Thread.return_value.start.assert_called_once_with()
@mock.patch('glance.async_._THREADPOOL_MODEL', new=None)
@mock.patch('glance.common.config.load_paste_app')
@mock.patch('glance.common.wsgi_app._get_config_files')
@mock.patch('threading.Timer')
@mock.patch('glance.image_cache.prefetcher.Prefetcher')
def test_run_cache_prefetcher(self, mock_prefetcher,
mock_Timer, mock_conf,
mock_load):
self.config(cache_prefetcher_interval=10)
self.config(flavor='keystone+cachemanagement', group='paste_deploy')
mock_conf.return_value = []
wsgi_app.init_app()
mock_Timer.assert_called_once_with(10, mock.ANY, (mock_prefetcher(),))
mock_Timer.return_value.start.assert_called_once_with()
@mock.patch('glance.async_._THREADPOOL_MODEL', new=None)
@mock.patch('glance.common.config.load_paste_app')
@mock.patch('glance.common.wsgi_app._get_config_files')

View File

@ -116,8 +116,47 @@ class TestCacheManageAPI(test_utils.BaseTestCase):
'get_cache_state',
'cache_list'], image_mock=False)
def test_queue_image_from_api(self):
@mock.patch.object(cached_images, 'WORKER')
def test_queue_image_from_api(self, mock_worker):
self._main_test_helper(['queue_image',
'queue_image_from_api',
'cache_image',
UUID1])
mock_worker.submit.assert_called_once_with(UUID1)
def test_init_no_config(self):
# Make sure the worker was reset to uninitialized
self.assertIsNone(cached_images.WORKER)
self.config(image_cache_dir=None)
cached_images.CacheController()
# Make sure it is still None because image_cache_dir was not
# set
self.assertIsNone(cached_images.WORKER)
def test_init_with_config(self):
# Make sure the worker was reset to uninitialized
self.assertIsNone(cached_images.WORKER)
self.config(image_cache_dir='/tmp')
cached_images.CacheController()
# Make sure we initialized it because config told us to
self.assertIsNotNone(cached_images.WORKER)
self.assertTrue(cached_images.WORKER.is_alive())
cached_images.WORKER.terminate()
class TestCacheWorker(test_utils.BaseTestCase):
@mock.patch('glance.image_cache.prefetcher.Prefetcher')
def test_worker_lifecycle(self, mock_pf):
worker = cached_images.CacheWorker()
self.assertFalse(worker.is_alive())
worker.start()
self.assertTrue(worker.is_alive())
worker.submit('123')
worker.submit('456')
self.assertTrue(worker.is_alive())
worker.terminate()
self.assertFalse(worker.is_alive())
mock_pf.return_value.fetch_image_into_cache.assert_has_calls([
mock.call('123'), mock.call('456')])

View File

@ -41,6 +41,7 @@ from oslo_utils import units
import testtools
import webob
from glance.api.v2 import cached_images
from glance.common import config
from glance.common import exception
from glance.common import property_utils
@ -94,6 +95,10 @@ class BaseTestCase(testtools.TestCase):
self.useFixture(log_fixture.get_logging_handle_error_fixture())
self.useFixture(glance_fixtures.StandardLogging())
if cached_images.WORKER:
cached_images.WORKER.terminate()
cached_images.WORKER = None
def set_policy(self):
conf_file = "policy.yaml"
self.policy_file = self._copy_data_file(conf_file, self.conf_dir)

View File

@ -0,0 +1,10 @@
---
features:
- |
Added support to immediately start caching of an image.
upgrade:
- |
Periodic job to prefetch image(s) into cache has been removed from the
glance api service with config option ``cache_prefetcher_interval``
which was added as an interval for the same periodic job also been
removed as image(s) will be immediately queued for caching.