Unverified Commit c4b78823 authored by perryitay's avatar perryitay Committed by GitHub
Browse files

Adding module api for processing commands during busy jobs and allow flagging...


Adding module api for processing commands during busy jobs and allow flagging the commands that should be handled at this status (#9963)

Some modules might perform a long-running logic in different stages of Redis lifetime, for example:
* command execution
* RDB loading
* thread safe context

During this long-running logic Redis is not responsive.

This PR offers 
1. An API to process events while a busy command is running (`RM_Yield`)
2. A new flag (`ALLOW_BUSY`) to mark the commands that should be handled during busy
  jobs which can also be used by modules (`allow-busy`)
3. In slow commands and thread safe contexts, this flag will start rejecting commands with -BUSY only
  after `busy-reply-threshold`
4. During loading (`rdb_load` callback), it'll process events right away (not wait for `busy-reply-threshold`),
  but either way, the processing is throttled to the server hz rate.
5. Allow modules to Yield to redis background tasks, but not to client commands

* rename `script-time-limit` to `busy-reply-threshold` (an alias to the pre-7.0 `lua-time-limit`)
Co-authored-by: default avatarOran Agra <oran@redislabs.com>
parent 22172a4a
......@@ -194,7 +194,7 @@ client *createClient(connection *conn) {
c->peerid = NULL;
c->sockname = NULL;
c->client_list_node = NULL;
c->paused_list_node = NULL;
c->postponed_list_node = NULL;
c->pending_read_list_node = NULL;
c->client_tracking_redirection = 0;
c->client_tracking_prefixes = NULL;
......@@ -3628,13 +3628,19 @@ static void updateClientPauseTypeAndEndTime(void) {
/* If the pause type is less restrictive than before, we unblock all clients
* so they are reprocessed (may get re-paused). */
if (type < old_type) {
listNode *ln;
listIter li;
listRewind(server.paused_clients, &li);
while ((ln = listNext(&li)) != NULL) {
client *c = listNodeValue(ln);
unblockClient(c);
}
unblockPostponedClients();
}
}
/* Unblock all paused clients (ones that where blocked by BLOCKED_POSTPONE (possibly in processCommand).
* This means they'll get re-processed in beforeSleep, and may get paused again if needed. */
void unblockPostponedClients() {
listNode *ln;
listIter li;
listRewind(server.postponed_clients, &li);
while ((ln = listNext(&li)) != NULL) {
client *c = listNodeValue(ln);
unblockClient(c);
}
}
......
......@@ -218,6 +218,10 @@ This flag should not be used directly by the module.
#define REDISMODULE_AUX_BEFORE_RDB (1<<0)
#define REDISMODULE_AUX_AFTER_RDB (1<<1)
/* RM_Yield flags */
#define REDISMODULE_YIELD_FLAG_NONE (1<<0)
#define REDISMODULE_YIELD_FLAG_CLIENTS (1<<1)
/* This type represents a timer handle, and is returned when a timer is
* registered and used in order to invalidate a timer. It's just a 64 bit
* number, because this is how each timer is represented inside the radix tree
......@@ -919,7 +923,7 @@ REDISMODULE_API int (*RedisModule_SetCommandKeySpecBeginSearchIndex)(RedisModule
REDISMODULE_API int (*RedisModule_SetCommandKeySpecBeginSearchKeyword)(RedisModuleCommand *command, int spec_id, const char *keyword, int startfrom) REDISMODULE_ATTR;
REDISMODULE_API int (*RedisModule_SetCommandKeySpecFindKeysRange)(RedisModuleCommand *command, int spec_id, int lastkey, int keystep, int limit) REDISMODULE_ATTR;
REDISMODULE_API int (*RedisModule_SetCommandKeySpecFindKeysKeynum)(RedisModuleCommand *command, int spec_id, int keynumidx, int firstkey, int keystep) REDISMODULE_ATTR;
REDISMODULE_API void (*RedisModule_Yield)(RedisModuleCtx *ctx, int flags, const char *busy_reply) REDISMODULE_ATTR;
REDISMODULE_API RedisModuleBlockedClient * (*RedisModule_BlockClient)(RedisModuleCtx *ctx, RedisModuleCmdFunc reply_callback, RedisModuleCmdFunc timeout_callback, void (*free_privdata)(RedisModuleCtx*,void*), long long timeout_ms) REDISMODULE_ATTR;
REDISMODULE_API int (*RedisModule_UnblockClient)(RedisModuleBlockedClient *bc, void *privdata) REDISMODULE_ATTR;
REDISMODULE_API int (*RedisModule_IsBlockedReplyRequest)(RedisModuleCtx *ctx) REDISMODULE_ATTR;
......@@ -1242,6 +1246,7 @@ static int RedisModule_Init(RedisModuleCtx *ctx, const char *name, int ver, int
REDISMODULE_GET_API(SetCommandKeySpecBeginSearchKeyword);
REDISMODULE_GET_API(SetCommandKeySpecFindKeysRange);
REDISMODULE_GET_API(SetCommandKeySpecFindKeysKeynum);
REDISMODULE_GET_API(Yield);
REDISMODULE_GET_API(GetThreadSafeContext);
REDISMODULE_GET_API(GetDetachedThreadSafeContext);
REDISMODULE_GET_API(FreeThreadSafeContext);
......
......@@ -85,7 +85,7 @@ int scriptInterrupt(scriptRunCtx *run_ctx) {
}
long long elapsed = elapsedMs(run_ctx->start_time);
if (elapsed < server.script_time_limit) {
if (elapsed < server.busy_reply_threshold) {
return SCRIPT_CONTINUE;
}
......
......@@ -1363,7 +1363,7 @@ void luaCallFunction(scriptRunCtx* run_ctx, lua_State *lua, robj** keys, size_t
* each time the Lua hook is invoked. */
luaSaveOnRegistry(lua, REGISTRY_RUN_CTX_NAME, run_ctx);
if (server.script_time_limit > 0 && !debug_enabled) {
if (server.busy_reply_threshold > 0 && !debug_enabled) {
lua_sethook(lua,luaMaskCountHook,LUA_MASKCOUNT,100000);
delhook = 1;
} else if (debug_enabled) {
......
......@@ -1613,6 +1613,8 @@ void createSharedObjects(void) {
"-BUSY Redis is busy running a script. You can only call SCRIPT KILL or SHUTDOWN NOSAVE.\r\n"));
shared.slowscripterr = createObject(OBJ_STRING,sdsnew(
"-BUSY Redis is busy running a script. You can only call FUNCTION KILL or SHUTDOWN NOSAVE.\r\n"));
shared.slowmoduleerr = createObject(OBJ_STRING,sdsnew(
"-BUSY Redis is busy running a module command.\r\n"));
shared.masterdownerr = createObject(OBJ_STRING,sdsnew(
"-MASTERDOWN Link with MASTER is down and replica-serve-stale-data is set to 'no'.\r\n"));
shared.bgsaveerr = createObject(OBJ_STRING,sdsnew(
......@@ -2324,7 +2326,7 @@ void initServer(void) {
server.client_pause_end_time = 0;
memset(server.client_pause_per_purpose, 0,
sizeof(server.client_pause_per_purpose));
server.paused_clients = listCreate();
server.postponed_clients = listCreate();
server.events_processed_while_blocked = 0;
server.system_memory_size = zmalloc_get_memory_size();
server.blocked_last_cron = 0;
......@@ -2411,6 +2413,8 @@ void initServer(void) {
server.cronloops = 0;
server.in_script = 0;
server.in_exec = 0;
server.busy_module_yield_flags = BUSY_MODULE_YIELD_NONE;
server.busy_module_yield_reply = NULL;
server.core_propagates = 0;
server.propagate_no_multi = 0;
server.module_ctx_nesting = 0;
......@@ -3367,6 +3371,16 @@ int processCommand(client *c) {
return C_ERR;
}
/* If we're inside a module blocked context yielding that wants to avoid
* processing clients, postpone the command. */
if (server.busy_module_yield_flags != BUSY_MODULE_YIELD_NONE &&
!(server.busy_module_yield_flags & BUSY_MODULE_YIELD_CLIENTS))
{
c->bpop.timeout = 0;
blockClient(c,BLOCKED_POSTPONE);
return C_OK;
}
/* Now lookup the command and check ASAP about trivial error conditions
* such as wrong arity, bad command name and so forth. */
c->cmd = c->lastcmd = lookupCommand(c->argv,c->argc);
......@@ -3645,30 +3659,19 @@ int processCommand(client *c) {
return C_OK;
}
/* Lua script too slow? Only allow a limited number of commands.
/* when a busy job is being done (script / module)
* Only allow a limited number of commands.
* Note that we need to allow the transactions commands, otherwise clients
* sending a transaction with pipelining without error checking, may have
* the MULTI plus a few initial commands refused, then the timeout
* condition resolves, and the bottom-half of the transaction gets
* executed, see Github PR #7022. */
if (scriptIsTimedout() &&
c->cmd->proc != authCommand &&
c->cmd->proc != helloCommand &&
c->cmd->proc != replconfCommand &&
c->cmd->proc != multiCommand &&
c->cmd->proc != discardCommand &&
c->cmd->proc != watchCommand &&
c->cmd->proc != unwatchCommand &&
c->cmd->proc != quitCommand &&
c->cmd->proc != resetCommand &&
c->cmd->proc != shutdownCommand && /* more checks in shutdownCommand */
!(c->cmd->proc == scriptCommand &&
c->argc == 2 &&
tolower(((char*)c->argv[1]->ptr)[0]) == 'k') &&
!(c->cmd->proc == functionKillCommand) &&
!(c->cmd->proc == functionStatsCommand))
{
if (scriptIsEval()) {
if ((scriptIsTimedout() || server.busy_module_yield_flags) && !(c->cmd->flags & CMD_ALLOW_BUSY)) {
if (server.busy_module_yield_flags && server.busy_module_yield_reply) {
rejectCommandFormat(c, "-BUSY %s", server.busy_module_yield_reply);
} else if (server.busy_module_yield_flags) {
rejectCommand(c, shared.slowmoduleerr);
} else if (scriptIsEval()) {
rejectCommand(c, shared.slowevalerr);
} else {
rejectCommand(c, shared.slowscripterr);
......@@ -3691,7 +3694,7 @@ int processCommand(client *c) {
(server.client_pause_type == CLIENT_PAUSE_WRITE && is_may_replicate_command)))
{
c->bpop.timeout = 0;
blockClient(c,BLOCKED_PAUSE);
blockClient(c,BLOCKED_POSTPONE);
return C_OK;
}
......@@ -4111,6 +4114,7 @@ void addReplyFlagsForCommand(client *c, struct redisCommand *cmd) {
{CMD_NO_ASYNC_LOADING, "no_async_loading"},
{CMD_NO_MULTI, "no_multi"},
{CMD_MOVABLE_KEYS, "movablekeys"},
{CMD_ALLOW_BUSY, "allow_busy"},
{0,NULL}
};
/* "sentinel" and "only-sentinel" are hidden on purpose. */
......
......@@ -209,6 +209,7 @@ extern int configOOMScoreAdjValuesDefaults[CONFIG_OOM_COUNT];
#define CMD_NO_ASYNC_LOADING (1ULL<<23)
#define CMD_NO_MULTI (1ULL<<24)
#define CMD_MOVABLE_KEYS (1ULL<<25) /* populated by populateCommandMovableKeys */
#define CMD_ALLOW_BUSY ((1ULL<<26))
/* Command flags that describe ACLs categories. */
#define ACL_CATEGORY_KEYSPACE (1ULL<<0)
......@@ -349,7 +350,7 @@ extern int configOOMScoreAdjValuesDefaults[CONFIG_OOM_COUNT];
#define BLOCKED_MODULE 3 /* Blocked by a loadable module. */
#define BLOCKED_STREAM 4 /* XREAD. */
#define BLOCKED_ZSET 5 /* BZPOP et al. */
#define BLOCKED_PAUSE 6 /* Blocked by CLIENT PAUSE */
#define BLOCKED_POSTPONE 6 /* Blocked by processCommand, re-try processing later. */
#define BLOCKED_SHUTDOWN 7 /* SHUTDOWN. */
#define BLOCKED_NUM 8 /* Number of blocked states. */
......@@ -603,6 +604,11 @@ typedef enum {
* Value quantization within the range will thus be no larger than 1/100th (or 1%) of any value.
* The total size per histogram should sit around 40 KiB Bytes. */
/* Busy module flags, see busy_module_yield_flags */
#define BUSY_MODULE_YIELD_NONE (0)
#define BUSY_MODULE_YIELD_EVENTS (1<<0)
#define BUSY_MODULE_YIELD_CLIENTS (1<<1)
/*-----------------------------------------------------------------------------
* Data types
*----------------------------------------------------------------------------*/
......@@ -1136,7 +1142,7 @@ typedef struct client {
sds peerid; /* Cached peer ID. */
sds sockname; /* Cached connection target address. */
listNode *client_list_node; /* list node in client list */
listNode *paused_list_node; /* list node within the pause list */
listNode *postponed_list_node; /* list node within the postponed list */
listNode *pending_read_list_node; /* list node in clients pending read list */
RedisModuleUserChangedFunc auth_callback; /* Module callback to execute
* when the authenticated user
......@@ -1211,7 +1217,8 @@ struct sharedObjectsStruct {
robj *crlf, *ok, *err, *emptybulk, *czero, *cone, *pong, *space,
*queued, *null[4], *nullarray[4], *emptymap[4], *emptyset[4],
*emptyarray, *wrongtypeerr, *nokeyerr, *syntaxerr, *sameobjecterr,
*outofrangeerr, *noscripterr, *loadingerr, *slowevalerr, *slowscripterr, *bgsaveerr,
*outofrangeerr, *noscripterr, *loadingerr,
*slowevalerr, *slowscripterr, *slowmoduleerr, *bgsaveerr,
*masterdownerr, *roslaveerr, *execaborterr, *noautherr, *noreplicaserr,
*busykeyerr, *oomerr, *plus, *messagebulk, *pmessagebulk, *subscribebulk,
*unsubscribebulk, *psubscribebulk, *punsubscribebulk, *del, *unlink,
......@@ -1459,6 +1466,8 @@ struct redisServer {
int always_show_logo; /* Show logo even for non-stdout logging. */
int in_script; /* Are we inside EVAL? */
int in_exec; /* Are we inside EXEC? */
int busy_module_yield_flags; /* Are we inside a busy module? (triggered by RM_Yield). see BUSY_MODULE_YIELD_ flags. */
const char *busy_module_yield_reply; /* When non-null, we are inside RM_Yield. */
int core_propagates; /* Is the core (in oppose to the module subsystem) is in charge of calling propagatePendingCommands? */
int propagate_no_multi; /* True if propagatePendingCommands should avoid wrapping command in MULTI/EXEC */
int module_ctx_nesting; /* moduleCreateContext() nesting level */
......@@ -1502,7 +1511,7 @@ struct redisServer {
int in_nested_call; /* If > 0, in a nested call of a call */
rax *clients_index; /* Active clients dictionary by client ID. */
pause_type client_pause_type; /* True if clients are currently paused */
list *paused_clients; /* List of pause clients */
list *postponed_clients; /* List of postponed clients */
mstime_t client_pause_end_time; /* Time when we undo clients_paused */
pause_event *client_pause_per_purpose[NUM_PAUSE_PURPOSES];
char neterr[ANET_ERR_LEN]; /* Error buffer for anet.c */
......@@ -1856,7 +1865,7 @@ struct redisServer {
* dropping packets of a specific type */
/* Scripting */
client *script_caller; /* The client running script right now, or NULL */
mstime_t script_time_limit; /* Script timeout in milliseconds */
mstime_t busy_reply_threshold; /* Script / module timeout in milliseconds */
int script_oom; /* OOM detected when script start */
int script_disable_deny_script; /* Allow running commands marked "no-script" inside a script. */
/* Lazy free */
......@@ -2461,6 +2470,7 @@ void pauseClients(pause_purpose purpose, mstime_t end, pause_type type);
void unpauseClients(pause_purpose purpose);
int areClientsPaused(void);
int checkClientPauseTimeoutAndReturnIfPaused(void);
void unblockPostponedClients();
void processEventsWhileBlocked(void);
void whileBlockedCron();
void blockingOperationStarts();
......
/* define macros for having usleep */
#define _BSD_SOURCE
#define _DEFAULT_SOURCE
#include <unistd.h>
#include "redismodule.h"
#include <assert.h>
#include <stdio.h>
......@@ -5,6 +10,10 @@
#define UNUSED(V) ((void) V)
/* used to test processing events during slow bg operation */
static volatile int g_slow_bg_operation = 0;
static volatile int g_is_in_slow_bg_operation = 0;
void *sub_worker(void *arg) {
// Get Redis module context
RedisModuleCtx *ctx = (RedisModuleCtx *)arg;
......@@ -99,6 +108,16 @@ void *bg_call_worker(void *arg) {
// Acquire GIL
RedisModule_ThreadSafeContextLock(ctx);
// Test slow operation yielding
if (g_slow_bg_operation) {
g_is_in_slow_bg_operation = 1;
while (g_slow_bg_operation) {
RedisModule_Yield(ctx, REDISMODULE_YIELD_FLAG_CLIENTS, "Slow module operation");
usleep(1000);
}
g_is_in_slow_bg_operation = 0;
}
// Call the command
const char* cmd = RedisModule_StringPtrLen(bg->argv[1], NULL);
RedisModuleCallReply* rep = RedisModule_Call(ctx, cmd, "v", bg->argv + 2, bg->argc - 2);
......@@ -203,6 +222,73 @@ int do_fake_bg_true(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
return REDISMODULE_OK;
}
/* this flag is used to work with busy commands, that might take a while
* and ability to stop the busy work with a different command*/
static volatile int abort_flag = 0;
int slow_fg_command(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
if (argc != 2) {
RedisModule_WrongArity(ctx);
return REDISMODULE_OK;
}
long long block_time = 0;
if (RedisModule_StringToLongLong(argv[1], &block_time) != REDISMODULE_OK) {
RedisModule_ReplyWithError(ctx, "Invalid integer value");
return REDISMODULE_OK;
}
uint64_t start_time = RedisModule_MonotonicMicroseconds();
/* when not blocking indefinitely, we don't process client commands in this test. */
int yield_flags = block_time? REDISMODULE_YIELD_FLAG_NONE: REDISMODULE_YIELD_FLAG_CLIENTS;
while (!abort_flag) {
RedisModule_Yield(ctx, yield_flags, "Slow module operation");
usleep(1000);
if (block_time && RedisModule_MonotonicMicroseconds() - start_time > (uint64_t)block_time)
break;
}
abort_flag = 0;
RedisModule_ReplyWithLongLong(ctx, 1);
return REDISMODULE_OK;
}
int stop_slow_fg_command(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
abort_flag = 1;
RedisModule_ReplyWithLongLong(ctx, 1);
return REDISMODULE_OK;
}
/* used to enable or disable slow operation in do_bg_rm_call */
static int set_slow_bg_operation(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
if (argc != 2) {
RedisModule_WrongArity(ctx);
return REDISMODULE_OK;
}
long long ll;
if (RedisModule_StringToLongLong(argv[1], &ll) != REDISMODULE_OK) {
RedisModule_ReplyWithError(ctx, "Invalid integer value");
return REDISMODULE_OK;
}
g_slow_bg_operation = ll;
RedisModule_ReplyWithSimpleString(ctx, "OK");
return REDISMODULE_OK;
}
/* used to test if we reached the slow operation in do_bg_rm_call */
static int is_in_slow_bg_operation(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
UNUSED(argv);
if (argc != 1) {
RedisModule_WrongArity(ctx);
return REDISMODULE_OK;
}
RedisModule_ReplyWithLongLong(ctx, g_is_in_slow_bg_operation);
return REDISMODULE_OK;
}
int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
......@@ -223,5 +309,17 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
if (RedisModule_CreateCommand(ctx, "do_fake_bg_true", do_fake_bg_true, "", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "slow_fg_command", slow_fg_command,"", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "stop_slow_fg_command", stop_slow_fg_command,"allow-busy", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "set_slow_bg_operation", set_slow_bg_operation, "allow-busy", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "is_in_slow_bg_operation", is_in_slow_bg_operation, "allow-busy", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
return REDISMODULE_OK;
}
......@@ -2,11 +2,20 @@
* for general ModuleDataType coverage.
*/
/* define macros for having usleep */
#define _BSD_SOURCE
#define _DEFAULT_SOURCE
#include <unistd.h>
#include "redismodule.h"
static RedisModuleType *datatype = NULL;
static int load_encver = 0;
/* used to test processing events during slow loading */
static volatile int slow_loading = 0;
static volatile int is_in_slow_loading = 0;
#define DATATYPE_ENC_VER 1
typedef struct {
......@@ -25,6 +34,17 @@ static void *datatype_load(RedisModuleIO *io, int encver) {
DataType *dt = (DataType *) RedisModule_Alloc(sizeof(DataType));
dt->intval = intval;
dt->strval = strval;
if (slow_loading) {
RedisModuleCtx *ctx = RedisModule_GetContextFromIO(io);
is_in_slow_loading = 1;
while (slow_loading) {
RedisModule_Yield(ctx, REDISMODULE_YIELD_FLAG_CLIENTS, "Slow module operation");
usleep(1000);
}
is_in_slow_loading = 0;
}
return dt;
}
......@@ -185,6 +205,35 @@ static int datatype_swap(RedisModuleCtx *ctx, RedisModuleString **argv, int argc
return REDISMODULE_OK;
}
/* used to enable or disable slow loading */
static int datatype_slow_loading(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
if (argc != 2) {
RedisModule_WrongArity(ctx);
return REDISMODULE_OK;
}
long long ll;
if (RedisModule_StringToLongLong(argv[1], &ll) != REDISMODULE_OK) {
RedisModule_ReplyWithError(ctx, "Invalid integer value");
return REDISMODULE_OK;
}
slow_loading = ll;
RedisModule_ReplyWithSimpleString(ctx, "OK");
return REDISMODULE_OK;
}
/* used to test if we reached the slow loading code */
static int datatype_is_in_slow_loading(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
REDISMODULE_NOT_USED(argv);
if (argc != 1) {
RedisModule_WrongArity(ctx);
return REDISMODULE_OK;
}
RedisModule_ReplyWithLongLong(ctx, is_in_slow_loading);
return REDISMODULE_OK;
}
int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc) {
REDISMODULE_NOT_USED(argv);
REDISMODULE_NOT_USED(argc);
......@@ -224,5 +273,13 @@ int RedisModule_OnLoad(RedisModuleCtx *ctx, RedisModuleString **argv, int argc)
"write", 1, 1, 1) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "datatype.slow_loading", datatype_slow_loading,
"allow-loading", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
if (RedisModule_CreateCommand(ctx, "datatype.is_in_slow_loading", datatype_is_in_slow_loading,
"allow-loading", 0, 0, 0) == REDISMODULE_ERR)
return REDISMODULE_ERR;
return REDISMODULE_OK;
}
......@@ -239,7 +239,7 @@ start_server {tags {"scripting"}} {
test {FUNCTION - test function kill} {
set rd [redis_deferring_client]
r config set script-time-limit 10
r config set busy-reply-threshold 10
r function load lua test REPLACE [get_function_code test {local a = 1 while true do a = a + 1 end}]
$rd fcall test 0
after 200
......@@ -253,7 +253,7 @@ start_server {tags {"scripting"}} {
test {FUNCTION - test script kill not working on function} {
set rd [redis_deferring_client]
r config set script-time-limit 10
r config set busy-reply-threshold 10
r function load lua test REPLACE [get_function_code test {local a = 1 while true do a = a + 1 end}]
$rd fcall test 0
after 200
......@@ -268,7 +268,7 @@ start_server {tags {"scripting"}} {
test {FUNCTION - test function kill not working on eval} {
set rd [redis_deferring_client]
r config set script-time-limit 10
r config set busy-reply-threshold 10
$rd eval {local a = 1 while true do a = a + 1 end} 0
after 200
catch {r ping} e
......
......@@ -92,6 +92,83 @@ start_server {tags {"modules"}} {
}
}
test {Busy module command} {
set busy_time_limit 50
set old_time_limit [lindex [r config get busy-reply-threshold] 1]
r config set busy-reply-threshold $busy_time_limit
set rd [redis_deferring_client]
# run command that blocks until released
set start [clock clicks -milliseconds]
$rd slow_fg_command 0
$rd flush
# make sure we get BUSY error, and that we didn't get it too early
assert_error {*BUSY Slow module operation*} {r ping}
assert_morethan_equal [expr [clock clicks -milliseconds]-$start] $busy_time_limit
# abort the blocking operation
r stop_slow_fg_command
wait_for_condition 50 100 {
[catch {r ping} e] == 0
} else {
fail "Failed waiting for busy command to end"
}
$rd read
#run command that blocks for 200ms
set start [clock clicks -milliseconds]
$rd slow_fg_command 200000
$rd flush
after 10 ;# try to make sure redis started running the command before we proceed
# make sure we didn't get BUSY error, it simply blocked till the command was done
r ping
assert_morethan_equal [expr [clock clicks -milliseconds]-$start] 200
$rd read
$rd close
r config set busy-reply-threshold $old_time_limit
}
test {RM_Call from blocked client} {
set busy_time_limit 50
set old_time_limit [lindex [r config get busy-reply-threshold] 1]
r config set busy-reply-threshold $busy_time_limit
# trigger slow operation
r set_slow_bg_operation 1
r hset hash foo bar
set rd [redis_deferring_client]
set start [clock clicks -milliseconds]
$rd do_bg_rm_call hgetall hash
# wait till we know we're blocked inside the module
wait_for_condition 50 100 {
[r is_in_slow_bg_operation] eq 1
} else {
fail "Failed waiting for slow operation to start"
}
# make sure we get BUSY error, and that we didn't get here too early
assert_error {*BUSY Slow module operation*} {r ping}
assert_morethan [expr [clock clicks -milliseconds]-$start] $busy_time_limit
# abort the blocking operation
r set_slow_bg_operation 0
wait_for_condition 50 100 {
[r is_in_slow_bg_operation] eq 0
} else {
fail "Failed waiting for slow operation to stop"
}
assert_equal [r ping] {PONG}
r config set busy-reply-threshold $old_time_limit
set res [$rd read]
$rd close
set _ $res
} {foo bar}
test {blocked client reaches client output buffer limit} {
r hset hash big [string repeat x 50000]
r hset hash bada [string repeat x 50000]
......
......@@ -55,4 +55,34 @@ start_server {tags {"modules"}} {
r copy sourcekey targetkey
r datatype.get targetkey
} {1234 AAA/sourcekey/targetkey}
test {DataType: Slow Loading} {
r config set busy-reply-threshold 5000 ;# make sure we're using a high default
# trigger slow loading
r datatype.slow_loading 1
set rd [redis_deferring_client]
set start [clock clicks -milliseconds]
$rd debug reload
# wait till we know we're blocked inside the module
wait_for_condition 50 100 {
[r datatype.is_in_slow_loading] eq 1
} else {
fail "Failed waiting for slow loading to start"
}
# make sure we get LOADING error, and that we didn't get here late (not waiting for busy-reply-threshold)
assert_error {*LOADING*} {r ping}
assert_lessthan [expr [clock clicks -milliseconds]-$start] 2000
# abort the blocking operation
r datatype.slow_loading 0
wait_for_condition 50 100 {
[s loading] eq {0}
} else {
fail "Failed waiting for loading to end"
}
$rd read
$rd close
}
}
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment