Move Redis driver's claim transaction to Lua
This avoids having to set a watch and retry if someone else beats us to the punch; the retry is less efficient than doing it with Lua, and would require adding a backoff sleep to avoid lots of failed requests under high concurrency. This patch also adds a Redis server version check in the driver since Lua requires redis-server >= 2.6 Change-Id: I0c34e267a6358965a54179a4b2fd03ac15b194e9 Closes-Bug: #1372510
This commit is contained in:
parent
db3cb1a529
commit
72598b2389
@ -1,3 +1,5 @@
|
|||||||
|
recursive-include zaqar *.lua
|
||||||
|
|
||||||
exclude .gitignore
|
exclude .gitignore
|
||||||
exclude .gitreview
|
exclude .gitreview
|
||||||
|
|
||||||
|
@ -175,6 +175,21 @@ class RedisDriverTest(testing.TestBase):
|
|||||||
|
|
||||||
self.assertTrue(isinstance(redis_driver.connection, redis.StrictRedis))
|
self.assertTrue(isinstance(redis_driver.connection, redis.StrictRedis))
|
||||||
|
|
||||||
|
def test_version_match(self):
|
||||||
|
cache = oslo_cache.get_cache()
|
||||||
|
|
||||||
|
with mock.patch('redis.StrictRedis.info') as info:
|
||||||
|
info.return_value = {'redis_version': '2.4.6'}
|
||||||
|
self.assertRaises(RuntimeError, driver.DataDriver,
|
||||||
|
self.conf, cache)
|
||||||
|
|
||||||
|
info.return_value = {'redis_version': '2.11'}
|
||||||
|
|
||||||
|
try:
|
||||||
|
driver.DataDriver(self.conf, cache)
|
||||||
|
except RuntimeError:
|
||||||
|
self.fail('version match failed')
|
||||||
|
|
||||||
|
|
||||||
@testing.requires_redis
|
@testing.requires_redis
|
||||||
class RedisQueuesTest(base.QueueControllerTest):
|
class RedisQueuesTest(base.QueueControllerTest):
|
||||||
|
@ -16,13 +16,13 @@ import functools
|
|||||||
|
|
||||||
import msgpack
|
import msgpack
|
||||||
from oslo.utils import timeutils
|
from oslo.utils import timeutils
|
||||||
import redis
|
|
||||||
|
|
||||||
from zaqar.common import decorators
|
from zaqar.common import decorators
|
||||||
from zaqar.openstack.common import log as logging
|
from zaqar.openstack.common import log as logging
|
||||||
from zaqar.queues import storage
|
from zaqar.queues import storage
|
||||||
from zaqar.queues.storage import errors
|
from zaqar.queues.storage import errors
|
||||||
from zaqar.queues.storage.redis import messages
|
from zaqar.queues.storage.redis import messages
|
||||||
|
from zaqar.queues.storage.redis import scripting
|
||||||
from zaqar.queues.storage.redis import utils
|
from zaqar.queues.storage.redis import utils
|
||||||
|
|
||||||
LOG = logging.getLogger(__name__)
|
LOG = logging.getLogger(__name__)
|
||||||
@ -41,7 +41,7 @@ RETRY_CLAIM_TIMEOUT = 10
|
|||||||
COUNTING_BATCH_SIZE = 100
|
COUNTING_BATCH_SIZE = 100
|
||||||
|
|
||||||
|
|
||||||
class ClaimController(storage.Claim):
|
class ClaimController(storage.Claim, scripting.Mixin):
|
||||||
"""Implements claim resource operations using Redis.
|
"""Implements claim resource operations using Redis.
|
||||||
|
|
||||||
Redis Data Structures:
|
Redis Data Structures:
|
||||||
@ -78,6 +78,8 @@ class ClaimController(storage.Claim):
|
|||||||
+----------------+---------+
|
+----------------+---------+
|
||||||
"""
|
"""
|
||||||
|
|
||||||
|
script_names = ['claim_messages']
|
||||||
|
|
||||||
def __init__(self, *args, **kwargs):
|
def __init__(self, *args, **kwargs):
|
||||||
super(ClaimController, self).__init__(*args, **kwargs)
|
super(ClaimController, self).__init__(*args, **kwargs)
|
||||||
self._client = self.driver.connection
|
self._client = self.driver.connection
|
||||||
@ -100,6 +102,17 @@ class ClaimController(storage.Claim):
|
|||||||
values = self._client.hmget(claim_id, fields)
|
values = self._client.hmget(claim_id, fields)
|
||||||
return [transform(v) for v in values] if transform else values
|
return [transform(v) for v in values] if transform else values
|
||||||
|
|
||||||
|
def _claim_messages(self, msgset_key, now, limit,
|
||||||
|
claim_id, claim_expires, msg_ttl, msg_expires):
|
||||||
|
|
||||||
|
# NOTE(kgriffs): A watch on a pipe could also be used, but that
|
||||||
|
# is less efficient and predictable, based on our experience in
|
||||||
|
# having to do something similar in the MongoDB driver.
|
||||||
|
func = self._scripts['claim_messages']
|
||||||
|
|
||||||
|
args = [now, limit, claim_id, claim_expires, msg_ttl, msg_expires]
|
||||||
|
return func(keys=[msgset_key], args=args)
|
||||||
|
|
||||||
def _exists(self, queue, claim_id, project):
|
def _exists(self, queue, claim_id, project):
|
||||||
client = self._client
|
client = self._client
|
||||||
claims_set_key = utils.scope_claims_set(queue, project,
|
claims_set_key = utils.scope_claims_set(queue, project,
|
||||||
@ -238,113 +251,41 @@ class ClaimController(storage.Claim):
|
|||||||
|
|
||||||
claim_ttl = int(metadata.get('ttl', 60))
|
claim_ttl = int(metadata.get('ttl', 60))
|
||||||
grace = int(metadata.get('grace', 60))
|
grace = int(metadata.get('grace', 60))
|
||||||
msg_ttl = claim_ttl + grace
|
|
||||||
|
|
||||||
claim_id = utils.generate_uuid()
|
|
||||||
claim_msgs_key = utils.scope_claim_messages(claim_id,
|
|
||||||
CLAIM_MESSAGES_SUFFIX)
|
|
||||||
|
|
||||||
claims_set_key = utils.scope_claims_set(queue, project,
|
|
||||||
QUEUE_CLAIMS_SUFFIX)
|
|
||||||
|
|
||||||
with self._client.pipeline() as pipe:
|
|
||||||
# NOTE(kgriffs): Retry the operation if another transaction
|
|
||||||
# completes before this one, in which case it will have
|
|
||||||
# claimed the same messages the current thread is trying
|
|
||||||
# to claim, and therefoe we must try for another batch.
|
|
||||||
#
|
|
||||||
# This loop will eventually time out if we can't manage to
|
|
||||||
# claim any messages due to other threads continually beating
|
|
||||||
# us to the punch.
|
|
||||||
|
|
||||||
# TODO(kgriffs): Would it be beneficial (or harmful) to
|
|
||||||
# introducce a backoff sleep in between retries?
|
|
||||||
|
|
||||||
start_ts = timeutils.utcnow_ts()
|
|
||||||
while (timeutils.utcnow_ts() - start_ts) < RETRY_CLAIM_TIMEOUT:
|
|
||||||
|
|
||||||
# NOTE(kgriffs): The algorithm for claiming messages:
|
|
||||||
#
|
|
||||||
# 1. Get a batch of messages that are currently active.
|
|
||||||
# 2. For each active message in the batch, extend its
|
|
||||||
# lifetime IFF it would otherwise expire before the
|
|
||||||
# claim itself does.
|
|
||||||
# 3. Associate the claim with each message
|
|
||||||
# 4. Create a claim record with details such as TTL
|
|
||||||
# and expiration time.
|
|
||||||
# 5. Add the claim's ID to a set to facilitate fast
|
|
||||||
# existence checks.
|
|
||||||
|
|
||||||
try:
|
|
||||||
# TODO(kgriffs): Is it faster/better to do this all
|
|
||||||
# in a Lua script instead of using an app-layer
|
|
||||||
# transaction? Lua requires Redis 2.6 or better.
|
|
||||||
|
|
||||||
# NOTE(kgriffs): Abort the entire transaction if
|
|
||||||
# another request beats us to the punch. We detect
|
|
||||||
# this by putting a watch on the key that will have
|
|
||||||
# one of its fields updated as the final step of
|
|
||||||
# the transaction.
|
|
||||||
#
|
|
||||||
# No other request to list active messages can
|
|
||||||
# proceed while this current transaction is in
|
|
||||||
# progress; therefore, it is not possible for
|
|
||||||
# a different process to get some active messages
|
|
||||||
# while the pipeline commands have partway
|
|
||||||
# completed. Either the other process will query
|
|
||||||
# for active messages at the same moment as
|
|
||||||
# the current proc and get the exact same set,
|
|
||||||
# or its request will have to wait while the
|
|
||||||
# current process performs the transaction in
|
|
||||||
# its entirety.
|
|
||||||
pipe.watch(claims_set_key)
|
|
||||||
pipe.multi()
|
|
||||||
|
|
||||||
results = self._message_ctrl._active(
|
|
||||||
queue, project=project, limit=limit)
|
|
||||||
|
|
||||||
cursor = next(results)
|
|
||||||
msg_list = list(cursor)
|
|
||||||
num_messages = len(msg_list)
|
|
||||||
|
|
||||||
# NOTE(kgriffs): If there are no active messages to
|
|
||||||
# claim, simply return an empty list.
|
|
||||||
if not msg_list:
|
|
||||||
return (None, iter([]))
|
|
||||||
|
|
||||||
basic_messages = []
|
|
||||||
|
|
||||||
now = timeutils.utcnow_ts()
|
now = timeutils.utcnow_ts()
|
||||||
|
msg_ttl = claim_ttl + grace
|
||||||
claim_expires = now + claim_ttl
|
claim_expires = now + claim_ttl
|
||||||
msg_expires = claim_expires + grace
|
msg_expires = claim_expires + grace
|
||||||
|
|
||||||
# Associate the claim with each message
|
claim_id = utils.generate_uuid()
|
||||||
for msg in msg_list:
|
claimed_msgs = []
|
||||||
msg.claim_id = claim_id
|
|
||||||
msg.claim_expires = claim_expires
|
|
||||||
|
|
||||||
if _msg_would_expire(msg, msg_expires):
|
# NOTE(kgriffs): Claim some messages
|
||||||
msg.ttl = msg_ttl
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
msg.expires = msg_expires
|
claimed_ids = self._claim_messages(msgset_key, now, limit,
|
||||||
|
claim_id, claim_expires,
|
||||||
|
msg_ttl, msg_expires)
|
||||||
|
|
||||||
pipe.rpush(claim_msgs_key, msg.id)
|
if claimed_ids:
|
||||||
|
claimed_msgs = messages.Message.from_redis_bulk(claimed_ids,
|
||||||
|
self._client)
|
||||||
|
claimed_msgs = [msg.to_basic(now) for msg in claimed_msgs]
|
||||||
|
|
||||||
# TODO(kgriffs): Rather than writing back the
|
# NOTE(kgriffs): Perist claim records
|
||||||
# entire message, only set the fields that
|
with self._client.pipeline() as pipe:
|
||||||
# have changed.
|
claim_msgs_key = utils.scope_claim_messages(
|
||||||
msg.to_redis(pipe, include_body=False)
|
claim_id, CLAIM_MESSAGES_SUFFIX)
|
||||||
|
|
||||||
basic_messages.append(msg.to_basic(now))
|
for mid in claimed_ids:
|
||||||
|
pipe.rpush(claim_msgs_key, mid)
|
||||||
|
|
||||||
pipe.expire(claim_msgs_key, claim_ttl)
|
pipe.expire(claim_msgs_key, claim_ttl)
|
||||||
|
|
||||||
# Create the claim
|
|
||||||
claim_info = {
|
claim_info = {
|
||||||
'id': claim_id,
|
'id': claim_id,
|
||||||
't': claim_ttl,
|
't': claim_ttl,
|
||||||
'e': claim_expires,
|
'e': claim_expires,
|
||||||
'n': num_messages,
|
'n': len(claimed_ids),
|
||||||
}
|
}
|
||||||
|
|
||||||
pipe.hmset(claim_id, claim_info)
|
pipe.hmset(claim_id, claim_info)
|
||||||
@ -357,15 +298,13 @@ class ClaimController(storage.Claim):
|
|||||||
#
|
#
|
||||||
# A sorted set is used to facilitate cleaning
|
# A sorted set is used to facilitate cleaning
|
||||||
# up the IDs of expired claims.
|
# up the IDs of expired claims.
|
||||||
|
claims_set_key = utils.scope_claims_set(queue, project,
|
||||||
|
QUEUE_CLAIMS_SUFFIX)
|
||||||
|
|
||||||
pipe.zadd(claims_set_key, claim_expires, claim_id)
|
pipe.zadd(claims_set_key, claim_expires, claim_id)
|
||||||
pipe.execute()
|
pipe.execute()
|
||||||
|
|
||||||
return claim_id, basic_messages
|
return claim_id, claimed_msgs
|
||||||
|
|
||||||
except redis.exceptions.WatchError:
|
|
||||||
continue
|
|
||||||
|
|
||||||
raise errors.ClaimConflict(queue, project)
|
|
||||||
|
|
||||||
@utils.raises_conn_error
|
@utils.raises_conn_error
|
||||||
@utils.retries_on_connection_error
|
@utils.retries_on_connection_error
|
||||||
|
@ -16,6 +16,7 @@ import redis
|
|||||||
from six.moves import urllib
|
from six.moves import urllib
|
||||||
|
|
||||||
from zaqar.common import decorators
|
from zaqar.common import decorators
|
||||||
|
from zaqar.i18n import _
|
||||||
from zaqar.openstack.common import log as logging
|
from zaqar.openstack.common import log as logging
|
||||||
from zaqar.queues import storage
|
from zaqar.queues import storage
|
||||||
from zaqar.queues.storage.redis import controllers
|
from zaqar.queues.storage.redis import controllers
|
||||||
@ -51,6 +52,13 @@ class DataDriver(storage.DataDriverBase):
|
|||||||
group=options.REDIS_GROUP)
|
group=options.REDIS_GROUP)
|
||||||
self.redis_conf = self.conf[options.REDIS_GROUP]
|
self.redis_conf = self.conf[options.REDIS_GROUP]
|
||||||
|
|
||||||
|
server_version = self.connection.info()['redis_version']
|
||||||
|
if tuple(map(int, server_version.split('.'))) < (2, 6):
|
||||||
|
msg = _('The Redis driver requires redis-server>=2.6, '
|
||||||
|
'%s found') % server_version
|
||||||
|
|
||||||
|
raise RuntimeError(msg)
|
||||||
|
|
||||||
def is_alive(self):
|
def is_alive(self):
|
||||||
try:
|
try:
|
||||||
return self.connection.ping()
|
return self.connection.ping()
|
||||||
|
@ -29,7 +29,6 @@ Message = models.Message
|
|||||||
MessageEnvelope = models.MessageEnvelope
|
MessageEnvelope = models.MessageEnvelope
|
||||||
|
|
||||||
|
|
||||||
MESSAGE_IDS_SUFFIX = 'messages'
|
|
||||||
MSGSET_INDEX_KEY = 'msgset_index'
|
MSGSET_INDEX_KEY = 'msgset_index'
|
||||||
|
|
||||||
# The rank counter is an atomic index to rank messages
|
# The rank counter is an atomic index to rank messages
|
||||||
@ -112,14 +111,6 @@ class MessageController(storage.Message):
|
|||||||
def _claim_ctrl(self):
|
def _claim_ctrl(self):
|
||||||
return self.driver.claim_controller
|
return self.driver.claim_controller
|
||||||
|
|
||||||
def _active(self, queue_name, marker=None, echo=False,
|
|
||||||
client_uuid=None, project=None,
|
|
||||||
limit=None):
|
|
||||||
return self._list(queue_name, project=project, marker=marker,
|
|
||||||
echo=echo, client_uuid=client_uuid,
|
|
||||||
include_claimed=False,
|
|
||||||
limit=limit, to_basic=False)
|
|
||||||
|
|
||||||
def _count(self, queue, project):
|
def _count(self, queue, project):
|
||||||
"""Return total number of messages in a queue.
|
"""Return total number of messages in a queue.
|
||||||
|
|
||||||
@ -127,22 +118,13 @@ class MessageController(storage.Message):
|
|||||||
they haven't been GC'd yet. This is done for performance.
|
they haven't been GC'd yet. This is done for performance.
|
||||||
"""
|
"""
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
return self._client.zcard(utils.msgset_key(queue, project))
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
return self._client.zcard(msgset_key)
|
|
||||||
|
|
||||||
def _create_msgset(self, queue, project, pipe):
|
def _create_msgset(self, queue, project, pipe):
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
pipe.zadd(MSGSET_INDEX_KEY, 1, utils.msgset_key(queue, project))
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
pipe.zadd(MSGSET_INDEX_KEY, 1, msgset_key)
|
|
||||||
|
|
||||||
def _delete_msgset(self, queue, project, pipe):
|
def _delete_msgset(self, queue, project, pipe):
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
pipe.zrem(MSGSET_INDEX_KEY, utils.msgset_key(queue, project))
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
pipe.zrem(MSGSET_INDEX_KEY, msgset_key)
|
|
||||||
|
|
||||||
@utils.raises_conn_error
|
@utils.raises_conn_error
|
||||||
@utils.retries_on_connection_error
|
@utils.retries_on_connection_error
|
||||||
@ -154,8 +136,7 @@ class MessageController(storage.Message):
|
|||||||
executing the operation.
|
executing the operation.
|
||||||
"""
|
"""
|
||||||
client = self._client
|
client = self._client
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
message_ids = client.zrange(msgset_key, 0, -1)
|
message_ids = client.zrange(msgset_key, 0, -1)
|
||||||
|
|
||||||
pipe.delete(msgset_key)
|
pipe.delete(msgset_key)
|
||||||
@ -166,8 +147,7 @@ class MessageController(storage.Message):
|
|||||||
def _find_first_unclaimed(self, queue, project, limit):
|
def _find_first_unclaimed(self, queue, project, limit):
|
||||||
"""Find the first unclaimed message in the queue."""
|
"""Find the first unclaimed message in the queue."""
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
now = timeutils.utcnow_ts()
|
now = timeutils.utcnow_ts()
|
||||||
|
|
||||||
# TODO(kgriffs): Generalize this paging pattern (DRY)
|
# TODO(kgriffs): Generalize this paging pattern (DRY)
|
||||||
@ -198,8 +178,7 @@ class MessageController(storage.Message):
|
|||||||
Helper function to get the first message in the queue
|
Helper function to get the first message in the queue
|
||||||
sort > 0 get from the left else from the right.
|
sort > 0 get from the left else from the right.
|
||||||
"""
|
"""
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
zrange = self._client.zrange if sort == 1 else self._client.zrevrange
|
zrange = self._client.zrange if sort == 1 else self._client.zrevrange
|
||||||
message_ids = zrange(msgset_key, 0, 0)
|
message_ids = zrange(msgset_key, 0, 0)
|
||||||
@ -242,9 +221,7 @@ class MessageController(storage.Message):
|
|||||||
raise errors.QueueDoesNotExist(queue,
|
raise errors.QueueDoesNotExist(queue,
|
||||||
project)
|
project)
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
project,
|
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
client = self._client
|
client = self._client
|
||||||
|
|
||||||
if not marker and not include_claimed:
|
if not marker and not include_claimed:
|
||||||
@ -419,9 +396,7 @@ class MessageController(storage.Message):
|
|||||||
if not self._queue_ctrl.exists(queue, project):
|
if not self._queue_ctrl.exists(queue, project):
|
||||||
raise errors.QueueDoesNotExist(queue, project)
|
raise errors.QueueDoesNotExist(queue, project)
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
counter_key = utils.scope_queue_index(queue, project,
|
counter_key = utils.scope_queue_index(queue, project,
|
||||||
MESSAGE_RANK_COUNTER_SUFFIX)
|
MESSAGE_RANK_COUNTER_SUFFIX)
|
||||||
|
|
||||||
@ -518,8 +493,8 @@ class MessageController(storage.Message):
|
|||||||
|
|
||||||
raise errors.MessageNotClaimedBy(message_id, claim)
|
raise errors.MessageNotClaimedBy(message_id, claim)
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
with self._client.pipeline() as pipe:
|
with self._client.pipeline() as pipe:
|
||||||
pipe.delete(message_id)
|
pipe.delete(message_id)
|
||||||
pipe.zrem(msgset_key, message_id)
|
pipe.zrem(msgset_key, message_id)
|
||||||
@ -538,8 +513,7 @@ class MessageController(storage.Message):
|
|||||||
raise errors.QueueDoesNotExist(queue,
|
raise errors.QueueDoesNotExist(queue,
|
||||||
project)
|
project)
|
||||||
|
|
||||||
msgset_key = utils.scope_message_ids_set(queue, project,
|
msgset_key = utils.msgset_key(queue, project)
|
||||||
MESSAGE_IDS_SUFFIX)
|
|
||||||
|
|
||||||
with self._client.pipeline() as pipe:
|
with self._client.pipeline() as pipe:
|
||||||
for mid in message_ids:
|
for mid in message_ids:
|
||||||
|
40
zaqar/queues/storage/redis/scripting.py
Normal file
40
zaqar/queues/storage/redis/scripting.py
Normal file
@ -0,0 +1,40 @@
|
|||||||
|
# Copyright (c) 2014 Rackspace Hosting, Inc.
|
||||||
|
#
|
||||||
|
# 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 os
|
||||||
|
|
||||||
|
from zaqar.common import decorators
|
||||||
|
|
||||||
|
|
||||||
|
class Mixin(object):
|
||||||
|
script_names = []
|
||||||
|
|
||||||
|
@decorators.lazy_property(write=False)
|
||||||
|
def _scripts(self):
|
||||||
|
scripts = {}
|
||||||
|
|
||||||
|
for name in self.script_names:
|
||||||
|
script = _read_script(name)
|
||||||
|
scripts[name] = self._client.register_script(script)
|
||||||
|
|
||||||
|
return scripts
|
||||||
|
|
||||||
|
|
||||||
|
def _read_script(script_name):
|
||||||
|
folder = os.path.abspath(os.path.dirname(__file__))
|
||||||
|
filename = os.path.join(folder, 'scripts', script_name + '.lua')
|
||||||
|
|
||||||
|
with open(filename, 'r') as script_file:
|
||||||
|
return script_file.read()
|
88
zaqar/queues/storage/redis/scripts/claim_messages.lua
Normal file
88
zaqar/queues/storage/redis/scripts/claim_messages.lua
Normal file
@ -0,0 +1,88 @@
|
|||||||
|
--[[
|
||||||
|
|
||||||
|
Copyright (c) 2014 Rackspace Hosting, Inc.
|
||||||
|
|
||||||
|
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.
|
||||||
|
|
||||||
|
--]]
|
||||||
|
|
||||||
|
-- Read params
|
||||||
|
local msgset_key = KEYS[1]
|
||||||
|
|
||||||
|
local now = tonumber(ARGV[1])
|
||||||
|
local limit = tonumber(ARGV[2])
|
||||||
|
local claim_id = ARGV[3]
|
||||||
|
local claim_expires = tonumber(ARGV[4])
|
||||||
|
local msg_ttl = tonumber(ARGV[5])
|
||||||
|
local msg_expires = tonumber(ARGV[6])
|
||||||
|
|
||||||
|
-- Scan for up to 'limit' unclaimed messages
|
||||||
|
local BATCH_SIZE = 100
|
||||||
|
|
||||||
|
local start = 0
|
||||||
|
local claimed_msgs = {}
|
||||||
|
|
||||||
|
local found_unclaimed = false
|
||||||
|
|
||||||
|
while (#claimed_msgs < limit) do
|
||||||
|
local stop = (start + BATCH_SIZE - 1)
|
||||||
|
local msg_ids = redis.call('ZRANGE', msgset_key, start, stop)
|
||||||
|
|
||||||
|
if (#msg_ids == 0) then
|
||||||
|
break
|
||||||
|
end
|
||||||
|
|
||||||
|
start = start + BATCH_SIZE
|
||||||
|
|
||||||
|
-- TODO(kgriffs): Try moving claimed IDs to a different set
|
||||||
|
-- to avoid scanning through already-claimed messages.
|
||||||
|
for i, mid in ipairs(msg_ids) do
|
||||||
|
-- NOTE(kgriffs): Since execution of this script can not
|
||||||
|
-- happen in parallel, once we find the first unclaimed
|
||||||
|
-- message, the remaining messages will always be
|
||||||
|
-- unclaimed as well.
|
||||||
|
|
||||||
|
if not found_unclaimed then
|
||||||
|
local msg = redis.call('HMGET', mid, 'c', 'c.e')
|
||||||
|
|
||||||
|
if msg[1] == '' or tonumber(msg[2]) <= now then
|
||||||
|
found_unclaimed = true
|
||||||
|
end
|
||||||
|
end
|
||||||
|
|
||||||
|
if found_unclaimed then
|
||||||
|
local msg_expires_prev = redis.call('HGET', mid, 'e')
|
||||||
|
|
||||||
|
-- Found an unclaimed message, so claim it
|
||||||
|
redis.call('HMSET', mid,
|
||||||
|
'c', claim_id,
|
||||||
|
'c.e', claim_expires)
|
||||||
|
|
||||||
|
-- Will the message expire early?
|
||||||
|
if tonumber(msg_expires_prev) < msg_expires then
|
||||||
|
redis.call('HMSET', mid,
|
||||||
|
't', msg_ttl,
|
||||||
|
'e', msg_expires)
|
||||||
|
end
|
||||||
|
|
||||||
|
claimed_msgs[#claimed_msgs + 1] = mid
|
||||||
|
|
||||||
|
if (#claimed_msgs == limit) then
|
||||||
|
break
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
end
|
||||||
|
|
||||||
|
return claimed_msgs
|
@ -26,6 +26,7 @@ from zaqar.openstack.common import log as logging
|
|||||||
from zaqar.queues.storage import errors
|
from zaqar.queues.storage import errors
|
||||||
|
|
||||||
LOG = logging.getLogger(__name__)
|
LOG = logging.getLogger(__name__)
|
||||||
|
MESSAGE_IDS_SUFFIX = 'messages'
|
||||||
|
|
||||||
|
|
||||||
def descope_queue_name(scoped_name):
|
def descope_queue_name(scoped_name):
|
||||||
@ -104,6 +105,10 @@ scope_queue_catalogue = scope_claims_set = scope_message_ids_set
|
|||||||
scope_queue_index = scope_message_ids_set
|
scope_queue_index = scope_message_ids_set
|
||||||
|
|
||||||
|
|
||||||
|
def msgset_key(queue, project=None):
|
||||||
|
return scope_message_ids_set(queue, project, MESSAGE_IDS_SUFFIX)
|
||||||
|
|
||||||
|
|
||||||
def raises_conn_error(func):
|
def raises_conn_error(func):
|
||||||
"""Handles the Redis ConnectionFailure error.
|
"""Handles the Redis ConnectionFailure error.
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user