From 8e1d760da8d1b6c8d2485bfc88afd664503a3114 Mon Sep 17 00:00:00 2001 From: Uri Yagelnik Date: Mon, 8 Jul 2024 07:05:46 +0000 Subject: [PATCH] Async IO threads --- .github/workflows/daily.yml | 4 +- src/Makefile | 2 +- src/ae.c | 4 +- src/ae.h | 5 +- src/blocked.c | 2 +- src/config.c | 6 +- src/config.h | 11 +- src/connection.h | 18 + src/debug.c | 2 + src/eval.c | 2 +- src/io_threads.c | 377 ++++++ src/io_threads.h | 13 + src/networking.c | 1569 ++++++++++++------------ src/rdb.c | 2 +- src/replication.c | 22 +- src/server.c | 112 +- src/server.h | 107 +- src/socket.c | 2 + src/tls.c | 55 +- src/unix.c | 2 + tests/integration/failover.tcl | 6 + tests/integration/replication.tcl | 1 + tests/integration/shutdown.tcl | 12 + tests/integration/valkey-cli.tcl | 1 + tests/unit/client-eviction.tcl | 23 +- tests/unit/cluster/pubsubshard.tcl | 8 +- tests/unit/dump.tcl | 1 + tests/unit/info.tcl | 83 +- tests/unit/maxmemory.tcl | 1 + tests/unit/memefficiency.tcl | 3 + tests/unit/moduleapi/blockedclient.tcl | 2 +- tests/unit/pubsub.tcl | 12 + tests/unit/pubsubshard.tcl | 8 + tests/unit/querybuf.tcl | 2 +- tests/unit/type/list.tcl | 7 + tests/unit/type/stream-cgroups.tcl | 11 +- tests/unit/type/zset.tcl | 1 + valkey.conf | 21 +- 38 files changed, 1551 insertions(+), 969 deletions(-) create mode 100644 src/io_threads.c create mode 100644 src/io_threads.h diff --git a/.github/workflows/daily.yml b/.github/workflows/daily.yml index 7679856d1d..91dbb26fab 100644 --- a/.github/workflows/daily.yml +++ b/.github/workflows/daily.yml @@ -358,10 +358,10 @@ jobs: run: sudo apt-get install tcl8.6 tclx - name: test if: true && !contains(github.event.inputs.skiptests, 'valkey') - run: ./runtest --config io-threads 4 --config io-threads-do-reads yes --accurate --verbose --tags network --dump-logs ${{github.event.inputs.test_args}} + run: ./runtest --config io-threads 2 --config events-per-io-thread 0 --accurate --verbose --tags network --dump-logs ${{github.event.inputs.test_args}} - name: cluster tests if: true && !contains(github.event.inputs.skiptests, 'cluster') - run: ./runtest-cluster --config io-threads 4 --config io-threads-do-reads yes ${{github.event.inputs.cluster_test_args}} + run: ./runtest-cluster --config io-threads 2 --config events-per-io-thread 0 ${{github.event.inputs.cluster_test_args}} test-ubuntu-reclaim-cache: runs-on: ubuntu-latest diff --git a/src/Makefile b/src/Makefile index 18e5527eff..4e8c34b253 100644 --- a/src/Makefile +++ b/src/Makefile @@ -401,7 +401,7 @@ endif ENGINE_NAME=valkey SERVER_NAME=$(ENGINE_NAME)-server$(PROG_SUFFIX) ENGINE_SENTINEL_NAME=$(ENGINE_NAME)-sentinel$(PROG_SUFFIX) -ENGINE_SERVER_OBJ=threads_mngr.o adlist.o quicklist.o ae.o anet.o dict.o kvstore.o server.o sds.o zmalloc.o lzf_c.o lzf_d.o pqsort.o zipmap.o sha1.o ziplist.o release.o networking.o util.o object.o db.o replication.o rdb.o t_string.o t_list.o t_set.o t_zset.o t_hash.o config.o aof.o pubsub.o multi.o debug.o sort.o intset.o syncio.o cluster.o cluster_legacy.o cluster_slot_stats.o crc16.o endianconv.o slowlog.o eval.o bio.o rio.o rand.o memtest.o syscheck.o crcspeed.o crccombine.o crc64.o bitops.o sentinel.o notify.o setproctitle.o blocked.o hyperloglog.o latency.o sparkline.o valkey-check-rdb.o valkey-check-aof.o geo.o lazyfree.o module.o evict.o expire.o geohash.o geohash_helper.o childinfo.o defrag.o siphash.o rax.o t_stream.o listpack.o localtime.o lolwut.o lolwut5.o lolwut6.o acl.o tracking.o socket.o tls.o sha256.o timeout.o setcpuaffinity.o monotonic.o mt19937-64.o resp_parser.o call_reply.o script_lua.o script.o functions.o function_lua.o commands.o strl.o connection.o unix.o logreqres.o +ENGINE_SERVER_OBJ=threads_mngr.o adlist.o quicklist.o ae.o anet.o dict.o kvstore.o server.o sds.o zmalloc.o lzf_c.o lzf_d.o pqsort.o zipmap.o sha1.o ziplist.o release.o io_threads.o networking.o util.o object.o db.o replication.o rdb.o t_string.o t_list.o t_set.o t_zset.o t_hash.o config.o aof.o pubsub.o multi.o debug.o sort.o intset.o syncio.o cluster.o cluster_legacy.o cluster_slot_stats.o crc16.o endianconv.o slowlog.o eval.o bio.o rio.o rand.o memtest.o syscheck.o crcspeed.o crccombine.o crc64.o bitops.o sentinel.o notify.o setproctitle.o blocked.o hyperloglog.o latency.o sparkline.o valkey-check-rdb.o valkey-check-aof.o geo.o lazyfree.o module.o evict.o expire.o geohash.o geohash_helper.o childinfo.o defrag.o siphash.o rax.o t_stream.o listpack.o localtime.o lolwut.o lolwut5.o lolwut6.o acl.o tracking.o socket.o tls.o sha256.o timeout.o setcpuaffinity.o monotonic.o mt19937-64.o resp_parser.o call_reply.o script_lua.o script.o functions.o function_lua.o commands.o strl.o connection.o unix.o logreqres.o ENGINE_CLI_NAME=$(ENGINE_NAME)-cli$(PROG_SUFFIX) ENGINE_CLI_OBJ=anet.o adlist.o dict.o valkey-cli.o zmalloc.o release.o ae.o serverassert.o crcspeed.o crccombine.o crc64.o siphash.o crc16.o monotonic.o cli_common.o mt19937-64.o strl.o cli_commands.o ENGINE_BENCHMARK_NAME=$(ENGINE_NAME)-benchmark$(PROG_SUFFIX) diff --git a/src/ae.c b/src/ae.c index 62031cbeea..28b50c660f 100644 --- a/src/ae.c +++ b/src/ae.c @@ -392,7 +392,7 @@ int aeProcessEvents(aeEventLoop *eventLoop, int flags) { } /* After sleep callback. */ - if (eventLoop->aftersleep != NULL && flags & AE_CALL_AFTER_SLEEP) eventLoop->aftersleep(eventLoop); + if (eventLoop->aftersleep != NULL && flags & AE_CALL_AFTER_SLEEP) eventLoop->aftersleep(eventLoop, numevents); for (j = 0; j < numevents; j++) { int fd = eventLoop->fired[j].fd; @@ -489,6 +489,6 @@ void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep eventLoop->beforesleep = beforesleep; } -void aeSetAfterSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *aftersleep) { +void aeSetAfterSleepProc(aeEventLoop *eventLoop, aeAfterSleepProc *aftersleep) { eventLoop->aftersleep = aftersleep; } diff --git a/src/ae.h b/src/ae.h index a6dcbce50d..3b1c96a01d 100644 --- a/src/ae.h +++ b/src/ae.h @@ -68,6 +68,7 @@ typedef void aeFileProc(struct aeEventLoop *eventLoop, int fd, void *clientData, typedef int aeTimeProc(struct aeEventLoop *eventLoop, long long id, void *clientData); typedef void aeEventFinalizerProc(struct aeEventLoop *eventLoop, void *clientData); typedef void aeBeforeSleepProc(struct aeEventLoop *eventLoop); +typedef void aeAfterSleepProc(struct aeEventLoop *eventLoop, int numevents); /* File event structure */ typedef struct aeFileEvent { @@ -107,7 +108,7 @@ typedef struct aeEventLoop { int stop; void *apidata; /* This is used for polling API specific data */ aeBeforeSleepProc *beforesleep; - aeBeforeSleepProc *aftersleep; + aeAfterSleepProc *aftersleep; int flags; } aeEventLoop; @@ -130,7 +131,7 @@ int aeWait(int fd, int mask, long long milliseconds); void aeMain(aeEventLoop *eventLoop); char *aeGetApiName(void); void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep); -void aeSetAfterSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *aftersleep); +void aeSetAfterSleepProc(aeEventLoop *eventLoop, aeAfterSleepProc *aftersleep); int aeGetSetSize(aeEventLoop *eventLoop); int aeResizeSetSize(aeEventLoop *eventLoop, int setsize); void aeSetDontWait(aeEventLoop *eventLoop, int noWait); diff --git a/src/blocked.c b/src/blocked.c index 15ef39af3b..a1d5306dad 100644 --- a/src/blocked.c +++ b/src/blocked.c @@ -146,7 +146,7 @@ void processUnblockedClients(void) { if (!c->flag.blocked) { /* If we have a queued command, execute it now. */ if (processPendingCommandAndInputBuffer(c) == C_ERR) { - c = NULL; + continue; } } beforeNextClient(c); diff --git a/src/config.c b/src/config.c index f8784413f9..32e6018ff2 100644 --- a/src/config.c +++ b/src/config.c @@ -590,6 +590,9 @@ void loadServerConfigFromString(char *config) { if (server.config_hz < CONFIG_MIN_HZ) server.config_hz = CONFIG_MIN_HZ; if (server.config_hz > CONFIG_MAX_HZ) server.config_hz = CONFIG_MAX_HZ; + /* To ensure backward compatibility when io_threads_num is according to the previous maximum of 128. */ + if (server.io_threads_num > IO_THREADS_MAX_NUM) server.io_threads_num = IO_THREADS_MAX_NUM; + sdsfreesplitres(lines, totlines); reading_config_file = 0; return; @@ -3023,7 +3026,7 @@ standardConfig static_configs[] = { /* Bool configs */ createBoolConfig("rdbchecksum", NULL, IMMUTABLE_CONFIG, server.rdb_checksum, 1, NULL, NULL), createBoolConfig("daemonize", NULL, IMMUTABLE_CONFIG, server.daemonize, 0, NULL, NULL), - createBoolConfig("io-threads-do-reads", NULL, DEBUG_CONFIG | IMMUTABLE_CONFIG, server.io_threads_do_reads, 0, NULL, NULL), /* Read + parse from threads? */ + createBoolConfig("io-threads-do-reads", NULL, DEBUG_CONFIG | IMMUTABLE_CONFIG, server.io_threads_do_reads, 1, NULL, NULL), /* Read + parse from threads */ createBoolConfig("always-show-logo", NULL, IMMUTABLE_CONFIG, server.always_show_logo, 0, NULL, NULL), createBoolConfig("protected-mode", NULL, MODIFIABLE_CONFIG, server.protected_mode, 1, NULL, NULL), createBoolConfig("rdbcompression", NULL, MODIFIABLE_CONFIG, server.rdb_compression, 1, NULL, NULL), @@ -3124,6 +3127,7 @@ standardConfig static_configs[] = { createIntConfig("databases", NULL, IMMUTABLE_CONFIG, 1, INT_MAX, server.dbnum, 16, INTEGER_CONFIG, NULL, NULL), createIntConfig("port", NULL, MODIFIABLE_CONFIG, 0, 65535, server.port, 6379, INTEGER_CONFIG, NULL, updatePort), /* TCP port. */ createIntConfig("io-threads", NULL, DEBUG_CONFIG | IMMUTABLE_CONFIG, 1, 128, server.io_threads_num, 1, INTEGER_CONFIG, NULL, NULL), /* Single threaded by default */ + createIntConfig("events-per-io-thread", NULL, MODIFIABLE_CONFIG, 0, INT_MAX, server.events_per_io_thread, 2, INTEGER_CONFIG, NULL, NULL), createIntConfig("auto-aof-rewrite-percentage", NULL, MODIFIABLE_CONFIG, 0, INT_MAX, server.aof_rewrite_perc, 100, INTEGER_CONFIG, NULL, NULL), createIntConfig("cluster-replica-validity-factor", "cluster-slave-validity-factor", MODIFIABLE_CONFIG, 0, INT_MAX, server.cluster_replica_validity_factor, 10, INTEGER_CONFIG, NULL, NULL), /* replica max data age factor. */ createIntConfig("list-max-listpack-size", "list-max-ziplist-size", MODIFIABLE_CONFIG, INT_MIN, INT_MAX, server.list_max_listpack_size, -2, INTEGER_CONFIG, NULL, NULL), diff --git a/src/config.h b/src/config.h index 95c2e84a00..201e421976 100644 --- a/src/config.h +++ b/src/config.h @@ -264,6 +264,15 @@ void setproctitle(const char *fmt, ...); #error "Undefined or invalid BYTE_ORDER" #endif +/* Cache line alignment */ +#ifndef CACHE_LINE_SIZE +#if defined(__aarch64__) && defined(__APPLE__) +#define CACHE_LINE_SIZE 128 +#else +#define CACHE_LINE_SIZE 64 +#endif /* __aarch64__ && __APPLE__ */ +#endif /* CACHE_LINE_SIZE */ + #if (__i386 || __amd64 || __powerpc__) && __GNUC__ #define GNUC_VERSION (__GNUC__ * 10000 + __GNUC_MINOR__ * 100 + __GNUC_PATCHLEVEL__) #if defined(__clang__) @@ -329,7 +338,7 @@ void setcpuaffinity(const char *cpulist); #define HAVE_FADVISE #endif -#define IO_THREADS_MAX_NUM 128 +#define IO_THREADS_MAX_NUM 16 #ifndef CACHE_LINE_SIZE #if defined(__aarch64__) && defined(__APPLE__) diff --git a/src/connection.h b/src/connection.h index 3de581b417..c6466c2d4c 100644 --- a/src/connection.h +++ b/src/connection.h @@ -112,6 +112,12 @@ typedef struct ConnectionType { int (*has_pending_data)(void); int (*process_pending_data)(void); + /* Postpone update state - with IO threads & TLS we don't want the IO threads to update the event loop events - let + * the main-thread do it */ + void (*postpone_update_state)(struct connection *conn, int); + /* Called by the main-thread */ + void (*update_state)(struct connection *conn); + /* TLS specified methods */ sds (*get_peer_cert)(struct connection *conn); } ConnectionType; @@ -456,4 +462,16 @@ static inline int connIsTLS(connection *conn) { return conn && conn->type == connectionTypeTls(); } +static inline void connUpdateState(connection *conn) { + if (conn->type->update_state) { + conn->type->update_state(conn); + } +} + +static inline void connSetPostponeUpdateState(connection *conn, int on) { + if (conn->type->postpone_update_state) { + conn->type->postpone_update_state(conn, on); + } +} + #endif /* __REDIS_CONNECTION_H */ diff --git a/src/debug.c b/src/debug.c index 36c425a4f4..9501b8a658 100644 --- a/src/debug.c +++ b/src/debug.c @@ -37,6 +37,7 @@ #include "fpconv_dtoa.h" #include "cluster.h" #include "threads_mngr.h" +#include "io_threads.h" #include #include @@ -2159,6 +2160,7 @@ void removeSigSegvHandlers(void) { } void printCrashReport(void) { + server.crashed = 1; /* Log INFO and CLIENT LIST */ logServerInfo(); diff --git a/src/eval.c b/src/eval.c index e4e51f7da5..2afbf445f5 100644 --- a/src/eval.c +++ b/src/eval.c @@ -928,7 +928,7 @@ void ldbEndSession(client *c) { /* If it's a fork()ed session, we just exit. */ if (ldb.forked) { - writeToClient(c, 0); + writeToClient(c); serverLog(LL_NOTICE, "Lua debugging session child exiting"); exitFromChild(0); } else { diff --git a/src/io_threads.c b/src/io_threads.c new file mode 100644 index 0000000000..6149febabc --- /dev/null +++ b/src/io_threads.c @@ -0,0 +1,377 @@ +#include "io_threads.h" + +static __thread int thread_id = 0; /* Thread local var */ +static pthread_t io_threads[IO_THREADS_MAX_NUM] = {0}; +static pthread_mutex_t io_threads_mutex[IO_THREADS_MAX_NUM]; + +/* IO jobs queue functions - Used to send jobs from the main-thread to the IO thread. */ +typedef void (*job_handler)(void *); +typedef struct iojob { + job_handler handler; + void *data; +} iojob; + +typedef struct IOJobQueue { + iojob *ring_buffer; + size_t size; + _Atomic size_t head __attribute__((aligned(CACHE_LINE_SIZE))); /* Next write index for producer (main-thread) */ + _Atomic size_t tail __attribute__((aligned(CACHE_LINE_SIZE))); /* Next read index for consumer (IO-thread) */ +} IOJobQueue; +IOJobQueue io_jobs[IO_THREADS_MAX_NUM] = {0}; + +/* Initialize the job queue with a specified number of items. */ +static void IOJobQueue_init(IOJobQueue *jq, size_t item_count) { + debugServerAssertWithInfo(NULL, NULL, inMainThread()); + jq->ring_buffer = zcalloc(item_count * sizeof(iojob)); + jq->size = item_count; /* Total number of items */ + jq->head = 0; + jq->tail = 0; +} + +/* Clean up the job queue and free allocated memory. */ +static void IOJobQueue_cleanup(IOJobQueue *jq) { + debugServerAssertWithInfo(NULL, NULL, inMainThread()); + zfree(jq->ring_buffer); + memset(jq, 0, sizeof(*jq)); +} + +static int IOJobQueue_isFull(const IOJobQueue *jq) { + debugServerAssertWithInfo(NULL, NULL, inMainThread()); + size_t current_head = atomic_load_explicit(&jq->head, memory_order_relaxed); + /* We don't use memory_order_acquire for the tail due to performance reasons, + * In the worst case we will just assume wrongly the buffer is full and the main thread will do the job by itself. */ + size_t current_tail = atomic_load_explicit(&jq->tail, memory_order_relaxed); + size_t next_head = (current_head + 1) % jq->size; + return next_head == current_tail; +} + +/* Attempt to push a new job to the queue from the main thread. + * the caller must ensure the queue is not full before calling this function. */ +static void IOJobQueue_push(IOJobQueue *jq, job_handler handler, void *data) { + debugServerAssertWithInfo(NULL, NULL, inMainThread()); + /* Assert the queue is not full - should not happen as the caller should check for it before. */ + serverAssert(!IOJobQueue_isFull(jq)); + + /* No need to use atomic acquire for the head, as the main thread is the only one that writes to the head index. */ + size_t current_head = atomic_load_explicit(&jq->head, memory_order_relaxed); + size_t next_head = (current_head + 1) % jq->size; + + /* We store directly the job's fields to avoid allocating a new iojob structure. */ + serverAssert(jq->ring_buffer[current_head].data == NULL); + serverAssert(jq->ring_buffer[current_head].handler == NULL); + jq->ring_buffer[current_head].data = data; + jq->ring_buffer[current_head].handler = handler; + + /* memory_order_release to make sure the data is visible to the consumer (the IO thread). */ + atomic_store_explicit(&jq->head, next_head, memory_order_release); +} + +/* Returns the number of jobs currently available for consumption in the given job queue. + * + * This function ensures memory visibility for the jobs by + * using a memory acquire fence when there are jobs available. */ +static size_t IOJobQueue_availableJobs(const IOJobQueue *jq) { + debugServerAssertWithInfo(NULL, NULL, !inMainThread()); + /* We use memory_order_acquire to make sure the head and the job's fields are visible to the consumer (IO thread). */ + size_t current_head = atomic_load_explicit(&jq->head, memory_order_acquire); + size_t current_tail = atomic_load_explicit(&jq->tail, memory_order_relaxed); + + if (current_head >= current_tail) { + return current_head - current_tail; + } else { + return jq->size - (current_tail - current_head); + } +} + +/* Checks if the job Queue is empty. + * returns 1 if the buffer is currently empty, 0 otherwise. + * Called by the main-thread only. + * This function uses relaxed memory order, so the caller need to use an acquire + * memory fence before calling this function to be sure it has the latest index + * from the other thread, especially when called repeatedly. */ +static int IOJobQueue_isEmpty(const IOJobQueue *jq) { + size_t current_head = atomic_load_explicit(&jq->head, memory_order_relaxed); + size_t current_tail = atomic_load_explicit(&jq->tail, memory_order_relaxed); + return current_head == current_tail; +} + +/* Removes the next job from the given job queue by advancing the tail index. + * Called by the IO thread. + * The caller must ensure that the queue is not empty before calling this function. + * This function uses relaxed memory order, so the caller need to use an release memory fence + * after calling this function to make sure the updated tail is visible to the producer (main thread). */ +static void IOJobQueue_removeJob(IOJobQueue *jq) { + debugServerAssertWithInfo(NULL, NULL, !inMainThread()); + size_t current_tail = atomic_load_explicit(&jq->tail, memory_order_relaxed); + jq->ring_buffer[current_tail].data = NULL; + jq->ring_buffer[current_tail].handler = NULL; + atomic_store_explicit(&jq->tail, (current_tail + 1) % jq->size, memory_order_relaxed); +} + +/* Retrieves the next job handler and data from the job queue without removal. + * Called by the consumer (IO thread). Caller must ensure queue is not empty.*/ +static void IOJobQueue_peek(const IOJobQueue *jq, job_handler *handler, void **data) { + debugServerAssertWithInfo(NULL, NULL, !inMainThread()); + size_t current_tail = atomic_load_explicit(&jq->tail, memory_order_relaxed); + iojob *job = &jq->ring_buffer[current_tail]; + *handler = job->handler; + *data = job->data; +} + +/* End of IO job queue functions */ + +int inMainThread(void) { + return thread_id == 0; +} + +/* Wait until the IO-thread is done with the client */ +void waitForClientIO(client *c) { + /* No need to wait if the client was not offloaded to the IO thread. */ + if (c->io_read_state == CLIENT_IDLE && c->io_write_state == CLIENT_IDLE) return; + + /* Wait for read operation to complete if pending. */ + while (c->io_read_state == CLIENT_PENDING_IO) { + atomic_thread_fence(memory_order_acquire); + } + + /* Wait for write operation to complete if pending. */ + while (c->io_write_state == CLIENT_PENDING_IO) { + atomic_thread_fence(memory_order_acquire); + } + + /* Final memory barrier to ensure all changes are visible */ + atomic_thread_fence(memory_order_acquire); +} + +/** Adjusts the number of active I/O threads based on the current event load. + * If increase_only is non-zero, only allows increasing the number of threads.*/ +void adjustIOThreadsByEventLoad(int numevents, int increase_only) { + if (server.io_threads_num == 1) return; /* All I/O is being done by the main thread. */ + debugServerAssertWithInfo(NULL, NULL, server.io_threads_num > 1); + + int target_threads = + server.events_per_io_thread == 0 ? server.io_threads_num : numevents / server.events_per_io_thread; + + target_threads = max(1, min(target_threads, server.io_threads_num)); + + if (target_threads == server.active_io_threads_num) return; + + if (target_threads < server.active_io_threads_num) { + if (increase_only) return; + + int threads_to_deactivate_num = server.active_io_threads_num - target_threads; + for (int i = 0; i < threads_to_deactivate_num; i++) { + int tid = server.active_io_threads_num - 1; + IOJobQueue *jq = &io_jobs[tid]; + /* We can't lock the thread if it may have pending jobs */ + if (!IOJobQueue_isEmpty(jq)) return; + pthread_mutex_lock(&io_threads_mutex[tid]); + server.active_io_threads_num--; + } + } else { + int threads_to_activate_num = target_threads - server.active_io_threads_num; + for (int i = 0; i < threads_to_activate_num; i++) { + pthread_mutex_unlock(&io_threads_mutex[server.active_io_threads_num]); + server.active_io_threads_num++; + } + } +} + +static void *IOThreadMain(void *myid) { + /* The ID is the thread ID number (from 1 to server.io_threads_num-1). ID 0 is the main thread. */ + long id = (long)myid; + char thdname[32]; + + serverAssert(server.io_threads_num > 0); + serverAssert(id > 0 && id < server.io_threads_num); + snprintf(thdname, sizeof(thdname), "io_thd_%ld", id); + valkey_set_thread_title(thdname); + serverSetCpuAffinity(server.server_cpulist); + makeThreadKillable(); + initSharedQueryBuf(); + + thread_id = (int)id; + size_t jobs_to_process = 0; + IOJobQueue *jq = &io_jobs[id]; + while (1) { + /* Wait for jobs */ + for (int j = 0; j < 1000000; j++) { + jobs_to_process = IOJobQueue_availableJobs(jq); + if (jobs_to_process) break; + } + + /* Give the main thread a chance to stop this thread. */ + if (jobs_to_process == 0) { + pthread_mutex_lock(&io_threads_mutex[id]); + pthread_mutex_unlock(&io_threads_mutex[id]); + continue; + } + + for (size_t j = 0; j < jobs_to_process; j++) { + job_handler handler; + void *data; + /* We keep the job in the queue until it's processed. This ensures that if the main thread checks + * and finds the queue empty, it can be certain that the IO thread is not currently handling any job. */ + IOJobQueue_peek(jq, &handler, &data); + handler(data); + /* Remove the job after it was processed */ + IOJobQueue_removeJob(jq); + } + /* Memory barrier to make sure the main thread sees the updated tail index. + * We do it once per loop and not per tail-update for optimization reasons. + * As the main-thread main concern is to check if the queue is empty, it's enough to do it once at the end. */ + atomic_thread_fence(memory_order_release); + } + freeSharedQueryBuf(); + return NULL; +} + +#define IO_JOB_QUEUE_SIZE 2048 +static void createIOThread(int id) { + pthread_t tid; + pthread_mutex_init(&io_threads_mutex[id], NULL); + IOJobQueue_init(&io_jobs[id], IO_JOB_QUEUE_SIZE); + pthread_mutex_lock(&io_threads_mutex[id]); /* Thread will be stopped. */ + if (pthread_create(&tid, NULL, IOThreadMain, (void *)(long)id) != 0) { + serverLog(LL_WARNING, "Fatal: Can't initialize IO thread, pthread_create failed with: %s", strerror(errno)); + exit(1); + } + io_threads[id] = tid; +} + +/* Terminates the IO thread specified by id. + * Called on server shutdown */ +static void shutdownIOThread(int id) { + int err; + pthread_t tid = io_threads[id]; + if (tid == pthread_self()) return; + if (tid == 0) return; + + pthread_cancel(tid); + + if ((err = pthread_join(tid, NULL)) != 0) { + serverLog(LL_WARNING, "IO thread(tid:%lu) can not be joined: %s", (unsigned long)tid, strerror(err)); + } else { + serverLog(LL_NOTICE, "IO thread(tid:%lu) terminated", (unsigned long)tid); + } + + IOJobQueue_cleanup(&io_jobs[id]); +} + +void killIOThreads(void) { + for (int j = 1; j < server.io_threads_num; j++) { /* We don't kill thread 0, which is the main thread. */ + shutdownIOThread(j); + } +} + +/* Initialize the data structures needed for I/O threads. */ +void initIOThreads(void) { + server.active_io_threads_num = 1; /* We start with threads not active. */ + + /* Don't spawn any thread if the user selected a single thread: + * we'll handle I/O directly from the main thread. */ + if (server.io_threads_num == 1) return; + + serverAssert(server.io_threads_num <= IO_THREADS_MAX_NUM); + + /* Spawn and initialize the I/O threads. */ + for (int i = 1; i < server.io_threads_num; i++) { + createIOThread(i); + } +} + +int trySendReadToIOThreads(client *c) { + if (server.active_io_threads_num <= 1) return C_ERR; + if (!server.io_threads_do_reads) return C_ERR; + /* If IO thread is areadty reading, return C_OK to make sure the main thread will not handle it. */ + if (c->io_read_state != CLIENT_IDLE) return C_OK; + /* Currently, replica/master writes are not offloaded and are processed synchronously. */ + if (c->flag.primary || getClientType(c) == CLIENT_TYPE_REPLICA) return C_ERR; + /* With Lua debug client we may call connWrite directly in the main thread */ + if (c->flag.lua_debug) return C_ERR; + /* For simplicity let the main-thread handle the blocked clients */ + if (c->flag.blocked || c->flag.unblocked) return C_ERR; + if (c->flag.close_asap) return C_ERR; + size_t tid = (c->id % (server.active_io_threads_num - 1)) + 1; + + /* Handle case where client has a pending IO write job on a different thread: + * 1. A write job is still pending (io_write_state == CLIENT_PENDING_IO) + * 2. The pending job is on a different thread (c->cur_tid != tid) + * + * This situation can occur if active_io_threads_num increased since the + * original job assignment. In this case, we keep the job on its current + * thread to ensure the same thread handles the client's I/O operations. */ + if (c->io_write_state == CLIENT_PENDING_IO && c->cur_tid != (uint8_t)tid) tid = c->cur_tid; + + IOJobQueue *jq = &io_jobs[tid]; + if (IOJobQueue_isFull(jq)) return C_ERR; + + c->cur_tid = tid; + c->read_flags = canParseCommand(c) ? 0 : READ_FLAGS_DONT_PARSE; + c->read_flags |= authRequired(c) ? READ_FLAGS_AUTH_REQUIRED : 0; + + c->io_read_state = CLIENT_PENDING_IO; + connSetPostponeUpdateState(c->conn, 1); + IOJobQueue_push(jq, ioThreadReadQueryFromClient, c); + c->flag.pending_read = 1; + listLinkNodeTail(server.clients_pending_io_read, &c->pending_read_list_node); + return C_OK; +} + +/* This function attempts to offload the client's write to an I/O thread. + * Returns C_OK if the client's writes were successfully offloaded to an I/O thread, + * or C_ERR if the client is not eligible for offloading. */ +int trySendWriteToIOThreads(client *c) { + if (server.active_io_threads_num <= 1) return C_ERR; + /* The I/O thread is already writing for this client. */ + if (c->io_write_state != CLIENT_IDLE) return C_OK; + /* Nothing to write */ + if (!clientHasPendingReplies(c)) return C_ERR; + /* Currently, replica/master writes are not offloaded and are processed synchronously. */ + if (c->flag.primary || getClientType(c) == CLIENT_TYPE_REPLICA) return C_ERR; + /* We can't offload debugged clients as the main-thread may read at the same time */ + if (c->flag.lua_debug) return C_ERR; + + size_t tid = (c->id % (server.active_io_threads_num - 1)) + 1; + /* Handle case where client has a pending IO read job on a different thread: + * 1. A read job is still pending (io_read_state == CLIENT_PENDING_IO) + * 2. The pending job is on a different thread (c->cur_tid != tid) + * + * This situation can occur if active_io_threads_num increased since the + * original job assignment. In this case, we keep the job on its current + * thread to ensure the same thread handles the client's I/O operations. */ + if (c->io_read_state == CLIENT_PENDING_IO && c->cur_tid != (uint8_t)tid) tid = c->cur_tid; + + IOJobQueue *jq = &io_jobs[tid]; + if (IOJobQueue_isFull(jq)) return C_ERR; + + c->cur_tid = tid; + if (c->flag.pending_write) { + /* We move the client to the io pending write queue */ + listUnlinkNode(server.clients_pending_write, &c->clients_pending_write_node); + } else { + c->flag.pending_write = 1; + } + serverAssert(c->clients_pending_write_node.prev == NULL && c->clients_pending_write_node.next == NULL); + listLinkNodeTail(server.clients_pending_io_write, &c->clients_pending_write_node); + + /* Save the last block of the reply list to io_last_reply_block and the used + * position to io_last_bufpos. The I/O thread will write only up to + * io_last_bufpos, regardless of the c->bufpos value. This is to prevent I/O + * threads from reading data that might be invalid in their local CPU cache. */ + c->io_last_reply_block = listLast(c->reply); + if (c->io_last_reply_block) { + c->io_last_bufpos = ((clientReplyBlock *)listNodeValue(c->io_last_reply_block))->used; + } else { + c->io_last_bufpos = (size_t)c->bufpos; + } + serverAssert(c->bufpos > 0 || c->io_last_bufpos > 0); + + /* The main-thread will update the client state after the I/O thread completes the write. */ + connSetPostponeUpdateState(c->conn, 1); + c->write_flags = 0; + c->io_write_state = CLIENT_PENDING_IO; + + IOJobQueue_push(jq, ioThreadWriteToClient, c); + return C_OK; +} diff --git a/src/io_threads.h b/src/io_threads.h new file mode 100644 index 0000000000..30d1cdad79 --- /dev/null +++ b/src/io_threads.h @@ -0,0 +1,13 @@ +#ifndef IO_THREADS_H +#define IO_THREADS_H + +#include "server.h" + +void initIOThreads(void); +void killIOThreads(void); +int inMainThread(void); +int trySendReadToIOThreads(client *c); +int trySendWriteToIOThreads(client *c); +void adjustIOThreadsByEventLoad(int numevents, int increase_only); + +#endif /* IO_THREADS_H */ diff --git a/src/networking.c b/src/networking.c index f017e7c034..b249aa61f3 100644 --- a/src/networking.c +++ b/src/networking.c @@ -33,6 +33,7 @@ #include "fpconv_dtoa.h" #include "fmtargs.h" #include +#include "io_threads.h" #include #include #include @@ -46,6 +47,8 @@ char *getClientSockname(client *c); int ProcessingEventsWhileBlocked = 0; /* See processEventsWhileBlocked(). */ __thread sds thread_shared_qb = NULL; +typedef enum { PARSE_OK = 0, PARSE_ERR = -1, PARSE_NEEDMORE = -2 } parseResult; + /* Return the size consumed from the allocator, for the specified SDS string, * including internal fragmentation. This function is used in order to compute * the client output buffer size. */ @@ -158,6 +161,9 @@ client *createClient(connection *conn) { c->argv_len_sum = 0; c->original_argc = 0; c->original_argv = NULL; + c->nread = 0; + c->read_flags = 0; + c->write_flags = 0; c->cmd = c->lastcmd = c->realcmd = NULL; c->cur_script = NULL; c->multibulklen = 0; @@ -199,7 +205,9 @@ client *createClient(connection *conn) { c->sockname = NULL; c->client_list_node = NULL; c->postponed_list_node = NULL; - c->pending_read_list_node = NULL; + c->io_read_state = CLIENT_IDLE; + c->io_write_state = CLIENT_IDLE; + c->nwritten = 0; c->client_tracking_redirection = 0; c->client_tracking_prefixes = NULL; c->last_memory_usage = 0; @@ -210,6 +218,7 @@ client *createClient(connection *conn) { c->auth_callback_privdata = NULL; c->auth_module = NULL; listInitNode(&c->clients_pending_write_node, c); + listInitNode(&c->pending_read_list_node, c); c->mem_usage_bucket = NULL; c->mem_usage_bucket_node = NULL; if (conn) linkClient(c); @@ -300,13 +309,8 @@ int prepareClientToWrite(client *c) { if (!c->conn) return C_ERR; /* Fake client for AOF loading. */ /* Schedule the client to write the output buffers to the socket, unless - * it should already be setup to do so (it has already pending data). - * - * If CLIENT_PENDING_READ is set, we're in an IO thread and should - * not put the client in pending write queue. Instead, it will be - * done by handleClientsWithPendingReadsUsingThreads() upon return. - */ - if (!clientHasPendingReplies(c) && io_threads_op == IO_THREADS_OP_IDLE) putClientInPendingWriteQueue(c); + * it should already be setup to do so (it has already pending data). */ + if (!clientHasPendingReplies(c)) putClientInPendingWriteQueue(c); /* Authorize the caller to queue in the output buffer of this client. */ return C_OK; @@ -745,7 +749,8 @@ void trimReplyUnusedTailSpace(client *c) { * allocation), otherwise there's a high chance realloc will NOP. * Also, to avoid large memmove which happens as part of realloc, we only do * that if the used part is small. */ - if (tail->size - tail->used > tail->size / 4 && tail->used < PROTO_REPLY_CHUNK_BYTES) { + if (tail->size - tail->used > tail->size / 4 && tail->used < PROTO_REPLY_CHUNK_BYTES && + c->io_write_state != CLIENT_PENDING_IO) { size_t usable_size; size_t old_size = tail->size; tail = zrealloc_usable(tail, tail->used + sizeof(clientReplyBlock), &usable_size); @@ -804,8 +809,10 @@ void setDeferredReply(client *c, void *node, const char *s, size_t length) { * - The prev node is non-NULL and has space in it or * - The next node is non-NULL, * - It has enough room already allocated - * - And not too large (avoid large memmove) */ - if (ln->prev != NULL && (prev = listNodeValue(ln->prev)) && prev->size - prev->used > 0) { + * - And not too large (avoid large memmove) + * - And the client is not in a pending I/O state */ + if (ln->prev != NULL && (prev = listNodeValue(ln->prev)) && prev->size - prev->used > 0 && + c->io_write_state != CLIENT_PENDING_IO) { size_t len_to_copy = prev->size - prev->used; if (len_to_copy > length) len_to_copy = length; memcpy(prev->buf + prev->used, s, len_to_copy); @@ -819,7 +826,7 @@ void setDeferredReply(client *c, void *node, const char *s, size_t length) { } if (ln->next != NULL && (next = listNodeValue(ln->next)) && next->size - next->used >= length && - next->used < PROTO_REPLY_CHUNK_BYTES * 4) { + next->used < PROTO_REPLY_CHUNK_BYTES * 4 && c->io_write_state != CLIENT_PENDING_IO) { memmove(next->buf + length, next->buf, next->used); memcpy(next->buf, s, length); next->used += length; @@ -1498,15 +1505,19 @@ void unlinkClient(client *c) { /* Remove from the list of pending writes if needed. */ if (c->flag.pending_write) { serverAssert(&c->clients_pending_write_node.next != NULL || &c->clients_pending_write_node.prev != NULL); - listUnlinkNode(server.clients_pending_write, &c->clients_pending_write_node); + if (c->io_write_state == CLIENT_IDLE) { + listUnlinkNode(server.clients_pending_write, &c->clients_pending_write_node); + } else { + listUnlinkNode(server.clients_pending_io_write, &c->clients_pending_write_node); + } c->flag.pending_write = 0; } /* Remove from the list of pending reads if needed. */ - serverAssert(!c->conn || io_threads_op == IO_THREADS_OP_IDLE); - if (c->pending_read_list_node != NULL) { - listDelNode(server.clients_pending_read, c->pending_read_list_node); - c->pending_read_list_node = NULL; + serverAssert(c->io_read_state != CLIENT_PENDING_IO && c->io_write_state != CLIENT_PENDING_IO); + if (c->flag.pending_read) { + listUnlinkNode(server.clients_pending_io_read, &c->pending_read_list_node); + c->flag.pending_read = 0; } @@ -1585,6 +1596,9 @@ void freeClient(client *c) { return; } + /* Wait for IO operations to be done before proceeding */ + waitForClientIO(c); + /* For connected clients, call the disconnection event of modules hooks. */ if (c->conn) { moduleFireServerEvent(VALKEYMODULE_EVENT_CLIENT_CHANGE, VALKEYMODULE_SUBEVENT_CLIENT_CHANGE_DISCONNECTED, c); @@ -1735,22 +1749,9 @@ void freeClient(client *c) { * a context where calling freeClient() is not possible, because the client * should be valid for the continuation of the flow of the program. */ void freeClientAsync(client *c) { - /* We need to handle concurrent access to the server.clients_to_close list - * only in the freeClientAsync() function, since it's the only function that - * may access the list while the server uses I/O threads. All the other accesses - * are in the context of the main thread while the other threads are - * idle. */ if (c->flag.close_asap || c->flag.script) return; c->flag.close_asap = 1; - if (server.io_threads_num == 1) { - /* no need to bother with locking if there's just one thread (the main thread) */ - listAddNodeTail(server.clients_to_close, c); - return; - } - static pthread_mutex_t async_free_queue_mutex = PTHREAD_MUTEX_INITIALIZER; - pthread_mutex_lock(&async_free_queue_mutex); listAddNodeTail(server.clients_to_close, c); - pthread_mutex_unlock(&async_free_queue_mutex); } /* Log errors for invalid use and free the client in async way. @@ -1769,31 +1770,90 @@ void logInvalidUseAndFreeClientAsync(client *c, const char *fmt, ...) { freeClientAsync(c); } -/* Perform processing of the client before moving on to processing the next client - * this is useful for performing operations that affect the global state but can't - * wait until we're done with all clients. In other words can't wait until beforeSleep() - * return C_ERR in case client is no longer valid after call. - * The input client argument: c, may be NULL in case the previous client was - * freed before the call. */ -int beforeNextClient(client *c) { +/* Resets the shared query buffer used by the given client. + * If any data remained in the buffer, the client will take ownership of the buffer + * and a new empty buffer will be allocated for the shared buffer. */ +void resetSharedQueryBuf(client *c) { + serverAssert(c->querybuf == thread_shared_qb); + size_t remaining = sdslen(c->querybuf) - c->qb_pos; + + if (remaining > 0) { + /* Let the client take ownership of the shared buffer. */ + initSharedQueryBuf(); + return; + } + + c->querybuf = NULL; + sdsclear(thread_shared_qb); + c->qb_pos = 0; +} + +/* Trims the client query buffer to the current position. */ +void trimClientQueryBuffer(client *c) { + if (c->querybuf == thread_shared_qb) { + resetSharedQueryBuf(c); + } + + if (c->querybuf == NULL) { + return; + } + + serverAssert(c->qb_pos <= sdslen(c->querybuf)); + + if (c->qb_pos > 0) { + sdsrange(c->querybuf, c->qb_pos, -1); + c->qb_pos = 0; + } +} + +/* Perform processing of the client before moving on to processing the next client. + * This is useful for performing operations that affect the global state but can't + * wait until we're done with all clients. In other words, it can't wait until beforeSleep(). + * With IO threads enabled, this function offloads the write to the IO threads if possible. */ +void beforeNextClient(client *c) { /* Notice, this code is also called from 'processUnblockedClients'. * But in case of a module blocked client (see RM_Call 'K' flag) we do not reach this code path. * So whenever we change the code here we need to consider if we need this change on module * blocked client as well */ - /* Skip the client processing if we're in an IO thread, in that case we'll perform - this operation later (this function is called again) in the fan-in stage of the threading mechanism */ - if (io_threads_op != IO_THREADS_OP_IDLE) return C_OK; + /* Trim the query buffer to the current position. */ + if (c->flag.primary) { + /* If the client is a primary, trim the querybuf to repl_applied, + * since primary client is very special, its querybuf not only + * used to parse command, but also proxy to sub-replicas. + * + * Here are some scenarios we cannot trim to qb_pos: + * 1. we don't receive complete command from primary + * 2. primary client blocked cause of client pause + * 3. io threads operate read, primary client flagged with CLIENT_PENDING_COMMAND + * + * In these scenarios, qb_pos points to the part of the current command + * or the beginning of next command, and the current command is not applied yet, + * so the repl_applied is not equal to qb_pos. */ + if (c->repl_applied) { + sdsrange(c->querybuf, c->repl_applied, -1); + c->qb_pos -= c->repl_applied; + c->repl_applied = 0; + } + } else { + trimClientQueryBuffer(c); + } /* Handle async frees */ /* Note: this doesn't make the server.clients_to_close list redundant because of * cases where we want an async free of a client other than myself. For example * in ACL modifications we disconnect clients authenticated to non-existent * users (see ACL LOAD). */ - if (c && (c->flag.close_asap)) { + if (c->flag.close_asap) { freeClient(c); - return C_ERR; + return; + } + + updateClientMemUsageAndBucket(c); + /* If IO threads are enabled try to write immediately the reply instead of waiting to beforeSleep, + * unless aof_fsync is set to always in which case we need to wait for beforeSleep after writing the aof buffer. */ + if (server.aof_fsync != AOF_FSYNC_ALWAYS) { + trySendWriteToIOThreads(c); } - return C_OK; } /* Free the clients marked as CLOSE_ASAP, return the number of clients @@ -1827,57 +1887,204 @@ client *lookupClientByID(uint64_t id) { return c; } +void writeToReplica(client *c) { + /* Can be called from main-thread only as replica write offload is not supported yet */ + serverAssert(inMainThread()); + int nwritten = 0; + serverAssert(c->bufpos == 0 && listLength(c->reply) == 0); + while (clientHasPendingReplies(c)) { + replBufBlock *o = listNodeValue(c->ref_repl_buf_node); + serverAssert(o->used >= c->ref_block_pos); + + /* Send current block if it is not fully sent. */ + if (o->used > c->ref_block_pos) { + nwritten = connWrite(c->conn, o->buf + c->ref_block_pos, o->used - c->ref_block_pos); + if (nwritten <= 0) { + c->write_flags |= WRITE_FLAGS_WRITE_ERROR; + return; + } + c->nwritten += nwritten; + c->ref_block_pos += nwritten; + } + + /* If we fully sent the object on head, go to the next one. */ + listNode *next = listNextNode(c->ref_repl_buf_node); + if (next && c->ref_block_pos == o->used) { + o->refcount--; + ((replBufBlock *)(listNodeValue(next)))->refcount++; + c->ref_repl_buf_node = next; + c->ref_block_pos = 0; + incrementalTrimReplicationBacklog(REPL_BACKLOG_TRIM_BLOCKS_PER_CALL); + } + } +} + /* This function should be called from _writeToClient when the reply list is not empty, * it gathers the scattered buffers from reply list and sends them away with connWritev. - * If we write successfully, it returns C_OK, otherwise, C_ERR is returned, - * and 'nwritten' is an output parameter, it means how many bytes server write - * to client. */ -static int _writevToClient(client *c, ssize_t *nwritten) { + * If we write successfully, it returns C_OK, otherwise, C_ERR is returned. + * Sets the c->nwritten to the number of bytes the server wrote to the client. + * Can be called from the main thread or an I/O thread */ +static int writevToClient(client *c) { int iovcnt = 0; int iovmax = min(IOV_MAX, c->conn->iovcnt); - struct iovec iov[iovmax]; - size_t iov_bytes_len = 0; + struct iovec iov_arr[iovmax]; + struct iovec *iov = iov_arr; + ssize_t bufpos, iov_bytes_len = 0; + listNode *lastblock; + + if (inMainThread()) { + lastblock = listLast(c->reply); + bufpos = c->bufpos; + } else { + lastblock = c->io_last_reply_block; + bufpos = lastblock ? (size_t)c->bufpos : c->io_last_bufpos; + } + /* If the static reply buffer is not empty, * add it to the iov array for writev() as well. */ - if (c->bufpos > 0) { + if (bufpos > 0) { iov[iovcnt].iov_base = c->buf + c->sentlen; - iov[iovcnt].iov_len = c->bufpos - c->sentlen; + iov[iovcnt].iov_len = bufpos - c->sentlen; iov_bytes_len += iov[iovcnt++].iov_len; } /* The first node of reply list might be incomplete from the last call, * thus it needs to be calibrated to get the actual data address and length. */ - size_t offset = c->bufpos > 0 ? 0 : c->sentlen; + size_t sentlen = bufpos > 0 ? 0 : c->sentlen; listIter iter; listNode *next; clientReplyBlock *o; + size_t used; listRewind(c->reply, &iter); while ((next = listNext(&iter)) && iovcnt < iovmax && iov_bytes_len < NET_MAX_WRITES_PER_EVENT) { o = listNodeValue(next); - if (o->used == 0) { /* empty node, just release it and skip. */ - c->reply_bytes -= o->size; - listDelNode(c->reply, next); - offset = 0; + + used = o->used; + /* Use c->io_last_bufpos as the currently used portion of the block. + * We use io_last_bufpos instead of o->used to ensure that we only access data guaranteed to be visible to the + * current thread. Using o->used, which may have been updated by the main thread, could lead to accessing data + * that may not yet be visible to the current thread*/ + if (!inMainThread() && next == lastblock) used = c->io_last_bufpos; + + if (used == 0) { /* empty node, skip over it. */ + if (next == lastblock) break; + sentlen = 0; continue; } - iov[iovcnt].iov_base = o->buf + offset; - iov[iovcnt].iov_len = o->used - offset; + iov[iovcnt].iov_base = o->buf + sentlen; + iov[iovcnt].iov_len = used - sentlen; iov_bytes_len += iov[iovcnt++].iov_len; - offset = 0; + + sentlen = 0; + if (next == lastblock) break; } - if (iovcnt == 0) return C_OK; - *nwritten = connWritev(c->conn, iov, iovcnt); - if (*nwritten <= 0) return C_ERR; + + serverAssert(iovcnt != 0); + + ssize_t totwritten = 0; + while (1) { + int nwritten = connWritev(c->conn, iov, iovcnt); + if (nwritten <= 0) { + c->write_flags |= WRITE_FLAGS_WRITE_ERROR; + totwritten = totwritten > 0 ? totwritten : nwritten; + break; + } + totwritten += nwritten; + + if (totwritten == iov_bytes_len) break; + + if (totwritten > NET_MAX_WRITES_PER_EVENT) { + /* Note that we avoid to send more than NET_MAX_WRITES_PER_EVENT + * bytes, Since it's a good idea to serve + * other clients as well, even if a very large request comes from + * super fast link that is always able to accept data (in real world + * scenario think about 'KEYS *' against the loopback interface). + * + * However if we are over the maxmemory limit we ignore that and + * just deliver as much data as it is possible to deliver. */ + int ignore_max_write_limit = server.maxmemory > 0 && zmalloc_used_memory() > server.maxmemory; + if (!ignore_max_write_limit) { + break; + } + } + + /* proceed to the unwritten blocks */ + while (nwritten > 0) { + if ((size_t)nwritten < iov[0].iov_len) { + iov[0].iov_base = (char *)iov[0].iov_base + nwritten; + iov[0].iov_len -= nwritten; + break; + } + nwritten -= iov[0].iov_len; + iov++; + iovcnt--; + } + } + + c->nwritten = totwritten; + return totwritten > 0 ? C_OK : C_ERR; +} + +/* This function does actual writing output buffers to non-replica client, it is called by writeToClient. + * If we write successfully, it returns C_OK, otherwise, C_ERR is returned, + * and 'c->nwritten' is set to the number of bytes the server wrote to the client. */ +int _writeToClient(client *c) { + listNode *lastblock; + size_t bufpos; + + if (inMainThread()) { + /* In the main thread, access bufpos and lastblock directly */ + lastblock = listLast(c->reply); + bufpos = (size_t)c->bufpos; + } else { + /* If there is a last block, use bufpos directly; otherwise, use io_last_bufpos */ + bufpos = c->io_last_reply_block ? (size_t)c->bufpos : c->io_last_bufpos; + lastblock = c->io_last_reply_block; + } + + /* If the reply list is not empty, use writev to save system calls and TCP packets */ + if (lastblock) return writevToClient(c); + + ssize_t bytes_to_write = bufpos - c->sentlen; + ssize_t tot_written = 0; + + while (tot_written < bytes_to_write) { + int nwritten = connWrite(c->conn, c->buf + c->sentlen, bytes_to_write - tot_written); + if (nwritten <= 0) { + c->write_flags |= WRITE_FLAGS_WRITE_ERROR; + tot_written = tot_written > 0 ? tot_written : nwritten; + break; + } + tot_written += nwritten; + } + + c->nwritten = tot_written; + return tot_written > 0 ? C_OK : C_ERR; +} + +static void postWriteToReplica(client *c) { + serverAssert(inMainThread()); + if (c->nwritten > 0) c->net_output_bytes += c->nwritten; +} + +static void _postWriteToClient(client *c) { + if (c->nwritten <= 0) return; + + listIter iter; + listNode *next; + clientReplyBlock *o; + + server.stat_net_output_bytes += c->nwritten; /* Locate the new node which has leftover data and * release all nodes in front of it. */ - ssize_t remaining = *nwritten; - if (c->bufpos > 0) { /* deal with static reply buffer first. */ + ssize_t remaining = c->nwritten; + if (c->bufpos > 0) { /* Deal with static reply buffer first. */ int buf_len = c->bufpos - c->sentlen; - c->sentlen += remaining; + c->sentlen += c->nwritten; /* If the buffer was sent, set bufpos to zero to continue with * the remainder of the reply. */ - if (remaining >= buf_len) { + if (c->nwritten >= buf_len) { c->bufpos = 0; c->sentlen = 0; } @@ -1896,116 +2103,31 @@ static int _writevToClient(client *c, ssize_t *nwritten) { listDelNode(c->reply, next); c->sentlen = 0; } - - return C_OK; -} - -/* This function does actual writing output buffers to different types of - * clients, it is called by writeToClient. - * If we write successfully, it returns C_OK, otherwise, C_ERR is returned, - * and 'nwritten' is an output parameter, it means how many bytes server write - * to client. */ -int _writeToClient(client *c, ssize_t *nwritten) { - *nwritten = 0; - if (getClientType(c) == CLIENT_TYPE_REPLICA) { - serverAssert(c->bufpos == 0 && listLength(c->reply) == 0); - - replBufBlock *o = listNodeValue(c->ref_repl_buf_node); - serverAssert(o->used >= c->ref_block_pos); - /* Send current block if it is not fully sent. */ - if (o->used > c->ref_block_pos) { - *nwritten = connWrite(c->conn, o->buf + c->ref_block_pos, o->used - c->ref_block_pos); - if (*nwritten <= 0) return C_ERR; - c->ref_block_pos += *nwritten; - } - - /* If we fully sent the object on head, go to the next one. */ - listNode *next = listNextNode(c->ref_repl_buf_node); - if (next && c->ref_block_pos == o->used) { - o->refcount--; - ((replBufBlock *)(listNodeValue(next)))->refcount++; - c->ref_repl_buf_node = next; - c->ref_block_pos = 0; - incrementalTrimReplicationBacklog(REPL_BACKLOG_TRIM_BLOCKS_PER_CALL); - } - return C_OK; - } - - /* When the reply list is not empty, it's better to use writev to save us some - * system calls and TCP packets. */ - if (listLength(c->reply) > 0) { - int ret = _writevToClient(c, nwritten); - if (ret != C_OK) return ret; - - /* If there are no longer objects in the list, we expect - * the count of reply bytes to be exactly zero. */ - if (listLength(c->reply) == 0) serverAssert(c->reply_bytes == 0); - } else if (c->bufpos > 0) { - *nwritten = connWrite(c->conn, c->buf + c->sentlen, c->bufpos - c->sentlen); - if (*nwritten <= 0) return C_ERR; - c->sentlen += *nwritten; - - /* If the buffer was sent, set bufpos to zero to continue with - * the remainder of the reply. */ - if ((int)c->sentlen == c->bufpos) { - c->bufpos = 0; - c->sentlen = 0; - } - } - - return C_OK; } -/* Write data in output buffers to client. Return C_OK if the client - * is still valid after the call, C_ERR if it was freed because of some - * error. If handler_installed is set, it will attempt to clear the - * write event. - * - * This function is called by threads, but always with handler_installed - * set to 0. So when handler_installed is set to 0 the function must be - * thread safe. */ -int writeToClient(client *c, int handler_installed) { +/* Updates the client's memory usage and bucket and server stats after writing. + * If a write handler is installed , it will attempt to clear the write event. + * If the client is no longer valid, it will return C_ERR, otherwise C_OK. */ +int postWriteToClient(client *c) { + c->io_last_reply_block = NULL; + c->io_last_bufpos = 0; /* Update total number of writes on server */ - atomic_fetch_add_explicit(&server.stat_total_writes_processed, 1, memory_order_relaxed); - - ssize_t nwritten = 0, totwritten = 0; - - while (clientHasPendingReplies(c)) { - int ret = _writeToClient(c, &nwritten); - if (ret == C_ERR) break; - totwritten += nwritten; - /* Note that we avoid to send more than NET_MAX_WRITES_PER_EVENT - * bytes, in a single threaded server it's a good idea to serve - * other clients as well, even if a very large request comes from - * super fast link that is always able to accept data (in real world - * scenario think about 'KEYS *' against the loopback interface). - * - * However if we are over the maxmemory limit we ignore that and - * just deliver as much data as it is possible to deliver. - * - * Moreover, we also send as much as possible if the client is - * a replica or a monitor (otherwise, on high-speed traffic, the - * replication/output buffer will grow indefinitely) */ - if (totwritten > NET_MAX_WRITES_PER_EVENT && - (server.maxmemory == 0 || zmalloc_used_memory() < server.maxmemory) && !c->flag.replica) - break; - } - + server.stat_total_writes_processed++; if (getClientType(c) == CLIENT_TYPE_REPLICA) { - atomic_fetch_add_explicit(&server.stat_net_repl_output_bytes, totwritten, memory_order_relaxed); + postWriteToReplica(c); } else { - atomic_fetch_add_explicit(&server.stat_net_output_bytes, totwritten, memory_order_relaxed); + _postWriteToClient(c); } - c->net_output_bytes += totwritten; - if (nwritten == -1) { + if (c->write_flags & WRITE_FLAGS_WRITE_ERROR) { if (connGetState(c->conn) != CONN_STATE_CONNECTED) { serverLog(LL_VERBOSE, "Error writing to client: %s", connGetLastError(c->conn)); freeClientAsync(c); return C_ERR; } } - if (totwritten > 0) { + if (c->nwritten > 0) { + c->net_output_bytes += c->nwritten; /* For clients representing primaries we don't count sending data * as an interaction, since we always send REPLCONF ACK commands * that take some time to just fill the socket output buffer. @@ -2014,12 +2136,7 @@ int writeToClient(client *c, int handler_installed) { } if (!clientHasPendingReplies(c)) { c->sentlen = 0; - /* Note that writeToClient() is called in a threaded way, but - * aeDeleteFileEvent() is not thread safe: however writeToClient() - * is always called with handler_installed set to 0 from threads - * so we are fine. */ - if (handler_installed) { - serverAssert(io_threads_op == IO_THREADS_OP_IDLE); + if (connHasWriteHandler(c->conn)) { connSetWriteHandler(c->conn, NULL); } @@ -2029,17 +2146,239 @@ int writeToClient(client *c, int handler_installed) { return C_ERR; } } - /* Update client's memory usage after writing. - * Since this isn't thread safe we do this conditionally. In case of threaded writes this is done in - * handleClientsWithPendingWritesUsingThreads(). */ - if (io_threads_op == IO_THREADS_OP_IDLE) updateClientMemUsageAndBucket(c); + /* Update client's memory usage after writing.*/ + updateClientMemUsageAndBucket(c); return C_OK; } +/* Write data in output buffers to client. Return C_OK if the client + * is still valid after the call, C_ERR if it was freed because of some + * error. + * + * This function is called by main-thread only */ +int writeToClient(client *c) { + if (c->io_write_state != CLIENT_IDLE || c->io_read_state != CLIENT_IDLE) return C_OK; + + c->nwritten = 0; + c->write_flags = 0; + + if (getClientType(c) == CLIENT_TYPE_REPLICA) { + writeToReplica(c); + } else { + _writeToClient(c); + } + + return postWriteToClient(c); +} + /* Write event handler. Just send data to the client. */ void sendReplyToClient(connection *conn) { client *c = connGetPrivateData(conn); - writeToClient(c, 1); + if (trySendWriteToIOThreads(c) == C_OK) return; + writeToClient(c); +} + +void handleQbLimitReached(client *c) { + sds ci = catClientInfoString(sdsempty(), c), bytes = sdsempty(); + bytes = sdscatrepr(bytes, c->querybuf, 64); + serverLog(LL_WARNING, "Closing client that reached max query buffer length: %s (qbuf initial bytes: %s)", ci, + bytes); + sdsfree(ci); + sdsfree(bytes); + freeClientAsync(c); + server.stat_client_qbuf_limit_disconnections++; +} + +/* Handle read errors and update statistics. + * + * Called only from the main thread. + * If the read was done in an I/O thread, this function is invoked after the + * read job has completed, in the main thread context. + * + * Returns: + * - C_OK if the querybuf can be further processed. + * - C_ERR if not. */ +int handleReadResult(client *c) { + serverAssert(inMainThread()); + server.stat_total_reads_processed++; + if (c->nread <= 0) { + if (c->nread == -1) { + if (connGetState(c->conn) != CONN_STATE_CONNECTED) { + serverLog(LL_VERBOSE, "Reading from client: %s", connGetLastError(c->conn)); + freeClientAsync(c); + } + } else if (c->nread == 0) { + if (server.verbosity <= LL_VERBOSE) { + sds info = catClientInfoString(sdsempty(), c); + serverLog(LL_VERBOSE, "Client closed connection %s", info); + sdsfree(info); + } + freeClientAsync(c); + } + return C_ERR; + } + + c->last_interaction = server.unixtime; + c->net_input_bytes += c->nread; + if (c->flag.primary) { + c->read_reploff += c->nread; + server.stat_net_repl_input_bytes += c->nread; + } else { + server.stat_net_input_bytes += c->nread; + } + + /* Handle QB limit */ + if (c->read_flags & READ_FLAGS_QB_LIMIT_REACHED) { + handleQbLimitReached(c); + return C_ERR; + } + return C_OK; +} + + +void handleParseError(client *c) { + int flags = c->read_flags; + if (flags & READ_FLAGS_ERROR_BIG_INLINE_REQUEST) { + addReplyError(c, "Protocol error: too big inline request"); + setProtocolError("too big inline request", c); + } else if (flags & READ_FLAGS_ERROR_BIG_MULTIBULK) { + addReplyError(c, "Protocol error: too big mbulk count string"); + setProtocolError("too big mbulk count string", c); + } else if (flags & READ_FLAGS_ERROR_INVALID_MULTIBULK_LEN) { + addReplyError(c, "Protocol error: invalid multibulk length"); + setProtocolError("invalid mbulk count", c); + } else if (flags & READ_FLAGS_ERROR_UNAUTHENTICATED_MULTIBULK_LEN) { + addReplyError(c, "Protocol error: unauthenticated multibulk length"); + setProtocolError("unauth mbulk count", c); + } else if (flags & READ_FLAGS_ERROR_UNAUTHENTICATED_BULK_LEN) { + addReplyError(c, "Protocol error: unauthenticated bulk length"); + setProtocolError("unauth bulk length", c); + } else if (flags & READ_FLAGS_ERROR_BIG_BULK_COUNT) { + addReplyError(c, "Protocol error: too big bulk count string"); + setProtocolError("too big bulk count string", c); + } else if (flags & READ_FLAGS_ERROR_MBULK_UNEXPECTED_CHARACTER) { + addReplyErrorFormat(c, "Protocol error: expected '$', got '%c'", c->querybuf[c->qb_pos]); + setProtocolError("expected $ but got something else", c); + } else if (flags & READ_FLAGS_ERROR_MBULK_INVALID_BULK_LEN) { + addReplyError(c, "Protocol error: invalid bulk length"); + setProtocolError("invalid bulk length", c); + } else if (flags & READ_FLAGS_ERROR_UNBALANCED_QUOTES) { + addReplyError(c, "Protocol error: unbalanced quotes in request"); + setProtocolError("unbalanced quotes in inline request", c); + } else if (flags & READ_FLAGS_ERROR_UNEXPECTED_INLINE_FROM_PRIMARY) { + serverLog(LL_WARNING, "WARNING: Receiving inline protocol from primary, primary stream corruption? Closing the " + "primary connection and discarding the cached primary."); + setProtocolError("Master using the inline protocol. Desync?", c); + } else { + serverAssertWithInfo(c, NULL, "Unknown parsing error"); + } +} + +int isParsingError(client *c) { + return c->read_flags & (READ_FLAGS_ERROR_BIG_INLINE_REQUEST | READ_FLAGS_ERROR_BIG_MULTIBULK | + READ_FLAGS_ERROR_INVALID_MULTIBULK_LEN | READ_FLAGS_ERROR_UNAUTHENTICATED_MULTIBULK_LEN | + READ_FLAGS_ERROR_UNAUTHENTICATED_BULK_LEN | READ_FLAGS_ERROR_MBULK_INVALID_BULK_LEN | + READ_FLAGS_ERROR_BIG_BULK_COUNT | READ_FLAGS_ERROR_MBULK_UNEXPECTED_CHARACTER | + READ_FLAGS_ERROR_UNEXPECTED_INLINE_FROM_PRIMARY | READ_FLAGS_ERROR_UNBALANCED_QUOTES); +} + +/* This function is called after the query-buffer was parsed. + * It is used to handle parsing errors and to update the client state. + * The function returns C_OK if a command can be executed, otherwise C_ERR. */ +parseResult handleParseResults(client *c) { + if (isParsingError(c)) { + handleParseError(c); + return PARSE_ERR; + } + + if (c->read_flags & READ_FLAGS_INLINE_ZERO_QUERY_LEN && getClientType(c) == CLIENT_TYPE_REPLICA) { + c->repl_ack_time = server.unixtime; + } + + if (c->read_flags & READ_FLAGS_INLINE_ZERO_QUERY_LEN) { + /* in case the client's query was an empty line we will ignore it and proceed to process the rest of the buffer + * if any */ + resetClient(c); + return PARSE_OK; + } + + if (c->read_flags & READ_FLAGS_PARSING_NEGATIVE_MBULK_LEN) { + /* Multibulk processing could see a <= 0 length. */ + resetClient(c); + return PARSE_OK; + } + + if (c->read_flags & READ_FLAGS_PARSING_COMPLETED) { + return PARSE_OK; + } else { + return PARSE_NEEDMORE; + } +} + +/* Process the completion of an IO write operation for a client. + * This function handles various post-write tasks, including updating client state, + * returns 1 if processing completed successfully, 0 if processing is skipped. */ +int processClientIOWriteDone(client *c) { + /* memory barrier acquire to get the latest client state */ + atomic_thread_fence(memory_order_acquire); + /* If a client is protected, don't proceed to check the write results as it may trigger conn close. */ + if (c->flag.protected) return 0; + + listUnlinkNode(server.clients_pending_io_write, &c->clients_pending_write_node); + c->flag.pending_write = 0; + c->io_write_state = CLIENT_IDLE; + + /* Don't post-process-writes to clients that are going to be closed anyway. */ + if (c->flag.close_asap) return 0; + + /* Update processed count on server */ + server.stat_io_writes_processed += 1; + + connSetPostponeUpdateState(c->conn, 0); + connUpdateState(c->conn); + if (postWriteToClient(c) == C_ERR) { + return 1; + } + + if (clientHasPendingReplies(c)) { + if (c->write_flags & WRITE_FLAGS_WRITE_ERROR) { + /* Install the write handler if there are pending writes in some of the clients as a result of not being + * able to write everything in one go. */ + installClientWriteHandler(c); + } else { + /* If we can send the client to the I/O thread, let it handle the write. */ + if (trySendWriteToIOThreads(c) == C_OK) return 1; + /* Try again in the next eventloop */ + putClientInPendingWriteQueue(c); + } + } + + return 1; +} + +/* This function handles the post-processing of I/O write operations that have been + * completed for clients. It iterates through the list of clients with pending I/O + * writes and performs necessary actions based on their current state. + * + * Returns The number of clients processed during this function call. */ +int processIOThreadsWriteDone(void) { + if (listLength(server.clients_pending_io_write) == 0) return 0; + int processed = 0; + listNode *ln; + + listNode *next = listFirst(server.clients_pending_io_write); + while (next) { + ln = next; + next = listNextNode(ln); + client *c = listNodeValue(ln); + + /* Client is still waiting for a pending I/O - skip it */ + if (c->io_write_state == CLIENT_PENDING_IO || c->io_read_state == CLIENT_PENDING_IO) continue; + + processed += processClientIOWriteDone(c); + } + + return processed; } /* This function is called just before entering the event loop, in the hope @@ -2047,10 +2386,16 @@ void sendReplyToClient(connection *conn) { * need to use a syscall in order to install the writable event handler, * get it called, and so forth. */ int handleClientsWithPendingWrites(void) { + int processed = 0; + int pending_writes = listLength(server.clients_pending_write); + if (pending_writes == 0) return processed; /* Return ASAP if there are no clients. */ + + /* Adjust the number of I/O threads based on the number of pending writes this is required in case pending_writes > + * poll_events (for example in pubsub) */ + adjustIOThreadsByEventLoad(pending_writes, 1); + listIter li; listNode *ln; - int processed = listLength(server.clients_pending_write); - listRewind(server.clients_pending_write, &li); while ((ln = listNext(&li))) { client *c = listNodeValue(ln); @@ -2064,8 +2409,18 @@ int handleClientsWithPendingWrites(void) { /* Don't write to clients that are going to be closed anyway. */ if (c->flag.close_asap) continue; + if (!clientHasPendingReplies(c)) continue; + + /* If we can send the client to the I/O thread, let it handle the write. */ + if (trySendWriteToIOThreads(c) == C_OK) continue; + + /* We can't write to the client while IO operation is in progress. */ + if (c->io_write_state != CLIENT_IDLE || c->io_read_state != CLIENT_IDLE) continue; + + processed++; + /* Try to write buffers to the client socket. */ - if (writeToClient(c, 0) == C_ERR) continue; + if (writeToClient(c) == C_ERR) continue; /* If after the synchronous writes above we still have data to * output to the client, we need to install the writable handler. */ @@ -2113,52 +2468,21 @@ void resetClient(client *c) { c->flag.reply_skip = 0; if (c->flag.reply_skip_next) { c->flag.reply_skip = 1; - c->flag.reply_skip_next = 0; - } -} - -/* Initializes the shared query buffer to a new sds with the default capacity */ -void initSharedQueryBuf(void) { - thread_shared_qb = sdsnewlen(NULL, PROTO_IOBUF_LEN); - sdsclear(thread_shared_qb); -} - -/* Resets the shared query buffer used by the given client. - * If any data remained in the buffer, the client will take ownership of the buffer - * and a new empty buffer will be allocated for the shared buffer. */ -void resetSharedQueryBuf(client *c) { - serverAssert(c->querybuf == thread_shared_qb); - size_t remaining = sdslen(c->querybuf) - c->qb_pos; - - if (remaining > 0) { - /* Let the client take ownership of the shared buffer. */ - initSharedQueryBuf(); - return; - } - - c->querybuf = NULL; - sdsclear(thread_shared_qb); - c->qb_pos = 0; -} - -/* Trims the client query buffer to the current position. */ -void trimClientQueryBuffer(client *c) { - if (c->querybuf == thread_shared_qb) { - resetSharedQueryBuf(c); - } - - if (c->querybuf == NULL) { - return; - } - - serverAssert(c->qb_pos <= sdslen(c->querybuf)); - - if (c->qb_pos > 0) { - sdsrange(c->querybuf, c->qb_pos, -1); - c->qb_pos = 0; + c->flag.reply_skip_next = 0; } } +/* Initializes the shared query buffer to a new sds with the default capacity */ +void initSharedQueryBuf(void) { + thread_shared_qb = sdsnewlen(NULL, PROTO_IOBUF_LEN); + sdsclear(thread_shared_qb); +} + +void freeSharedQueryBuf(void) { + sdsfree(thread_shared_qb); + thread_shared_qb = NULL; +} + /* This function is used when we want to re-enter the event loop but there * is the risk that the client we are dealing with will be freed in some * way. This happens for instance in: @@ -2193,16 +2517,14 @@ void unprotectClient(client *c) { /* Like processMultibulkBuffer(), but for the inline protocol instead of RESP, * this function consumes the client query buffer and creates a command ready - * to be executed inside the client structure. Returns C_OK if the command - * is ready to be executed, or C_ERR if there is still protocol to read to - * have a well formed command. The function also returns C_ERR when there is - * a protocol error: in such a case the client structure is setup to reply - * with the error and close the connection. */ -int processInlineBuffer(client *c) { + * to be executed inside the client structure. + * Sets the client read_flags to indicate the parsing outcome. */ +void processInlineBuffer(client *c) { char *newline; int argc, j, linefeed_chars = 1; sds *argv, aux; size_t querylen; + int is_primary = c->read_flags & READ_FLAGS_PRIMARY; /* Search for end of line */ newline = strchr(c->querybuf + c->qb_pos, '\n'); @@ -2210,10 +2532,9 @@ int processInlineBuffer(client *c) { /* Nothing to do without a \r\n */ if (newline == NULL) { if (sdslen(c->querybuf) - c->qb_pos > PROTO_INLINE_MAX_SIZE) { - addReplyError(c, "Protocol error: too big inline request"); - setProtocolError("too big inline request", c); + c->read_flags |= READ_FLAGS_ERROR_BIG_INLINE_REQUEST; } - return C_ERR; + return; } /* Handle the \r\n case. */ @@ -2225,15 +2546,13 @@ int processInlineBuffer(client *c) { argv = sdssplitargs(aux, &argc); sdsfree(aux); if (argv == NULL) { - addReplyError(c, "Protocol error: unbalanced quotes in request"); - setProtocolError("unbalanced quotes in inline request", c); - return C_ERR; + c->read_flags |= READ_FLAGS_ERROR_UNBALANCED_QUOTES; + return; } - /* Newline from replicas can be used to refresh the last ACK time. - * This is useful for a replica to ping back while loading a big - * RDB file. */ - if (querylen == 0 && getClientType(c) == CLIENT_TYPE_REPLICA) c->repl_ack_time = server.unixtime; + if (querylen == 0) { + c->read_flags |= READ_FLAGS_INLINE_ZERO_QUERY_LEN; + } /* Primaries should never send us inline protocol to run actual * commands. If this happens, it is likely due to a bug in the server where @@ -2242,12 +2561,10 @@ int processInlineBuffer(client *c) { * * However there is an exception: primaries may send us just a newline * to keep the connection active. */ - if (querylen != 0 && c->flag.primary) { + if (querylen != 0 && is_primary) { sdsfreesplitres(argv, argc); - serverLog(LL_WARNING, "WARNING: Receiving inline protocol from primary, primary stream corruption? Closing the " - "primary connection and discarding the cached primary."); - setProtocolError("Primary using the inline protocol. Desync?", c); - return C_ERR; + c->read_flags |= READ_FLAGS_ERROR_UNEXPECTED_INLINE_FROM_PRIMARY; + return; } /* Move querybuffer position to the next query in the buffer. */ @@ -2268,7 +2585,7 @@ int processInlineBuffer(client *c) { c->argv_len_sum += sdslen(argv[j]); } zfree(argv); - return C_OK; + c->read_flags |= READ_FLAGS_PARSING_COMPLETED; } /* Helper function. Record protocol error details in server log, @@ -2281,9 +2598,10 @@ static void setProtocolError(const char *errstr, client *c) { /* Sample some protocol to given an idea about what was inside. */ char buf[256]; - if (sdslen(c->querybuf) - c->qb_pos < PROTO_DUMP_LEN) { + buf[0] = '\0'; + if (c->querybuf && sdslen(c->querybuf) - c->qb_pos < PROTO_DUMP_LEN) { snprintf(buf, sizeof(buf), "Query buffer during protocol error: '%s'", c->querybuf + c->qb_pos); - } else { + } else if (c->querybuf) { snprintf(buf, sizeof(buf), "Query buffer during protocol error: '%.*s' (... more %zu bytes ...) '%.*s'", PROTO_DUMP_LEN / 2, c->querybuf + c->qb_pos, sdslen(c->querybuf) - c->qb_pos - PROTO_DUMP_LEN, PROTO_DUMP_LEN / 2, c->querybuf + sdslen(c->querybuf) - PROTO_DUMP_LEN / 2); @@ -2306,20 +2624,18 @@ static void setProtocolError(const char *errstr, client *c) { } /* Process the query buffer for client 'c', setting up the client argument - * vector for command execution. Returns C_OK if after running the function - * the client has a well-formed ready to be processed command, otherwise - * C_ERR if there is still to read more buffer to get the full command. - * The function also returns C_ERR when there is a protocol error: in such a - * case the client structure is setup to reply with the error and close - * the connection. + * vector for command execution. + * Sets the client's read_flags to indicate the parsing outcome. * * This function is called if processInputBuffer() detects that the next * command is in RESP format, so the first byte in the command is found * to be '*'. Otherwise for inline commands processInlineBuffer() is called. */ -int processMultibulkBuffer(client *c) { +void processMultibulkBuffer(client *c) { char *newline = NULL; int ok; long long ll; + int is_primary = c->read_flags & READ_FLAGS_PRIMARY; + int auth_required = c->read_flags & READ_FLAGS_AUTH_REQUIRED; if (c->multibulklen == 0) { /* The client should have been reset */ @@ -2329,32 +2645,32 @@ int processMultibulkBuffer(client *c) { newline = strchr(c->querybuf + c->qb_pos, '\r'); if (newline == NULL) { if (sdslen(c->querybuf) - c->qb_pos > PROTO_INLINE_MAX_SIZE) { - addReplyError(c, "Protocol error: too big mbulk count string"); - setProtocolError("too big mbulk count string", c); + c->read_flags |= READ_FLAGS_ERROR_BIG_MULTIBULK; } - return C_ERR; + return; } /* Buffer should also contain \n */ - if (newline - (c->querybuf + c->qb_pos) > (ssize_t)(sdslen(c->querybuf) - c->qb_pos - 2)) return C_ERR; + if (newline - (c->querybuf + c->qb_pos) > (ssize_t)(sdslen(c->querybuf) - c->qb_pos - 2)) return; /* We know for sure there is a whole line since newline != NULL, * so go ahead and find out the multi bulk length. */ serverAssertWithInfo(c, NULL, c->querybuf[c->qb_pos] == '*'); ok = string2ll(c->querybuf + 1 + c->qb_pos, newline - (c->querybuf + 1 + c->qb_pos), &ll); if (!ok || ll > INT_MAX) { - addReplyError(c, "Protocol error: invalid multibulk length"); - setProtocolError("invalid mbulk count", c); - return C_ERR; - } else if (ll > 10 && authRequired(c)) { - addReplyError(c, "Protocol error: unauthenticated multibulk length"); - setProtocolError("unauth mbulk count", c); - return C_ERR; + c->read_flags |= READ_FLAGS_ERROR_INVALID_MULTIBULK_LEN; + return; + } else if (ll > 10 && auth_required) { + c->read_flags |= READ_FLAGS_ERROR_UNAUTHENTICATED_MULTIBULK_LEN; + return; } c->qb_pos = (newline - c->querybuf) + 2; - if (ll <= 0) return C_OK; + if (ll <= 0) { + c->read_flags |= READ_FLAGS_PARSING_NEGATIVE_MBULK_LEN; + return; + } c->multibulklen = ll; @@ -2372,9 +2688,8 @@ int processMultibulkBuffer(client *c) { newline = strchr(c->querybuf + c->qb_pos, '\r'); if (newline == NULL) { if (sdslen(c->querybuf) - c->qb_pos > PROTO_INLINE_MAX_SIZE) { - addReplyError(c, "Protocol error: too big bulk count string"); - setProtocolError("too big bulk count string", c); - return C_ERR; + c->read_flags |= READ_FLAGS_ERROR_BIG_BULK_COUNT; + return; } break; } @@ -2383,24 +2698,21 @@ int processMultibulkBuffer(client *c) { if (newline - (c->querybuf + c->qb_pos) > (ssize_t)(sdslen(c->querybuf) - c->qb_pos - 2)) break; if (c->querybuf[c->qb_pos] != '$') { - addReplyErrorFormat(c, "Protocol error: expected '$', got '%c'", c->querybuf[c->qb_pos]); - setProtocolError("expected $ but got something else", c); - return C_ERR; + c->read_flags |= READ_FLAGS_ERROR_MBULK_UNEXPECTED_CHARACTER; + return; } ok = string2ll(c->querybuf + c->qb_pos + 1, newline - (c->querybuf + c->qb_pos + 1), &ll); - if (!ok || ll < 0 || (!c->flag.primary && ll > server.proto_max_bulk_len)) { - addReplyError(c, "Protocol error: invalid bulk length"); - setProtocolError("invalid bulk length", c); - return C_ERR; - } else if (ll > 16384 && authRequired(c)) { - addReplyError(c, "Protocol error: unauthenticated bulk length"); - setProtocolError("unauth bulk length", c); - return C_ERR; + if (!ok || ll < 0 || (!(is_primary) && ll > server.proto_max_bulk_len)) { + c->read_flags |= READ_FLAGS_ERROR_MBULK_INVALID_BULK_LEN; + return; + } else if (ll > 16384 && auth_required) { + c->read_flags |= READ_FLAGS_ERROR_UNAUTHENTICATED_BULK_LEN; + return; } c->qb_pos = newline - c->querybuf + 2; - if (!c->flag.primary && ll >= PROTO_MBULK_BIG_ARG) { + if (!(is_primary) && ll >= PROTO_MBULK_BIG_ARG) { /* When the client is not a primary client (because primary * client's querybuf can only be trimmed after data applied * and sent to replicas). @@ -2446,7 +2758,7 @@ int processMultibulkBuffer(client *c) { /* Optimization: if a non-primary client's buffer contains JUST our bulk element * instead of creating a new object by *copying* the sds we * just use the current sds string. */ - if (!c->flag.primary && c->qb_pos == 0 && c->bulklen >= PROTO_MBULK_BIG_ARG && + if (!is_primary && c->qb_pos == 0 && c->bulklen >= PROTO_MBULK_BIG_ARG && sdslen(c->querybuf) == (size_t)(c->bulklen + 2)) { c->argv[c->argc++] = createObject(OBJ_STRING, c->querybuf); c->argv_len_sum += c->bulklen; @@ -2466,10 +2778,7 @@ int processMultibulkBuffer(client *c) { } /* We're done when c->multibulk == 0 */ - if (c->multibulklen == 0) return C_OK; - - /* Still not ready to process the command */ - return C_ERR; + if (c->multibulklen == 0) c->read_flags |= READ_FLAGS_PARSING_COMPLETED; } /* Perform necessary tasks after a command was executed: @@ -2572,122 +2881,103 @@ int processPendingCommandAndInputBuffer(client *c) { return C_OK; } -/* This function is called every time, in the client structure 'c', there is - * more query buffer to process, because we read more data from the socket - * or because a client was blocked and later reactivated, so there could be - * pending query buffer, already representing a full command, to process. - * return C_ERR in case the client was freed during the processing */ +/* Parse a single command from the query buf. + * + * This function may be called from the main thread or from the I/O thread. + * + * Sets the client's read_flags to indicate the parsing outcome */ +void parseCommand(client *c) { + /* Determine request type when unknown. */ + if (!c->reqtype) { + if (c->querybuf[c->qb_pos] == '*') { + c->reqtype = PROTO_REQ_MULTIBULK; + } else { + c->reqtype = PROTO_REQ_INLINE; + } + } + + if (c->reqtype == PROTO_REQ_INLINE) { + processInlineBuffer(c); + } else if (c->reqtype == PROTO_REQ_MULTIBULK) { + processMultibulkBuffer(c); + } else { + serverPanic("Unknown request type"); + } +} + +int canParseCommand(client *c) { + if (c->cmd != NULL) return 0; + + /* Don't parse a command if the client is in the middle of something. */ + if (c->flag.blocked || c->flag.unblocked) return 0; + + /* Don't process more buffers from clients that have already pending + * commands to execute in c->argv. */ + if (c->flag.pending_command) return 0; + + /* Don't process input from the primary while there is a busy script + * condition on the replica. We want just to accumulate the replication + * stream (instead of replying -BUSY like we do with other clients) and + * later resume the processing. */ + if (isInsideYieldingLongCommand() && c->flag.primary) return 0; + + /* CLIENT_CLOSE_AFTER_REPLY closes the connection once the reply is + * written to the client. Make sure to not let the reply grow after + * this flag has been set (i.e. don't process more commands). + * + * The same applies for clients we want to terminate ASAP. */ + if (c->flag.close_after_reply || c->flag.close_asap) return 0; + + return 1; +} + int processInputBuffer(client *c) { - /* Keep processing while there is something in the input buffer */ + /* Parse the query buffer. */ while (c->querybuf && c->qb_pos < sdslen(c->querybuf)) { - /* Immediately abort if the client is in the middle of something. */ - if (c->flag.blocked) break; - - /* Don't process more buffers from clients that have already pending - * commands to execute in c->argv. */ - if (c->flag.pending_command) break; - - /* Don't process input from the primary while there is a busy script - * condition on the replica. We want just to accumulate the replication - * stream (instead of replying -BUSY like we do with other clients) and - * later resume the processing. */ - if (isInsideYieldingLongCommand() && c->flag.primary) break; - - /* CLIENT_CLOSE_AFTER_REPLY closes the connection once the reply is - * written to the client. Make sure to not let the reply grow after - * this flag has been set (i.e. don't process more commands). - * - * The same applies for clients we want to terminate ASAP. */ - if (c->flag.close_after_reply || c->flag.close_asap) break; - - /* Determine request type when unknown. */ - if (!c->reqtype) { - if (c->querybuf[c->qb_pos] == '*') { - c->reqtype = PROTO_REQ_MULTIBULK; - } else { - c->reqtype = PROTO_REQ_INLINE; - } + if (!canParseCommand(c)) { + break; } - if (c->reqtype == PROTO_REQ_INLINE) { - if (processInlineBuffer(c) != C_OK) break; - } else if (c->reqtype == PROTO_REQ_MULTIBULK) { - if (processMultibulkBuffer(c) != C_OK) break; - } else { - serverPanic("Unknown request type"); + c->read_flags = c->flag.primary ? READ_FLAGS_PRIMARY : 0; + c->read_flags |= authRequired(c) ? READ_FLAGS_AUTH_REQUIRED : 0; + + parseCommand(c); + + if (handleParseResults(c) != PARSE_OK) { + break; } - /* Multibulk processing could see a <= 0 length. */ if (c->argc == 0) { - resetClient(c); - } else { - /* If we are in the context of an I/O thread, we can't really - * execute the command here. All we can do is to flag the client - * as one that needs to process the command. */ - if (io_threads_op != IO_THREADS_OP_IDLE) { - serverAssert(io_threads_op == IO_THREADS_OP_READ); - c->flag.pending_command = 1; - break; - } - - if (c->querybuf == thread_shared_qb) { - /* Before processing the command, reset the shared query buffer to its default state. - * This avoids unintentionally modifying the shared qb during processCommand as we may use - * the shared qb for other clients during processEventsWhileBlocked */ - resetSharedQueryBuf(c); - } + /* No command to process - continue parsing the query buf. */ + continue; + } - /* We are finally ready to execute the command. */ - if (processCommandAndResetClient(c) == C_ERR) { - /* If the client is no longer valid, we avoid exiting this - * loop and trimming the client buffer later. So we return - * ASAP in that case. */ - return C_ERR; - } + if (c->querybuf == thread_shared_qb) { + /* Before processing the command, reset the shared query buffer to its default state. + * This avoids unintentionally modifying the shared qb during processCommand as we may use + * the shared qb for other clients during processEventsWhileBlocked */ + resetSharedQueryBuf(c); } - } - if (c->flag.primary) { - /* If the client is a primary, trim the querybuf to repl_applied, - * since primary client is very special, its querybuf not only - * used to parse command, but also proxy to sub-replicas. - * - * Here are some scenarios we cannot trim to qb_pos: - * 1. we don't receive complete command from primary - * 2. primary client blocked cause of client pause - * 3. io threads operate read, primary client flagged with CLIENT_PENDING_COMMAND - * - * In these scenarios, qb_pos points to the part of the current command - * or the beginning of next command, and the current command is not applied yet, - * so the repl_applied is not equal to qb_pos. */ - if (c->repl_applied) { - sdsrange(c->querybuf, c->repl_applied, -1); - c->qb_pos -= c->repl_applied; - c->repl_applied = 0; + /* We are finally ready to execute the command. */ + if (processCommandAndResetClient(c) == C_ERR) { + /* If the client is no longer valid, we avoid exiting this + * loop and trimming the client buffer later. So we return + * ASAP in that case. */ + return C_ERR; } - } else { - trimClientQueryBuffer(c); } - /* Update client memory usage after processing the query buffer, this is - * important in case the query buffer is big and wasn't drained during - * the above loop (because of partially sent big commands). */ - if (io_threads_op == IO_THREADS_OP_IDLE) updateClientMemUsageAndBucket(c); - return C_OK; } -void readQueryFromClient(connection *conn) { - client *c = connGetPrivateData(conn); - int nread, big_arg = 0; +/* This function can be called from the main-thread or from the IO-thread. + * The function allocates query-buf for the client if required and reads to it from the network. + * It will set c->nread to the bytes read from the network. */ +void readToQueryBuf(client *c) { + int big_arg = 0; size_t qblen, readlen; - - /* Check if we want to read from the client later when exiting from - * the event loop. This is the case if threaded I/O is enabled. */ - if (postponeClientRead(c)) return; - - /* Update total number of reads on server */ - atomic_fetch_add_explicit(&server.stat_total_reads_processed, 1, memory_order_relaxed); + int is_primary = c->read_flags & READ_FLAGS_PRIMARY; readlen = PROTO_IOBUF_LEN; qblen = c->querybuf ? sdslen(c->querybuf) : 0; @@ -2717,7 +3007,7 @@ void readQueryFromClient(connection *conn) { qblen = sdslen(c->querybuf); } - if (!c->flag.primary && // primary client's querybuf can grow greedy. + if (!is_primary && // primary client's querybuf can grow greedy. (big_arg || sdsalloc(c->querybuf) < PROTO_IOBUF_LEN)) { /* When reading a BIG_ARG we won't be reading more than that one arg * into the query buffer, so we don't need to pre-allocate more than we @@ -2734,65 +3024,38 @@ void readQueryFromClient(connection *conn) { /* Read as much as possible from the socket to save read(2) system calls. */ readlen = sdsavail(c->querybuf); } - nread = connRead(c->conn, c->querybuf + qblen, readlen); - if (nread == -1) { - if (connGetState(conn) == CONN_STATE_CONNECTED) { - goto done; - } else { - serverLog(LL_VERBOSE, "Reading from client: %s", connGetLastError(c->conn)); - freeClientAsync(c); - goto done; - } - } else if (nread == 0) { - if (server.verbosity <= LL_VERBOSE) { - sds info = catClientInfoString(sdsempty(), c); - serverLog(LL_VERBOSE, "Client closed connection %s", info); - sdsfree(info); - } - freeClientAsync(c); - goto done; + c->nread = connRead(c->conn, c->querybuf + qblen, readlen); + if (c->nread <= 0) { + return; } - sdsIncrLen(c->querybuf, nread); + sdsIncrLen(c->querybuf, c->nread); qblen = sdslen(c->querybuf); if (c->querybuf_peak < qblen) c->querybuf_peak = qblen; - - c->last_interaction = server.unixtime; - if (c->flag.primary) { - c->read_reploff += nread; - atomic_fetch_add_explicit(&server.stat_net_repl_input_bytes, nread, memory_order_relaxed); - } else { - atomic_fetch_add_explicit(&server.stat_net_input_bytes, nread, memory_order_relaxed); - } - c->net_input_bytes += nread; - - if (!c->flag.primary && + if (!is_primary) { /* The commands cached in the MULTI/EXEC queue have not been executed yet, * so they are also considered a part of the query buffer in a broader sense. * * For unauthenticated clients, the query buffer cannot exceed 1MB at most. */ - (c->mstate.argv_len_sums + sdslen(c->querybuf) > server.client_max_querybuf_len || - (c->mstate.argv_len_sums + sdslen(c->querybuf) > 1024 * 1024 && authRequired(c)))) { - sds ci = catClientInfoString(sdsempty(), c), bytes = sdsempty(); - - bytes = sdscatrepr(bytes, c->querybuf, 64); - serverLog(LL_WARNING, "Closing client that reached max query buffer length: %s (qbuf initial bytes: %s)", ci, - bytes); - sdsfree(ci); - sdsfree(bytes); - freeClientAsync(c); - atomic_fetch_add_explicit(&server.stat_client_qbuf_limit_disconnections, 1, memory_order_relaxed); - goto done; + size_t qb_memory = sdslen(c->querybuf) + c->mstate.argv_len_sums; + if (qb_memory > server.client_max_querybuf_len || + (qb_memory > 1024 * 1024 && (c->read_flags & READ_FLAGS_AUTH_REQUIRED))) { + c->read_flags |= READ_FLAGS_QB_LIMIT_REACHED; + } } +} - /* There is more data in the client input buffer, continue parsing it - * and check if there is a full command to execute. */ - if (processInputBuffer(c) == C_ERR) c = NULL; +void readQueryFromClient(connection *conn) { + client *c = connGetPrivateData(conn); + /* Check if we can send the client to be handled by the IO-thread */ + if (postponeClientRead(c)) return; -done: - if (c && c->querybuf == thread_shared_qb) { - sdsclear(thread_shared_qb); - c->querybuf = NULL; + if (c->io_write_state != CLIENT_IDLE || c->io_read_state != CLIENT_IDLE) return; + + readToQueryBuf(c); + + if (handleReadResult(c) == C_OK) { + if (processInputBuffer(c) == C_ERR) return; } beforeNextClient(c); } @@ -2849,6 +3112,7 @@ char *getClientSockname(client *c) { /* Concatenate a string representing the state of a client in a human * readable format, into the sds string 's'. */ sds catClientInfoString(sds s, client *client) { + if (!server.crashed) waitForClientIO(client); char flags[17], events[3], conninfo[CONN_INFO_LEN], *p; p = flags; @@ -4056,7 +4320,7 @@ void flushReplicasOutputBuffers(void) { */ if (replica->repl_state == REPLICA_STATE_ONLINE && !(replica->flag.close_asap) && can_receive_writes && !replica->repl_start_cmd_stream_on_ack && clientHasPendingReplies(replica)) { - writeToClient(replica, 0); + writeToClient(replica); } } } @@ -4218,375 +4482,79 @@ void processEventsWhileBlocked(void) { server.cmd_time_snapshot = prev_cmd_time_snapshot; } -/* ========================================================================== - * Threaded I/O - * ========================================================================== */ - -typedef struct __attribute__((aligned(CACHE_LINE_SIZE))) threads_pending { - _Atomic unsigned long value; -} threads_pending; - -pthread_t io_threads[IO_THREADS_MAX_NUM]; -pthread_mutex_t io_threads_mutex[IO_THREADS_MAX_NUM]; -threads_pending io_threads_pending[IO_THREADS_MAX_NUM]; -int io_threads_op; -/* IO_THREADS_OP_IDLE, IO_THREADS_OP_READ or IO_THREADS_OP_WRITE. */ // TODO: should access to this be atomic??! - -/* This is the list of clients each thread will serve when threaded I/O is - * used. We spawn io_threads_num-1 threads, since one is the main thread - * itself. */ -list *io_threads_list[IO_THREADS_MAX_NUM]; - -static inline unsigned long getIOPendingCount(int i) { - unsigned long count = atomic_load(&io_threads_pending[i].value); - return count; -} - -static inline void setIOPendingCount(int i, unsigned long count) { - atomic_store(&io_threads_pending[i].value, count); -} - -void *IOThreadMain(void *myid) { - /* The ID is the thread number (from 0 to server.io_threads_num-1), and is - * used by the thread to just manipulate a single sub-array of clients. */ - long id = (unsigned long)myid; - char thdname[16]; - - snprintf(thdname, sizeof(thdname), "io_thd_%ld", id); - valkey_set_thread_title(thdname); - serverSetCpuAffinity(server.server_cpulist); - makeThreadKillable(); - initSharedQueryBuf(); - - while (1) { - /* Wait for start */ - for (int j = 0; j < 1000000; j++) { - if (getIOPendingCount(id) != 0) break; - } - - /* Give the main thread a chance to stop this thread. */ - if (getIOPendingCount(id) == 0) { - pthread_mutex_lock(&io_threads_mutex[id]); - pthread_mutex_unlock(&io_threads_mutex[id]); - continue; - } +/* Return 1 if the client read is handled using threaded I/O. + * 0 otherwise. */ +int postponeClientRead(client *c) { + if (ProcessingEventsWhileBlocked) return 0; - serverAssert(getIOPendingCount(id) != 0); - - /* Process: note that the main thread will never touch our list - * before we drop the pending count to 0. */ - listIter li; - listNode *ln; - listRewind(io_threads_list[id], &li); - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - if (io_threads_op == IO_THREADS_OP_WRITE) { - writeToClient(c, 0); - } else if (io_threads_op == IO_THREADS_OP_READ) { - readQueryFromClient(c->conn); - } else { - serverPanic("io_threads_op value is unknown"); - } - } - listEmpty(io_threads_list[id]); - setIOPendingCount(id, 0); - } + return (trySendReadToIOThreads(c) == C_OK); } -/* Initialize the data structures needed for threaded I/O. */ -void initThreadedIO(void) { - server.io_threads_active = 0; /* We start with threads not active. */ - - /* Indicate that io-threads are currently idle */ - io_threads_op = IO_THREADS_OP_IDLE; - - /* Don't spawn any thread if the user selected a single thread: - * we'll handle I/O directly from the main thread. */ - if (server.io_threads_num == 1) return; +int processIOThreadsReadDone(void) { + if (listLength(server.clients_pending_io_read) == 0) return 0; + int processed = 0; + listNode *ln; - if (server.io_threads_num > IO_THREADS_MAX_NUM) { - serverLog(LL_WARNING, - "Fatal: too many I/O threads configured. " - "The maximum number is %d.", - IO_THREADS_MAX_NUM); - exit(1); - } - - /* Spawn and initialize the I/O threads. */ - for (int i = 0; i < server.io_threads_num; i++) { - /* Things we do for all the threads including the main thread. */ - io_threads_list[i] = listCreate(); - if (i == 0) continue; /* Thread 0 is the main thread. */ - - /* Things we do only for the additional threads. */ - pthread_t tid; - pthread_mutex_init(&io_threads_mutex[i], NULL); - setIOPendingCount(i, 0); - pthread_mutex_lock(&io_threads_mutex[i]); /* Thread will be stopped. */ - if (pthread_create(&tid, NULL, IOThreadMain, (void *)(long)i) != 0) { - serverLog(LL_WARNING, "Fatal: Can't initialize IO thread."); - exit(1); - } - io_threads[i] = tid; - } -} + listNode *next = listFirst(server.clients_pending_io_read); + while (next) { + ln = next; + next = listNextNode(ln); + client *c = listNodeValue(ln); -void killIOThreads(void) { - int err, j; - for (j = 0; j < server.io_threads_num; j++) { - if (io_threads[j] == pthread_self()) continue; - if (io_threads[j] && pthread_cancel(io_threads[j]) == 0) { - if ((err = pthread_join(io_threads[j], NULL)) != 0) { - serverLog(LL_WARNING, "IO thread(tid:%lu) can not be joined: %s", (unsigned long)io_threads[j], - strerror(err)); - } else { - serverLog(LL_WARNING, "IO thread(tid:%lu) terminated", (unsigned long)io_threads[j]); - } + /* Client is still waiting for a pending I/O - skip it */ + if (c->io_write_state == CLIENT_PENDING_IO || c->io_read_state == CLIENT_PENDING_IO) continue; + /* If the write job is done, process it ASAP to free the buffer and handle connection errors */ + if (c->io_write_state == CLIENT_COMPLETED_IO) { + processClientIOWriteDone(c); } - } -} - -void startThreadedIO(void) { - serverAssert(server.io_threads_active == 0); - for (int j = 1; j < server.io_threads_num; j++) pthread_mutex_unlock(&io_threads_mutex[j]); - server.io_threads_active = 1; -} - -void stopThreadedIO(void) { - /* We may have still clients with pending reads when this function - * is called: handle them before stopping the threads. */ - handleClientsWithPendingReadsUsingThreads(); - serverAssert(server.io_threads_active == 1); - for (int j = 1; j < server.io_threads_num; j++) pthread_mutex_lock(&io_threads_mutex[j]); - server.io_threads_active = 0; -} - -/* This function checks if there are not enough pending clients to justify - * taking the I/O threads active: in that case I/O threads are stopped if - * currently active. We track the pending writes as a measure of clients - * we need to handle in parallel, however the I/O threading is disabled - * globally for reads as well if we have too little pending clients. - * - * The function returns 0 if the I/O threading should be used because there - * are enough active threads, otherwise 1 is returned and the I/O threads - * could be possibly stopped (if already active) as a side effect. */ -int stopThreadedIOIfNeeded(void) { - int pending = listLength(server.clients_pending_write); - - /* Return ASAP if IO threads are disabled (single threaded mode). */ - if (server.io_threads_num == 1) return 1; - - if (pending < (server.io_threads_num * 2)) { - if (server.io_threads_active) stopThreadedIO(); - return 1; - } else { - return 0; - } -} - -/* This function achieves thread safety using a fan-out -> fan-in paradigm: - * Fan out: The main thread fans out work to the io-threads which block until - * setIOPendingCount() is called with a value larger than 0 by the main thread. - * Fan in: The main thread waits until getIOPendingCount() returns 0. Then - * it can safely perform post-processing and return to normal synchronous - * work. */ -int handleClientsWithPendingWritesUsingThreads(void) { - int processed = listLength(server.clients_pending_write); - if (processed == 0) return 0; /* Return ASAP if there are no clients. */ + /* memory barrier acquire to get the updated client state */ + atomic_thread_fence(memory_order_acquire); + /* Don't post-process-writes to clients that are going to be closed anyway. */ + if (c->flag.close_asap) continue; + /* If a client is protected, don't do anything, + * that may trigger read/write error or recreate handler. */ + if (c->flag.protected) continue; - /* If I/O threads are disabled or we have few clients to serve, don't - * use I/O threads, but the boring synchronous code. */ - if (server.io_threads_num == 1 || stopThreadedIOIfNeeded()) { - return handleClientsWithPendingWrites(); - } + listUnlinkNode(server.clients_pending_io_read, ln); + c->flag.pending_read = 0; + c->io_read_state = CLIENT_IDLE; - /* Start threads if needed. */ - if (!server.io_threads_active) startThreadedIO(); + processed++; + server.stat_io_reads_processed++; - /* Distribute the clients across N different lists. */ - listIter li; - listNode *ln; - listRewind(server.clients_pending_write, &li); - int item_id = 0; - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - c->flag.pending_write = 0; + connSetPostponeUpdateState(c->conn, 0); + connUpdateState(c->conn); - /* Remove clients from the list of pending writes since - * they are going to be closed ASAP. */ - if (c->flag.close_asap) { - listUnlinkNode(server.clients_pending_write, ln); + /* On read error - stop here. */ + if (handleReadResult(c) == C_ERR) { continue; } - /* Since all replicas and replication backlog use global replication - * buffer, to guarantee data accessing thread safe, we must put all - * replicas client into io_threads_list[0] i.e. main thread handles - * sending the output buffer of all replicas. */ - if (getClientType(c) == CLIENT_TYPE_REPLICA) { - listAddNodeTail(io_threads_list[0], c); - continue; + if (!(c->read_flags & READ_FLAGS_DONT_PARSE)) { + parseResult res = handleParseResults(c); + /* On parse error - stop here. */ + if (res == PARSE_ERR) { + continue; + } else if (res == PARSE_NEEDMORE) { + beforeNextClient(c); + continue; + } } - int target_id = item_id % server.io_threads_num; - listAddNodeTail(io_threads_list[target_id], c); - item_id++; - } - - /* Give the start condition to the waiting threads, by setting the - * start condition atomic var. */ - io_threads_op = IO_THREADS_OP_WRITE; - for (int j = 1; j < server.io_threads_num; j++) { - int count = listLength(io_threads_list[j]); - setIOPendingCount(j, count); - } - - /* Also use the main thread to process a slice of clients. */ - listRewind(io_threads_list[0], &li); - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - writeToClient(c, 0); - } - listEmpty(io_threads_list[0]); - - /* Wait for all the other threads to end their work. */ - while (1) { - unsigned long pending = 0; - for (int j = 1; j < server.io_threads_num; j++) pending += getIOPendingCount(j); - if (pending == 0) break; - } - - io_threads_op = IO_THREADS_OP_IDLE; - - /* Run the list of clients again to install the write handler where - * needed. */ - listRewind(server.clients_pending_write, &li); - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - - /* Update the client in the mem usage after we're done processing it in the io-threads */ - updateClientMemUsageAndBucket(c); - - /* Install the write handler if there are pending writes in some - * of the clients. */ - if (clientHasPendingReplies(c)) { - installClientWriteHandler(c); + if (c->argc > 0) { + c->flag.pending_command = 1; } - } - while (listLength(server.clients_pending_write) > 0) { - listUnlinkNode(server.clients_pending_write, server.clients_pending_write->head); - } - - /* Update processed count on server */ - server.stat_io_writes_processed += processed; - - return processed; -} - -/* Return 1 if we want to handle the client read later using threaded I/O. - * This is called by the readable handler of the event loop. - * As a side effect of calling this function the client is put in the - * pending read clients and flagged as such. */ -int postponeClientRead(client *c) { - if (server.io_threads_active && server.io_threads_do_reads && !ProcessingEventsWhileBlocked && - !(c->flag.primary || c->flag.replica || c->flag.blocked) && io_threads_op == IO_THREADS_OP_IDLE) { - listAddNodeHead(server.clients_pending_read, c); - c->pending_read_list_node = listFirst(server.clients_pending_read); - return 1; - } else { - return 0; - } -} - -/* When threaded I/O is also enabled for the reading + parsing side, the - * readable handler will just put normal clients into a queue of clients to - * process (instead of serving them synchronously). This function runs - * the queue using the I/O threads, and process them in order to accumulate - * the reads in the buffers, and also parse the first command available - * rendering it in the client structures. - * This function achieves thread safety using a fan-out -> fan-in paradigm: - * Fan out: The main thread fans out work to the io-threads which block until - * setIOPendingCount() is called with a value larger than 0 by the main thread. - * Fan in: The main thread waits until getIOPendingCount() returns 0. Then - * it can safely perform post-processing and return to normal synchronous - * work. */ -int handleClientsWithPendingReadsUsingThreads(void) { - if (!server.io_threads_active || !server.io_threads_do_reads) return 0; - int processed = listLength(server.clients_pending_read); - if (processed == 0) return 0; - - /* Distribute the clients across N different lists. */ - listIter li; - listNode *ln; - listRewind(server.clients_pending_read, &li); - int item_id = 0; - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - int target_id = item_id % server.io_threads_num; - listAddNodeTail(io_threads_list[target_id], c); - item_id++; - } - - /* Give the start condition to the waiting threads, by setting the - * start condition atomic var. */ - io_threads_op = IO_THREADS_OP_READ; - for (int j = 1; j < server.io_threads_num; j++) { - int count = listLength(io_threads_list[j]); - setIOPendingCount(j, count); - } - - /* Also use the main thread to process a slice of clients. */ - listRewind(io_threads_list[0], &li); - while ((ln = listNext(&li))) { - client *c = listNodeValue(ln); - readQueryFromClient(c->conn); - } - listEmpty(io_threads_list[0]); - - /* Wait for all the other threads to end their work. */ - while (1) { - unsigned long pending = 0; - for (int j = 1; j < server.io_threads_num; j++) pending += getIOPendingCount(j); - if (pending == 0) break; - } - - io_threads_op = IO_THREADS_OP_IDLE; - - /* Run the list of clients again to process the new buffers. */ - while (listLength(server.clients_pending_read)) { - ln = listFirst(server.clients_pending_read); - client *c = listNodeValue(ln); - listDelNode(server.clients_pending_read, ln); - c->pending_read_list_node = NULL; - - serverAssert(!c->flag.blocked); - if (beforeNextClient(c) == C_ERR) { - /* If the client is no longer valid, we avoid - * processing the client later. So we just go - * to the next. */ - continue; + size_t list_length_before_command_execute = listLength(server.clients_pending_io_read); + if (processPendingCommandAndInputBuffer(c) == C_OK) { + beforeNextClient(c); } - - /* Once io-threads are idle we can update the client in the mem usage */ - updateClientMemUsageAndBucket(c); - - if (processPendingCommandAndInputBuffer(c) == C_ERR) { - /* If the client is no longer valid, we avoid - * processing the client later. So we just go - * to the next. */ - continue; + if (list_length_before_command_execute != listLength(server.clients_pending_io_read)) { + /* A client was unlink from the list possibly making the next node invalid */ + next = listFirst(server.clients_pending_io_read); } - - /* We may have pending replies if a thread readQueryFromClient() produced - * replies and did not put the client in pending write queue (it can't). - */ - if (!c->flag.pending_write && clientHasPendingReplies(c)) putClientInPendingWriteQueue(c); } - /* Update processed count on server */ - server.stat_io_reads_processed += processed; - return processed; } @@ -4640,3 +4608,44 @@ void evictClients(void) { } } } + +/* IO threads functions */ + +void ioThreadReadQueryFromClient(void *data) { + client *c = data; + serverAssert(c->io_read_state == CLIENT_PENDING_IO); + + /* Read */ + readToQueryBuf(c); + + /* Check for read errors. */ + if (c->nread <= 0) { + goto done; + } + + /* Skip command parsing if the READ_FLAGS_DONT_PARSE flag is set. */ + if (c->read_flags & READ_FLAGS_DONT_PARSE) { + goto done; + } + + /* Handle QB limit */ + if (c->read_flags & READ_FLAGS_QB_LIMIT_REACHED) { + goto done; + } + + parseCommand(c); + +done: + trimClientQueryBuffer(c); + atomic_thread_fence(memory_order_release); + c->io_read_state = CLIENT_COMPLETED_IO; +} + +void ioThreadWriteToClient(void *data) { + client *c = data; + serverAssert(c->io_write_state == CLIENT_PENDING_IO); + c->nwritten = 0; + _writeToClient(c); + atomic_thread_fence(memory_order_release); + c->io_write_state = CLIENT_COMPLETED_IO; +} diff --git a/src/rdb.c b/src/rdb.c index 8b1037ab93..f9ccd676fd 100644 --- a/src/rdb.c +++ b/src/rdb.c @@ -2931,7 +2931,7 @@ void rdbLoadProgressCallback(rio *r, const void *buf, size_t len) { processModuleLoadingProgressEvent(0); } if (server.repl_state == REPL_STATE_TRANSFER && rioCheckType(r) == RIO_TYPE_CONN) { - atomic_fetch_add_explicit(&server.stat_net_repl_input_bytes, len, memory_order_relaxed); + server.stat_net_repl_input_bytes += len; } } diff --git a/src/replication.c b/src/replication.c index 6779b4f1b4..21ccb0e92d 100644 --- a/src/replication.c +++ b/src/replication.c @@ -765,9 +765,11 @@ int primaryTryPartialResynchronization(client *c, long long psync_offset) { } /* If we reached this point, we are able to perform a partial resync: - * 1) Set client state to make it a replica. - * 2) Inform the client we can continue with +CONTINUE - * 3) Send the backlog data (from the offset to the end) to the replica. */ + * 1) Make sure no IO operations are being performed before changing the client state. + * 2) Set client state to make it a replica. + * 3) Inform the client we can continue with +CONTINUE + * 4) Send the backlog data (from the offset to the end) to the replica. */ + waitForClientIO(c); c->flag.replica = 1; c->repl_state = REPLICA_STATE_ONLINE; c->repl_ack_time = server.unixtime; @@ -1009,6 +1011,8 @@ void syncCommand(client *c) { c->repl_state = REPLICA_STATE_WAIT_BGSAVE_START; if (server.repl_disable_tcp_nodelay) connDisableTcpNoDelay(c->conn); /* Non critical if it fails. */ c->repldbfd = -1; + /* Wait for any IO pending operation to finish before changing the client state */ + waitForClientIO(c); c->flag.replica = 1; listAddNodeTail(server.replicas, c); @@ -1377,7 +1381,7 @@ void sendBulkToReplica(connection *conn) { freeClient(replica); return; } - atomic_fetch_add_explicit(&server.stat_net_repl_output_bytes, nwritten, memory_order_relaxed); + server.stat_net_repl_output_bytes += nwritten; sdsrange(replica->replpreamble, nwritten, -1); if (sdslen(replica->replpreamble) == 0) { sdsfree(replica->replpreamble); @@ -1405,7 +1409,7 @@ void sendBulkToReplica(connection *conn) { return; } replica->repldboff += nwritten; - atomic_fetch_add_explicit(&server.stat_net_repl_output_bytes, nwritten, memory_order_relaxed); + server.stat_net_repl_output_bytes += nwritten; if (replica->repldboff == replica->repldbsize) { closeRepldbfd(replica); connSetWriteHandler(replica->conn, NULL); @@ -1447,7 +1451,7 @@ void rdbPipeWriteHandler(struct connection *conn) { return; } else { replica->repldboff += nwritten; - atomic_fetch_add_explicit(&server.stat_net_repl_output_bytes, nwritten, memory_order_relaxed); + server.stat_net_repl_output_bytes += nwritten; if (replica->repldboff < server.rdb_pipe_bufflen) { replica->repl_last_partial_write = server.unixtime; return; /* more data to write.. */ @@ -1520,7 +1524,7 @@ void rdbPipeReadHandler(struct aeEventLoop *eventLoop, int fd, void *clientData, /* Note: when use diskless replication, 'repldboff' is the offset * of 'rdb_pipe_buff' sent rather than the offset of entire RDB. */ replica->repldboff = nwritten; - atomic_fetch_add_explicit(&server.stat_net_repl_output_bytes, nwritten, memory_order_relaxed); + server.stat_net_repl_output_bytes += nwritten; } /* If we were unable to write all the data to one of the replicas, * setup write handler (and disable pipe read handler, below) */ @@ -1831,7 +1835,7 @@ void readSyncBulkPayload(connection *conn) { } else { /* nread here is returned by connSyncReadLine(), which calls syncReadLine() and * convert "\r\n" to '\0' so 1 byte is lost. */ - atomic_fetch_add_explicit(&server.stat_net_repl_input_bytes, nread + 1, memory_order_relaxed); + server.stat_net_repl_input_bytes += nread + 1; } if (buf[0] == '-') { @@ -1900,7 +1904,7 @@ void readSyncBulkPayload(connection *conn) { cancelReplicationHandshake(1); return; } - atomic_fetch_add_explicit(&server.stat_net_repl_input_bytes, nread, memory_order_relaxed); + server.stat_net_repl_input_bytes += nread; /* When a mark is used, we want to detect EOF asap in order to avoid * writing the EOF mark into the file... */ diff --git a/src/server.c b/src/server.c index 57456c6597..465aa29391 100644 --- a/src/server.c +++ b/src/server.c @@ -39,6 +39,7 @@ #include "syscheck.h" #include "threads_mngr.h" #include "fmtargs.h" +#include "io_threads.h" #include #include @@ -754,6 +755,8 @@ int clientsCronResizeQueryBuffer(client *c) { * The buffer peak will be reset back to the buffer position every server.reply_buffer_peak_reset_time milliseconds * The function always returns 0 as it never terminates the client. */ int clientsCronResizeOutputBuffer(client *c, mstime_t now_ms) { + if (c->io_write_state != CLIENT_IDLE) return 0; + size_t new_buffer_size = 0; char *oldbuf = NULL; const size_t buffer_target_shrink_size = c->buf_usable_size / 2; @@ -904,7 +907,6 @@ void removeClientFromMemUsageBucket(client *c, int allow_eviction) { * returns 1 if client eviction for this client is allowed, 0 otherwise. */ int updateClientMemUsageAndBucket(client *c) { - serverAssert(io_threads_op == IO_THREADS_OP_IDLE && c->conn); int allow_eviction = clientEvictionAllowed(c); removeClientFromMemUsageBucket(c, allow_eviction); @@ -997,6 +999,7 @@ void clientsCron(void) { head = listFirst(server.clients); c = listNodeValue(head); listRotateHeadToTail(server.clients); + if (c->io_read_state != CLIENT_IDLE || c->io_write_state != CLIENT_IDLE) continue; /* The following functions do different service checks on the client. * The protocol is that they return non-zero if the client was * terminated. */ @@ -1075,8 +1078,7 @@ void databasesCron(void) { static inline void updateCachedTimeWithUs(int update_daylight_info, const long long ustime) { server.ustime = ustime; server.mstime = server.ustime / 1000; - time_t unixtime = server.mstime / 1000; - atomic_store_explicit(&server.unixtime, unixtime, memory_order_relaxed); + server.unixtime = server.mstime / 1000; /* To get information about daylight saving time, we need to call * localtime_r and cache the result. However calling localtime_r in this @@ -1257,23 +1259,18 @@ int serverCron(struct aeEventLoop *eventLoop, long long id, void *clientData) { monotime cron_start = getMonotonicUs(); run_with_period(100) { - long long stat_net_input_bytes, stat_net_output_bytes; - long long stat_net_repl_input_bytes, stat_net_repl_output_bytes; - - stat_net_input_bytes = atomic_load_explicit(&server.stat_net_input_bytes, memory_order_relaxed); - stat_net_output_bytes = atomic_load_explicit(&server.stat_net_output_bytes, memory_order_relaxed); - stat_net_repl_input_bytes = atomic_load_explicit(&server.stat_net_repl_input_bytes, memory_order_relaxed); - stat_net_repl_output_bytes = atomic_load_explicit(&server.stat_net_repl_output_bytes, memory_order_relaxed); - monotime current_time = getMonotonicUs(); long long factor = 1000000; // us trackInstantaneousMetric(STATS_METRIC_COMMAND, server.stat_numcommands, current_time, factor); - trackInstantaneousMetric(STATS_METRIC_NET_INPUT, stat_net_input_bytes + stat_net_repl_input_bytes, current_time, - factor); - trackInstantaneousMetric(STATS_METRIC_NET_OUTPUT, stat_net_output_bytes + stat_net_repl_output_bytes, + trackInstantaneousMetric(STATS_METRIC_NET_INPUT, server.stat_net_input_bytes + server.stat_net_repl_input_bytes, current_time, factor); - trackInstantaneousMetric(STATS_METRIC_NET_INPUT_REPLICATION, stat_net_repl_input_bytes, current_time, factor); - trackInstantaneousMetric(STATS_METRIC_NET_OUTPUT_REPLICATION, stat_net_repl_output_bytes, current_time, factor); + trackInstantaneousMetric(STATS_METRIC_NET_OUTPUT, + server.stat_net_output_bytes + server.stat_net_repl_output_bytes, current_time, + factor); + trackInstantaneousMetric(STATS_METRIC_NET_INPUT_REPLICATION, server.stat_net_repl_input_bytes, current_time, + factor); + trackInstantaneousMetric(STATS_METRIC_NET_OUTPUT_REPLICATION, server.stat_net_repl_output_bytes, current_time, + factor); trackInstantaneousMetric(STATS_METRIC_EL_CYCLE, server.duration_stats[EL_DURATION_TYPE_EL].cnt, current_time, factor); trackInstantaneousMetric(STATS_METRIC_EL_DURATION, server.duration_stats[EL_DURATION_TYPE_EL].sum, @@ -1433,9 +1430,6 @@ int serverCron(struct aeEventLoop *eventLoop, long long id, void *clientData) { migrateCloseTimedoutSockets(); } - /* Stop the I/O threads if we don't have enough pending work. */ - stopThreadedIOIfNeeded(); - /* Resize tracking keys table if needed. This is also done at every * command execution, but we want to be sure that if the last command * executed changes the value via CONFIG SET, the server will perform @@ -1580,23 +1574,31 @@ void beforeSleep(struct aeEventLoop *eventLoop) { * events to handle. */ if (ProcessingEventsWhileBlocked) { uint64_t processed = 0; - processed += handleClientsWithPendingReadsUsingThreads(); + processed += processIOThreadsReadDone(); processed += connTypeProcessPendingData(); if (server.aof_state == AOF_ON || server.aof_state == AOF_WAIT_REWRITE) flushAppendOnlyFile(0); processed += handleClientsWithPendingWrites(); + int last_procssed = 0; + do { + /* Try to process all the pending IO events. */ + last_procssed = processIOThreadsReadDone() + processIOThreadsWriteDone(); + processed += last_procssed; + } while (last_procssed != 0); processed += freeClientsInAsyncFreeQueue(); server.events_processed_while_blocked += processed; return; } /* We should handle pending reads clients ASAP after event loop. */ - handleClientsWithPendingReadsUsingThreads(); + processIOThreadsReadDone(); /* Handle pending data(typical TLS). (must be done before flushAppendOnlyFile) */ connTypeProcessPendingData(); - /* If any connection type(typical TLS) still has pending unread data don't sleep at all. */ - int dont_sleep = connTypeHasPendingData(); + /* If any connection type(typical TLS) still has pending unread data or if there are clients + * with pending IO reads/writes, don't sleep at all. */ + int dont_sleep = connTypeHasPendingData() || listLength(server.clients_pending_io_read) > 0 || + listLength(server.clients_pending_io_write) > 0; /* Call the Cluster before sleep function. Note that this function * may change the state of Cluster (from ok to fail or vice versa), @@ -1659,7 +1661,7 @@ void beforeSleep(struct aeEventLoop *eventLoop) { long long prev_fsynced_reploff = server.fsynced_reploff; /* Write the AOF buffer on disk, - * must be done before handleClientsWithPendingWritesUsingThreads, + * must be done before handleClientsWithPendingWrites, * in case of appendfsync=always. */ if (server.aof_state == AOF_ON || server.aof_state == AOF_WAIT_REWRITE) flushAppendOnlyFile(0); @@ -1679,7 +1681,14 @@ void beforeSleep(struct aeEventLoop *eventLoop) { } /* Handle writes with pending output buffers. */ - handleClientsWithPendingWritesUsingThreads(); + handleClientsWithPendingWrites(); + + /* Try to process more IO reads that are ready to be processed. */ + if (server.aof_fsync != AOF_FSYNC_ALWAYS) { + processIOThreadsReadDone(); + } + + processIOThreadsWriteDone(); /* Record cron time in beforeSleep. This does not include the time consumed by AOF writing and IO writing above. */ monotime cron_start_time_after_write = getMonotonicUs(); @@ -1729,7 +1738,7 @@ void beforeSleep(struct aeEventLoop *eventLoop) { /* This function is called immediately after the event loop multiplexing * API returned, and the control is going to soon return to the server by invoking * the different events callbacks. */ -void afterSleep(struct aeEventLoop *eventLoop) { +void afterSleep(struct aeEventLoop *eventLoop, int numevents) { UNUSED(eventLoop); /********************* WARNING ******************** * Do NOT add anything above moduleAcquireGIL !!! * @@ -1761,6 +1770,8 @@ void afterSleep(struct aeEventLoop *eventLoop) { if (!ProcessingEventsWhileBlocked) { server.cmd_time_snapshot = server.mstime; } + + adjustIOThreadsByEventLoad(numevents, 0); } /* =========================== Server initialization ======================== */ @@ -2478,10 +2489,10 @@ void resetServerStats(void) { server.stat_sync_partial_ok = 0; server.stat_sync_partial_err = 0; server.stat_io_reads_processed = 0; - atomic_store_explicit(&server.stat_total_reads_processed, 0, memory_order_relaxed); + server.stat_total_reads_processed = 0; server.stat_io_writes_processed = 0; - atomic_store_explicit(&server.stat_total_writes_processed, 0, memory_order_relaxed); - atomic_store_explicit(&server.stat_client_qbuf_limit_disconnections, 0, memory_order_relaxed); + server.stat_total_writes_processed = 0; + server.stat_client_qbuf_limit_disconnections = 0; server.stat_client_outbuf_limit_disconnections = 0; for (j = 0; j < STATS_METRIC_COUNT; j++) { server.inst_metric[j].idx = 0; @@ -2492,10 +2503,10 @@ void resetServerStats(void) { server.stat_aof_rewrites = 0; server.stat_rdb_saves = 0; server.stat_aofrw_consecutive_failures = 0; - atomic_store_explicit(&server.stat_net_input_bytes, 0, memory_order_relaxed); - atomic_store_explicit(&server.stat_net_output_bytes, 0, memory_order_relaxed); - atomic_store_explicit(&server.stat_net_repl_input_bytes, 0, memory_order_relaxed); - atomic_store_explicit(&server.stat_net_repl_output_bytes, 0, memory_order_relaxed); + server.stat_net_input_bytes = 0; + server.stat_net_output_bytes = 0; + server.stat_net_repl_input_bytes = 0; + server.stat_net_repl_output_bytes = 0; server.stat_unexpected_error_replies = 0; server.stat_total_error_replies = 0; server.stat_dump_payload_sanitizations = 0; @@ -2545,7 +2556,8 @@ void initServer(void) { server.replicas = listCreate(); server.monitors = listCreate(); server.clients_pending_write = listCreate(); - server.clients_pending_read = listCreate(); + server.clients_pending_io_write = listCreate(); + server.clients_pending_io_read = listCreate(); server.clients_timeout_table = raxNew(); server.replication_allowed = 1; server.replicas_eldb = -1; /* Force to emit the first SELECT command. */ @@ -2641,6 +2653,7 @@ void initServer(void) { server.rdb_last_load_keys_expired = 0; server.rdb_last_load_keys_loaded = 0; server.dirty = 0; + server.crashed = 0; resetServerStats(); /* A few stats we don't want to reset: server startup time, and peak mem. */ server.stat_starttime = time(NULL); @@ -2796,7 +2809,7 @@ void initListeners(void) { * see: https://sourceware.org/bugzilla/show_bug.cgi?id=19329 */ void InitServerLast(void) { bioInit(); - initThreadedIO(); + initIOThreads(); set_jemalloc_bg_thread(server.jemalloc_bg_thread); server.initial_memory_usage = zmalloc_used_memory(); } @@ -5395,7 +5408,7 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { "lru_clock:%u\r\n", server.lruclock, "executable:%s\r\n", server.executable ? server.executable : "", "config_file:%s\r\n", server.configfile ? server.configfile : "", - "io_threads_active:%i\r\n", server.io_threads_active, + "io_threads_active:%i\r\n", server.active_io_threads_num > 1, "availability_zone:%s\r\n", server.availability_zone)); /* clang-format on */ @@ -5630,23 +5643,10 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { /* Stats */ if (all_sections || (dictFind(section_dict, "stats") != NULL)) { - long long stat_total_reads_processed, stat_total_writes_processed; - long long stat_net_input_bytes, stat_net_output_bytes; - long long stat_net_repl_input_bytes, stat_net_repl_output_bytes; long long current_eviction_exceeded_time = server.stat_last_eviction_exceeded_time ? (long long)elapsedUs(server.stat_last_eviction_exceeded_time) : 0; long long current_active_defrag_time = server.stat_last_active_defrag_time ? (long long)elapsedUs(server.stat_last_active_defrag_time) : 0; - long long stat_client_qbuf_limit_disconnections; - - stat_total_reads_processed = atomic_load_explicit(&server.stat_total_reads_processed, memory_order_relaxed); - stat_total_writes_processed = atomic_load_explicit(&server.stat_total_writes_processed, memory_order_relaxed); - stat_net_input_bytes = atomic_load_explicit(&server.stat_net_input_bytes, memory_order_relaxed); - stat_net_output_bytes = atomic_load_explicit(&server.stat_net_output_bytes, memory_order_relaxed); - stat_net_repl_input_bytes = atomic_load_explicit(&server.stat_net_repl_input_bytes, memory_order_relaxed); - stat_net_repl_output_bytes = atomic_load_explicit(&server.stat_net_repl_output_bytes, memory_order_relaxed); - stat_client_qbuf_limit_disconnections = - atomic_load_explicit(&server.stat_client_qbuf_limit_disconnections, memory_order_relaxed); if (sections++) info = sdscat(info, "\r\n"); /* clang-format off */ @@ -5654,10 +5654,10 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { "total_connections_received:%lld\r\n", server.stat_numconnections, "total_commands_processed:%lld\r\n", server.stat_numcommands, "instantaneous_ops_per_sec:%lld\r\n", getInstantaneousMetric(STATS_METRIC_COMMAND), - "total_net_input_bytes:%lld\r\n", stat_net_input_bytes + stat_net_repl_input_bytes, - "total_net_output_bytes:%lld\r\n", stat_net_output_bytes + stat_net_repl_output_bytes, - "total_net_repl_input_bytes:%lld\r\n", stat_net_repl_input_bytes, - "total_net_repl_output_bytes:%lld\r\n", stat_net_repl_output_bytes, + "total_net_input_bytes:%lld\r\n", server.stat_net_input_bytes + server.stat_net_repl_input_bytes, + "total_net_output_bytes:%lld\r\n", server.stat_net_output_bytes + server.stat_net_repl_output_bytes, + "total_net_repl_input_bytes:%lld\r\n", server.stat_net_repl_input_bytes, + "total_net_repl_output_bytes:%lld\r\n", server.stat_net_repl_output_bytes, "instantaneous_input_kbps:%.2f\r\n", (float)getInstantaneousMetric(STATS_METRIC_NET_INPUT)/1024, "instantaneous_output_kbps:%.2f\r\n", (float)getInstantaneousMetric(STATS_METRIC_NET_OUTPUT)/1024, "instantaneous_input_repl_kbps:%.2f\r\n", (float)getInstantaneousMetric(STATS_METRIC_NET_INPUT_REPLICATION)/1024, @@ -5696,11 +5696,11 @@ sds genValkeyInfoString(dict *section_dict, int all_sections, int everything) { "unexpected_error_replies:%lld\r\n", server.stat_unexpected_error_replies, "total_error_replies:%lld\r\n", server.stat_total_error_replies, "dump_payload_sanitizations:%lld\r\n", server.stat_dump_payload_sanitizations, - "total_reads_processed:%lld\r\n", stat_total_reads_processed, - "total_writes_processed:%lld\r\n", stat_total_writes_processed, + "total_reads_processed:%lld\r\n", server.stat_total_reads_processed, + "total_writes_processed:%lld\r\n", server.stat_total_writes_processed, "io_threaded_reads_processed:%lld\r\n", server.stat_io_reads_processed, "io_threaded_writes_processed:%lld\r\n", server.stat_io_writes_processed, - "client_query_buffer_limit_disconnections:%lld\r\n", stat_client_qbuf_limit_disconnections, + "client_query_buffer_limit_disconnections:%lld\r\n", server.stat_client_qbuf_limit_disconnections, "client_output_buffer_limit_disconnections:%lld\r\n", server.stat_client_outbuf_limit_disconnections, "reply_buffer_shrinks:%lld\r\n", server.stat_reply_buffer_shrinks, "reply_buffer_expands:%lld\r\n", server.stat_reply_buffer_expands, diff --git a/src/server.h b/src/server.h index 73f68a73d4..36a4b641e7 100644 --- a/src/server.h +++ b/src/server.h @@ -1117,6 +1117,12 @@ typedef struct { } clientReqResInfo; #endif +typedef enum { + CLIENT_IDLE = 0, /* Initial state: client is idle. */ + CLIENT_PENDING_IO = 1, /* Main-thread sets this state when client is sent to IO-thread for read/write. */ + CLIENT_COMPLETED_IO = 2 /* IO-thread sets this state after completing IO operation. */ +} clientIOState; + typedef struct ClientFlags { uint64_t primary : 1; /* This client is a primary */ uint64_t replica : 1; /* This client is a replica */ @@ -1141,6 +1147,7 @@ typedef struct ClientFlags { uint64_t prevent_repl_prop : 1; /* Don't propagate to replicas. */ uint64_t prevent_prop : 1; /* Don't propagate to AOF or replicas. */ uint64_t pending_write : 1; /* Client has output to send but a write handler is yet not installed. */ + uint64_t pending_read : 1; /* Client has output to send but a write handler is yet not installed. */ uint64_t reply_off : 1; /* Don't send replies to client. */ uint64_t reply_skip_next : 1; /* Set CLIENT_REPLY_SKIP for next cmd */ uint64_t reply_skip : 1; /* Don't send just this reply. */ @@ -1173,7 +1180,7 @@ typedef struct ClientFlags { uint64_t reprocessing_command : 1; /* The client is re-processing the command. */ uint64_t replication_done : 1; /* Indicate that replication has been done on the client */ uint64_t authenticated : 1; /* Indicate a client has successfully authenticated */ - uint64_t reserved : 10; /* Reserved for future use */ + uint64_t reserved : 9; /* Reserved for future use */ } ClientFlags; typedef struct client { @@ -1198,6 +1205,13 @@ typedef struct client { int original_argc; /* Num of arguments of original command if arguments were rewritten. */ robj **original_argv; /* Arguments of original command if arguments were rewritten. */ size_t argv_len_sum; /* Sum of lengths of objects in argv list. */ + volatile uint8_t io_read_state; /* Indicate the IO read state of the client */ + volatile uint8_t io_write_state; /* Indicate the IO write state of the client */ + uint8_t cur_tid; /* ID of IO thread currently performing IO for this client */ + int nread; /* Number of bytes of the last read. */ + int nwritten; /* Number of bytes of the last write. */ + int read_flags; /* Client Read flags - used to communicate the client read state. */ + uint16_t write_flags; /* Client Write flags - used to communicate the client write state. */ struct serverCommand *cmd, *lastcmd; /* Last command executed. */ struct serverCommand *realcmd; /* The original command that was executed by the client, Used to update error stats in case the c->cmd was modified @@ -1209,6 +1223,7 @@ typedef struct client { int multibulklen; /* Number of multi bulk arguments left to read. */ long bulklen; /* Length of bulk argument in multi bulk request. */ list *reply; /* List of reply objects to send to the client. */ + listNode *io_last_reply_block; /* Last client reply block when sent to IO thread */ unsigned long long reply_bytes; /* Tot bytes of objects in reply list. */ list *deferred_reply_errors; /* Used for module thread safe contexts. */ size_t sentlen; /* Amount of bytes already sent in the current @@ -1253,7 +1268,6 @@ typedef struct client { sds sockname; /* Cached connection target address. */ listNode *client_list_node; /* list node in client list */ listNode *postponed_list_node; /* list node within the postponed list */ - listNode *pending_read_list_node; /* list node in clients pending read list */ void *module_blocked_client; /* Pointer to the ValkeyModuleBlockedClient associated with this * client. This is set in case of module authentication before the * unblocked client is reprocessed to handle reply callbacks. */ @@ -1293,12 +1307,14 @@ typedef struct client { size_t ref_block_pos; /* Access position of referenced buffer block, * i.e. the next offset to send. */ - /* list node in clients_pending_write list */ + /* list node in clients_pending_write or in clients_pending_io_write list */ listNode clients_pending_write_node; + listNode pending_read_list_node; /* list node in clients_pending_io_read list */ /* Response buffer */ size_t buf_peak; /* Peak used size of buffer in last 5 sec interval. */ mstime_t buf_peak_last_reset_time; /* keeps the last time the buffer peak value was reset */ int bufpos; + size_t io_last_bufpos; /* The client's bufpos at the time it was sent to the IO thread */ size_t buf_usable_size; /* Usable size of buffer. */ char *buf; #ifdef LOG_REQ_RES @@ -1629,7 +1645,8 @@ struct valkeyServer { list *clients; /* List of active clients */ list *clients_to_close; /* Clients to close asynchronously */ list *clients_pending_write; /* There is to write or install handler. */ - list *clients_pending_read; /* Client has pending read socket buffers. */ + list *clients_pending_io_read; /* List of clients with pending read to be process by I/O threads. */ + list *clients_pending_io_write; /* List of clients with pending write to be process by I/O threads. */ list *replicas, *monitors; /* List of replicas and MONITORs */ client *current_client; /* The client that triggered the command execution (External or AOF). */ client *executing_client; /* The client executing the current command (possibly script or module). */ @@ -1657,7 +1674,8 @@ struct valkeyServer { int protected_mode; /* Don't accept external connections. */ int io_threads_num; /* Number of IO threads to use. */ int io_threads_do_reads; /* Read and parse from IO threads? */ - int io_threads_active; /* Is IO threads currently active? */ + int active_io_threads_num; /* Current number of active IO threads, includes main thread. */ + int events_per_io_thread; /* Number of events on the event loop to trigger IO threads activation. */ long long events_processed_while_blocked; /* processEventsWhileBlocked() */ int enable_protected_configs; /* Enable the modification of protected configs, see PROTECTED_ACTION_ALLOWED_* */ int enable_debug_cmd; /* Enable DEBUG commands, see PROTECTED_ACTION_ALLOWED_* */ @@ -1710,15 +1728,14 @@ struct valkeyServer { long long slowlog_log_slower_than; /* SLOWLOG time limit (to get logged) */ unsigned long slowlog_max_len; /* SLOWLOG max number of items logged */ struct malloc_stats cron_malloc_stats; /* sampled in serverCron(). */ - _Atomic long long stat_net_input_bytes; /* Bytes read from network. */ - _Atomic long long stat_net_output_bytes; /* Bytes written to network. */ - _Atomic long long - stat_net_repl_input_bytes; /* Bytes read during replication, added to stat_net_input_bytes in 'info'. */ - _Atomic long long - stat_net_repl_output_bytes; /* Bytes written during replication, added to stat_net_output_bytes in 'info'. */ - size_t stat_current_cow_peak; /* Peak size of copy on write bytes. */ - size_t stat_current_cow_bytes; /* Copy on write bytes while child is active. */ - monotime stat_current_cow_updated; /* Last update time of stat_current_cow_bytes */ + long long stat_net_input_bytes; /* Bytes read from network. */ + long long stat_net_output_bytes; /* Bytes written to network. */ + long long stat_net_repl_input_bytes; /* Bytes read during replication, added to stat_net_input_bytes in 'info'. */ + /* Bytes written during replication, added to stat_net_output_bytes in 'info'. */ + long long stat_net_repl_output_bytes; + size_t stat_current_cow_peak; /* Peak size of copy on write bytes. */ + size_t stat_current_cow_bytes; /* Copy on write bytes while child is active. */ + monotime stat_current_cow_updated; /* Last update time of stat_current_cow_bytes */ size_t stat_current_save_keys_processed; /* Processed keys while child is active. */ size_t stat_current_save_keys_total; /* Number of keys when child started. */ size_t stat_rdb_cow_bytes; /* Copy on write bytes during RDB saving. */ @@ -1730,12 +1747,12 @@ struct valkeyServer { long long stat_unexpected_error_replies; /* Number of unexpected (aof-loading, replica to primary, etc.) error replies */ long long stat_total_error_replies; /* Total number of issued error replies ( command + rejected errors ) */ - long long stat_dump_payload_sanitizations; /* Number deep dump payloads integrity validations. */ - long long stat_io_reads_processed; /* Number of read events processed by IO / Main threads */ - long long stat_io_writes_processed; /* Number of write events processed by IO / Main threads */ - _Atomic long long stat_total_reads_processed; /* Total number of read events processed */ - _Atomic long long stat_total_writes_processed; /* Total number of write events processed */ - _Atomic long long stat_client_qbuf_limit_disconnections; /* Total number of clients reached query buf length limit */ + long long stat_dump_payload_sanitizations; /* Number deep dump payloads integrity validations. */ + long long stat_io_reads_processed; /* Number of read events processed by IO threads */ + long long stat_io_writes_processed; /* Number of write events processed by IO threads */ + long long stat_total_reads_processed; /* Total number of read events processed */ + long long stat_total_writes_processed; /* Total number of write events processed */ + long long stat_client_qbuf_limit_disconnections; /* Total number of clients reached query buf length limit */ long long stat_client_outbuf_limit_disconnections; /* Total number of clients reached output buf length limit */ /* The following two are used to track instantaneous metrics, like * number of operations per second, network traffic. */ @@ -1881,6 +1898,8 @@ struct valkeyServer { int syslog_facility; /* Syslog facility */ int crashlog_enabled; /* Enable signal handler for crashlog. * disable for clean core dumps. */ + int crashed; /* True if the server has crashed, used in catClientInfoString + * to indicate that no wait for IO threads is needed. */ int memcheck_enabled; /* Enable memory check on crash. */ int use_exit_on_panic; /* Use exit() on panic and assert rather than * abort(). useful for Valgrind. */ @@ -2002,7 +2021,7 @@ struct valkeyServer { int list_max_listpack_size; int list_compress_depth; /* time cache */ - _Atomic time_t unixtime; /* Unix time sampled every cron cycle. */ + time_t unixtime; /* Unix time sampled every cron cycle. */ time_t timezone; /* Cached timezone. As set by tzset(). */ int daylight_active; /* Currently in daylight saving time. */ mstime_t mstime; /* 'unixtime' in milliseconds. */ @@ -2491,11 +2510,6 @@ typedef struct { #define OBJ_HASH_KEY 1 #define OBJ_HASH_VALUE 2 -#define IO_THREADS_OP_IDLE 0 -#define IO_THREADS_OP_READ 1 -#define IO_THREADS_OP_WRITE 2 -extern int io_threads_op; - /*----------------------------------------------------------------------------- * Extern declarations *----------------------------------------------------------------------------*/ @@ -2601,11 +2615,35 @@ void dictVanillaFree(dict *d, void *val); (1ULL << 0) /* Indicating that we should not update \ error stats after sending error reply */ /* networking.c -- Networking and Client related operations */ + +/* Read flags for various read errors and states */ +#define READ_FLAGS_QB_LIMIT_REACHED (1 << 0) +#define READ_FLAGS_ERROR_BIG_INLINE_REQUEST (1 << 1) +#define READ_FLAGS_ERROR_BIG_MULTIBULK (1 << 2) +#define READ_FLAGS_ERROR_INVALID_MULTIBULK_LEN (1 << 3) +#define READ_FLAGS_ERROR_UNAUTHENTICATED_MULTIBULK_LEN (1 << 4) +#define READ_FLAGS_ERROR_UNAUTHENTICATED_BULK_LEN (1 << 5) +#define READ_FLAGS_ERROR_BIG_BULK_COUNT (1 << 6) +#define READ_FLAGS_ERROR_MBULK_UNEXPECTED_CHARACTER (1 << 7) +#define READ_FLAGS_ERROR_MBULK_INVALID_BULK_LEN (1 << 8) +#define READ_FLAGS_ERROR_UNEXPECTED_INLINE_FROM_PRIMARY (1 << 9) +#define READ_FLAGS_ERROR_UNBALANCED_QUOTES (1 << 10) +#define READ_FLAGS_INLINE_ZERO_QUERY_LEN (1 << 11) +#define READ_FLAGS_PARSING_NEGATIVE_MBULK_LEN (1 << 12) +#define READ_FLAGS_PARSING_COMPLETED (1 << 13) +#define READ_FLAGS_PRIMARY (1 << 14) +#define READ_FLAGS_DONT_PARSE (1 << 15) +#define READ_FLAGS_AUTH_REQUIRED (1 << 16) + +/* Write flags for various write errors and states */ +#define WRITE_FLAGS_WRITE_ERROR (1 << 0) + + client *createClient(connection *conn); void freeClient(client *c); void freeClientAsync(client *c); void logInvalidUseAndFreeClientAsync(client *c, const char *fmt, ...); -int beforeNextClient(client *c); +void beforeNextClient(client *c); void clearClientConnectionState(client *c); void resetClient(client *c); void freeClientOriginalArgv(client *c); @@ -2698,24 +2736,28 @@ void whileBlockedCron(void); void blockingOperationStarts(void); void blockingOperationEnds(void); int handleClientsWithPendingWrites(void); -int handleClientsWithPendingWritesUsingThreads(void); -int handleClientsWithPendingReadsUsingThreads(void); -int stopThreadedIOIfNeeded(void); +void adjustThreadedIOIfNeeded(void); int clientHasPendingReplies(client *c); int updateClientMemUsageAndBucket(client *c); void removeClientFromMemUsageBucket(client *c, int allow_eviction); void unlinkClient(client *c); -int writeToClient(client *c, int handler_installed); +int writeToClient(client *c); void linkClient(client *c); void protectClient(client *c); void unprotectClient(client *c); -void initThreadedIO(void); void initSharedQueryBuf(void); +void freeSharedQueryBuf(void); client *lookupClientByID(uint64_t id); int authRequired(client *c); void putClientInPendingWriteQueue(client *c); client *createCachedResponseClient(int resp); void deleteCachedResponseClient(client *recording_client); +void waitForClientIO(client *c); +void ioThreadReadQueryFromClient(void *data); +void ioThreadWriteToClient(void *data); +int canParseCommand(client *c); +int processIOThreadsReadDone(void); +int processIOThreadsWriteDone(void); /* logreqres.c - logging of requests and responses */ void reqresReset(client *c, int free_buf); @@ -3834,7 +3876,6 @@ void xorDigest(unsigned char *digest, const void *ptr, size_t len); sds catSubCommandFullname(const char *parent_name, const char *sub_name); void commandAddSubcommand(struct serverCommand *parent, struct serverCommand *subcommand, const char *declared_name); void debugDelay(int usec); -void killIOThreads(void); void killThreads(void); void makeThreadKillable(void); void swapMainDbWithTempDb(serverDb *tempDb); diff --git a/src/socket.c b/src/socket.c index 5aa3606990..b2f8f1aaec 100644 --- a/src/socket.c +++ b/src/socket.c @@ -423,6 +423,8 @@ static ConnectionType CT_Socket = { /* pending data */ .has_pending_data = NULL, .process_pending_data = NULL, + .postpone_update_state = NULL, + .update_state = NULL, }; int connBlock(connection *conn) { diff --git a/src/tls.c b/src/tls.c index 2d4d6cd0ae..1913d876fa 100644 --- a/src/tls.c +++ b/src/tls.c @@ -442,6 +442,7 @@ typedef enum { WANT_READ = 1, WANT_WRITE } WantIOType; #define TLS_CONN_FLAG_READ_WANT_WRITE (1 << 0) #define TLS_CONN_FLAG_WRITE_WANT_READ (1 << 1) #define TLS_CONN_FLAG_FD_SET (1 << 2) +#define TLS_CONN_FLAG_POSTPONE_UPDATE_STATE (1 << 3) typedef struct tls_connection { connection c; @@ -596,7 +597,34 @@ static void registerSSLEvent(tls_connection *conn, WantIOType want) { } } +static void postPoneUpdateSSLState(connection *conn_, int postpone) { + tls_connection *conn = (tls_connection *)conn_; + if (postpone) { + conn->flags |= TLS_CONN_FLAG_POSTPONE_UPDATE_STATE; + } else { + conn->flags &= ~TLS_CONN_FLAG_POSTPONE_UPDATE_STATE; + } +} + +static void updatePendingData(tls_connection *conn) { + if (conn->flags & TLS_CONN_FLAG_POSTPONE_UPDATE_STATE) return; + + /* If SSL has pending data, already read from the socket, we're at risk of not calling the read handler again, make + * sure to add it to a list of pending connection that should be handled anyway. */ + if (SSL_pending(conn->ssl) > 0) { + if (!conn->pending_list_node) { + listAddNodeTail(pending_list, conn); + conn->pending_list_node = listLast(pending_list); + } + } else if (conn->pending_list_node) { + listDelNode(pending_list, conn->pending_list_node); + conn->pending_list_node = NULL; + } +} + static void updateSSLEvent(tls_connection *conn) { + if (conn->flags & TLS_CONN_FLAG_POSTPONE_UPDATE_STATE) return; + int mask = aeGetFileEvents(server.el, conn->c.fd); int need_read = conn->c.read_handler || (conn->flags & TLS_CONN_FLAG_WRITE_WANT_READ); int need_write = conn->c.write_handler || (conn->flags & TLS_CONN_FLAG_READ_WANT_WRITE); @@ -610,6 +638,12 @@ static void updateSSLEvent(tls_connection *conn) { if (!need_write && (mask & AE_WRITABLE)) aeDeleteFileEvent(server.el, conn->c.fd, AE_WRITABLE); } +static void updateSSLState(connection *conn_) { + tls_connection *conn = (tls_connection *)conn_; + updateSSLEvent(conn); + updatePendingData(conn); +} + static void tlsHandleEvent(tls_connection *conn, int mask) { int ret, conn_error; @@ -711,19 +745,8 @@ static void tlsHandleEvent(tls_connection *conn, int mask) { if (!callHandler((connection *)conn, conn->c.read_handler)) return; } - /* If SSL has pending that, already read from the socket, we're at - * risk of not calling the read handler again, make sure to add it - * to a list of pending connection that should be handled anyway. */ - if ((mask & AE_READABLE)) { - if (SSL_pending(conn->ssl) > 0) { - if (!conn->pending_list_node) { - listAddNodeTail(pending_list, conn); - conn->pending_list_node = listLast(pending_list); - } - } else if (conn->pending_list_node) { - listDelNode(pending_list, conn->pending_list_node); - conn->pending_list_node = NULL; - } + if (mask & AE_READABLE) { + updatePendingData(conn); } break; @@ -1051,11 +1074,13 @@ static int tlsProcessPendingData(void) { listIter li; listNode *ln; - int processed = listLength(pending_list); + int processed = 0; listRewind(pending_list, &li); while ((ln = listNext(&li))) { tls_connection *conn = listNodeValue(ln); + if (conn->flags & TLS_CONN_FLAG_POSTPONE_UPDATE_STATE) continue; tlsHandleEvent(conn, AE_READABLE); + processed++; } return processed; } @@ -1125,6 +1150,8 @@ static ConnectionType CT_TLS = { /* pending data */ .has_pending_data = tlsHasPendingData, .process_pending_data = tlsProcessPendingData, + .postpone_update_state = postPoneUpdateSSLState, + .update_state = updateSSLState, /* TLS specified methods */ .get_peer_cert = connTLSGetPeerCert, diff --git a/src/unix.c b/src/unix.c index ca38e83ed0..795b2db9f1 100644 --- a/src/unix.c +++ b/src/unix.c @@ -198,6 +198,8 @@ static ConnectionType CT_Unix = { /* pending data */ .has_pending_data = NULL, .process_pending_data = NULL, + .postpone_update_state = NULL, + .update_state = NULL, }; int RedisRegisterConnectionTypeUnix(void) { diff --git a/tests/integration/failover.tcl b/tests/integration/failover.tcl index 70bb66284d..3049cd0ca0 100644 --- a/tests/integration/failover.tcl +++ b/tests/integration/failover.tcl @@ -257,6 +257,12 @@ start_server {overrides {save {}}} { # during the pause. This write will not be interrupted. pause_process [srv -1 pid] set rd [valkey_deferring_client] + # wait for the client creation + wait_for_condition 50 100 { + [s connected_clients] == 2 + } else { + fail "Client creation failed" + } $rd SET FOO BAR $node_0 failover to $node_1_host $node_1_port resume_process [srv -1 pid] diff --git a/tests/integration/replication.tcl b/tests/integration/replication.tcl index f56fe0a1dc..9634f78252 100644 --- a/tests/integration/replication.tcl +++ b/tests/integration/replication.tcl @@ -167,6 +167,7 @@ start_server {tags {"repl external:skip"}} { test {BLPOP followed by role change, issue #2473} { set rd [valkey_deferring_client] $rd blpop foo 0 ; # Block while B is a master + wait_for_blocked_clients_count 1 # Turn B into master of A $A slaveof no one diff --git a/tests/integration/shutdown.tcl b/tests/integration/shutdown.tcl index b2fdb845a3..9949afe27c 100644 --- a/tests/integration/shutdown.tcl +++ b/tests/integration/shutdown.tcl @@ -156,6 +156,12 @@ test "Shutting down master waits for replica then fails" { set rd2 [valkey_deferring_client -1] $rd1 shutdown $rd2 shutdown + wait_for_condition 50 100 { + [llength [lsearch -all [split [string trim [$master client list]] "\r\n"] *cmd=shutdown*]] == 2 + } else { + fail "SHUTDOWN not called on all clients" + } + set info_clients [$master info clients] assert_match "*connected_clients:3*" $info_clients assert_match "*blocked_clients:2*" $info_clients @@ -209,6 +215,12 @@ test "Shutting down master waits for replica then aborted" { set rd2 [valkey_deferring_client -1] $rd1 shutdown $rd2 shutdown + wait_for_condition 50 100 { + [llength [lsearch -all [split [string trim [$master client list]] "\r\n"] *cmd=shutdown*]] == 2 + } else { + fail "SHUTDOWN not called on all clients" + } + set info_clients [$master info clients] assert_match "*connected_clients:3*" $info_clients assert_match "*blocked_clients:2*" $info_clients diff --git a/tests/integration/valkey-cli.tcl b/tests/integration/valkey-cli.tcl index 153c527055..6344215a25 100644 --- a/tests/integration/valkey-cli.tcl +++ b/tests/integration/valkey-cli.tcl @@ -65,6 +65,7 @@ start_server {tags {"cli"}} { proc run_command {fd cmd} { write_cli $fd $cmd + after 50 set _ [format_output [read_cli $fd]] } diff --git a/tests/unit/client-eviction.tcl b/tests/unit/client-eviction.tcl index afcdcd1323..ceeb20f7b6 100644 --- a/tests/unit/client-eviction.tcl +++ b/tests/unit/client-eviction.tcl @@ -91,17 +91,31 @@ start_server {} { lassign [gen_client] rr cname # Attempt to fill the query buff with only half the percentage threshold verify we're not disconnected set n [expr $maxmemory_clients_actual / 2] - $rr write [join [list "*1\r\n\$$n\r\n" [string repeat v $n]] ""] + # send incomplete command (n - 1) to make sure we don't use the shared qb + $rr write [join [list "*1\r\n\$$n\r\n" [string repeat v [expr {$n - 1}]]] ""] $rr flush + # Wait for the client to start using a private query buffer. + wait_for_condition 10 10 { + [client_field $cname qbuf] > 0 + } else { + fail "client should start using a private query buffer" + } set tot_mem [client_field $cname tot-mem] assert {$tot_mem >= $n && $tot_mem < $maxmemory_clients_actual} # Attempt to fill the query buff with the percentage threshold of maxmemory and verify we're evicted $rr close lassign [gen_client] rr cname + # send incomplete command (maxmemory_clients_actual - 1) to make sure we don't use the shared qb catch { - $rr write [join [list "*1\r\n\$$maxmemory_clients_actual\r\n" [string repeat v $maxmemory_clients_actual]] ""] + $rr write [join [list "*1\r\n\$$maxmemory_clients_actual\r\n" [string repeat v [expr {$maxmemory_clients_actual - 1}]]] ""] $rr flush + # Wait for the client to start using a private query buffer. + wait_for_condition 10 10 { + [client_field $cname qbuf] > 0 + } else { + fail "client should start using a private query buffer" + } } e assert {![client_exists $cname]} $rr close @@ -399,6 +413,11 @@ start_server {} { # Decrease maxmemory_clients and expect client eviction r config set maxmemory-clients [expr $maxmemory_clients / 2] + wait_for_condition 50 10 { + [llength [lsearch -all [split [string trim [r client list]] "\r\n"] *name=client*]] < $client_count + } else { + fail "Failed to evict clients" + } set connected_clients [llength [lsearch -all [split [string trim [r client list]] "\r\n"] *name=client*]] assert {$connected_clients > 0 && $connected_clients < $client_count} diff --git a/tests/unit/cluster/pubsubshard.tcl b/tests/unit/cluster/pubsubshard.tcl index e32b6a3a0e..d38c22dedb 100644 --- a/tests/unit/cluster/pubsubshard.tcl +++ b/tests/unit/cluster/pubsubshard.tcl @@ -62,7 +62,13 @@ test "sunsubscribe without specifying any channel would unsubscribe all shard ch set sub_res [ssubscribe $subscribeclient [list "\{channel.0\}1" "\{channel.0\}2" "\{channel.0\}3"]] assert_equal [list 1 2 3] $sub_res sunsubscribe $subscribeclient - + + # wait for the unsubscribe to take effect + wait_for_condition 50 10 { + [$publishclient spublish "\{channel.0\}1" hello] eq 0 + } else { + fail "unsubscribe did not take effect as expected" + } assert_equal 0 [$publishclient spublish "\{channel.0\}1" hello] assert_equal 0 [$publishclient spublish "\{channel.0\}2" hello] assert_equal 0 [$publishclient spublish "\{channel.0\}3" hello] diff --git a/tests/unit/dump.tcl b/tests/unit/dump.tcl index 9018270d67..e4c0f9d312 100644 --- a/tests/unit/dump.tcl +++ b/tests/unit/dump.tcl @@ -287,6 +287,7 @@ start_server {tags {"dump"}} { set rd [valkey_deferring_client] $rd debug sleep 1.0 ; # Make second server unable to reply. + after 100; # wait to make sure DEBUG command was executed. set e {} catch {r -1 migrate $second_host $second_port key 9 500} e assert_match {IOERR*} $e diff --git a/tests/unit/info.tcl b/tests/unit/info.tcl index 17dc6a1861..befecae220 100644 --- a/tests/unit/info.tcl +++ b/tests/unit/info.tcl @@ -295,47 +295,50 @@ start_server {tags {"info" "external:skip"}} { } } - test {stats: eventloop metrics} { - set info1 [r info stats] - set cycle1 [getInfoProperty $info1 eventloop_cycles] - set el_sum1 [getInfoProperty $info1 eventloop_duration_sum] - set cmd_sum1 [getInfoProperty $info1 eventloop_duration_cmd_sum] - assert_morethan $cycle1 0 - assert_morethan $el_sum1 0 - assert_morethan $cmd_sum1 0 - after 110 ;# default hz is 10, wait for a cron tick. - set info2 [r info stats] - set cycle2 [getInfoProperty $info2 eventloop_cycles] - set el_sum2 [getInfoProperty $info2 eventloop_duration_sum] - set cmd_sum2 [getInfoProperty $info2 eventloop_duration_cmd_sum] - if {$::verbose} { puts "eventloop metrics cycle1: $cycle1, cycle2: $cycle2" } - assert_morethan $cycle2 $cycle1 - assert_lessthan $cycle2 [expr $cycle1+10] ;# we expect 2 or 3 cycles here, but allow some tolerance - if {$::verbose} { puts "eventloop metrics el_sum1: $el_sum1, el_sum2: $el_sum2" } - assert_morethan $el_sum2 $el_sum1 - assert_lessthan $el_sum2 [expr $el_sum1+30000] ;# we expect roughly 100ms here, but allow some tolerance - if {$::verbose} { puts "eventloop metrics cmd_sum1: $cmd_sum1, cmd_sum2: $cmd_sum2" } - assert_morethan $cmd_sum2 $cmd_sum1 - assert_lessthan $cmd_sum2 [expr $cmd_sum1+15000] ;# we expect about tens of ms here, but allow some tolerance - } - - test {stats: instantaneous metrics} { - r config resetstat - set retries 0 - for {set retries 1} {$retries < 4} {incr retries} { - after 1600 ;# hz is 10, wait for 16 cron tick so that sample array is fulfilled - set value [s instantaneous_eventloop_cycles_per_sec] - if {$value > 0} break + # skip the following 2 tests if we are running with io-threads as the eventloop metrics are different in that case. + if {[r config get io-threads] eq 0} { + test {stats: eventloop metrics} { + set info1 [r info stats] + set cycle1 [getInfoProperty $info1 eventloop_cycles] + set el_sum1 [getInfoProperty $info1 eventloop_duration_sum] + set cmd_sum1 [getInfoProperty $info1 eventloop_duration_cmd_sum] + assert_morethan $cycle1 0 + assert_morethan $el_sum1 0 + assert_morethan $cmd_sum1 0 + after 110 ;# default hz is 10, wait for a cron tick. + set info2 [r info stats] + set cycle2 [getInfoProperty $info2 eventloop_cycles] + set el_sum2 [getInfoProperty $info2 eventloop_duration_sum] + set cmd_sum2 [getInfoProperty $info2 eventloop_duration_cmd_sum] + if {$::verbose} { puts "eventloop metrics cycle1: $cycle1, cycle2: $cycle2" } + assert_morethan $cycle2 $cycle1 + assert_lessthan $cycle2 [expr $cycle1+10] ;# we expect 2 or 3 cycles here, but allow some tolerance + if {$::verbose} { puts "eventloop metrics el_sum1: $el_sum1, el_sum2: $el_sum2" } + assert_morethan $el_sum2 $el_sum1 + assert_lessthan $el_sum2 [expr $el_sum1+30000] ;# we expect roughly 100ms here, but allow some tolerance + if {$::verbose} { puts "eventloop metrics cmd_sum1: $cmd_sum1, cmd_sum2: $cmd_sum2" } + assert_morethan $cmd_sum2 $cmd_sum1 + assert_lessthan $cmd_sum2 [expr $cmd_sum1+15000] ;# we expect about tens of ms here, but allow some tolerance + } + + test {stats: instantaneous metrics} { + r config resetstat + set retries 0 + for {set retries 1} {$retries < 4} {incr retries} { + after 1600 ;# hz is 10, wait for 16 cron tick so that sample array is fulfilled + set value [s instantaneous_eventloop_cycles_per_sec] + if {$value > 0} break + } + + assert_lessthan $retries 4 + if {$::verbose} { puts "instantaneous metrics instantaneous_eventloop_cycles_per_sec: $value" } + assert_morethan $value 0 + assert_lessthan $value [expr $retries*15] ;# default hz is 10 + set value [s instantaneous_eventloop_duration_usec] + if {$::verbose} { puts "instantaneous metrics instantaneous_eventloop_duration_usec: $value" } + assert_morethan $value 0 + assert_lessthan $value [expr $retries*22000] ;# default hz is 10, so duration < 1000 / 10, allow some tolerance } - - assert_lessthan $retries 4 - if {$::verbose} { puts "instantaneous metrics instantaneous_eventloop_cycles_per_sec: $value" } - assert_morethan $value 0 - assert_lessthan $value [expr $retries*15] ;# default hz is 10 - set value [s instantaneous_eventloop_duration_usec] - if {$::verbose} { puts "instantaneous metrics instantaneous_eventloop_duration_usec: $value" } - assert_morethan $value 0 - assert_lessthan $value [expr $retries*22000] ;# default hz is 10, so duration < 1000 / 10, allow some tolerance } test {stats: debug metrics} { diff --git a/tests/unit/maxmemory.tcl b/tests/unit/maxmemory.tcl index ee1232796d..66dae2546a 100644 --- a/tests/unit/maxmemory.tcl +++ b/tests/unit/maxmemory.tcl @@ -98,6 +98,7 @@ start_server {tags {"maxmemory" "external:skip"}} { $rr write "\r\n" $rr flush } + after 100; # give the server some time to process the input buffer - this was added to make sure the test pass with io-threads active. }]} { lremove clients $rr } diff --git a/tests/unit/memefficiency.tcl b/tests/unit/memefficiency.tcl index 525db407bf..feb98d9cdd 100644 --- a/tests/unit/memefficiency.tcl +++ b/tests/unit/memefficiency.tcl @@ -404,6 +404,8 @@ run_solo {defrag} { r save ;# saving an rdb iterates over all the data / pointers } {OK} + # Skip the following two tests if we are running with IO threads, as the IO threads allocate the command arguments in a different arena. As a result, fragmentation is not as expected. + if {[r config get io-threads] eq 0} { test "Active defrag pubsub: $type" { r flushdb r config resetstat @@ -502,6 +504,7 @@ run_solo {defrag} { } $rd_pubsub close } + } ;# io-threads if {$type eq "standalone"} { ;# skip in cluster mode test "Active defrag big list: $type" { diff --git a/tests/unit/moduleapi/blockedclient.tcl b/tests/unit/moduleapi/blockedclient.tcl index d94ef5c5ba..bb0a15db50 100644 --- a/tests/unit/moduleapi/blockedclient.tcl +++ b/tests/unit/moduleapi/blockedclient.tcl @@ -128,7 +128,7 @@ foreach call_type {nested normal} { # send another command after the blocked one, to make sure we don't attempt to process it $rd ping $rd flush - + after 100 # 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 diff --git a/tests/unit/pubsub.tcl b/tests/unit/pubsub.tcl index 2f336dfcb9..72d0498ce1 100644 --- a/tests/unit/pubsub.tcl +++ b/tests/unit/pubsub.tcl @@ -85,6 +85,12 @@ start_server {tags {"pubsub network"}} { set rd1 [valkey_deferring_client] assert_equal {1 2 3} [subscribe $rd1 {chan1 chan2 chan3}] unsubscribe $rd1 + # wait for the unsubscribe to take effect + wait_for_condition 50 100 { + [r publish chan1 hello] eq 0 + } else { + fail "unsubscribe did not take effect" + } assert_equal 0 [r publish chan1 hello] assert_equal 0 [r publish chan2 hello] assert_equal 0 [r publish chan3 hello] @@ -158,6 +164,12 @@ start_server {tags {"pubsub network"}} { set rd1 [valkey_deferring_client] assert_equal {1 2 3} [psubscribe $rd1 {chan1.* chan2.* chan3.*}] punsubscribe $rd1 + # wait for the unsubscribe to take effect + wait_for_condition 50 100 { + [r publish chan1.hi hello] eq 0 + } else { + fail "unsubscribe did not take effect" + } assert_equal 0 [r publish chan1.hi hello] assert_equal 0 [r publish chan2.hi hello] assert_equal 0 [r publish chan3.hi hello] diff --git a/tests/unit/pubsubshard.tcl b/tests/unit/pubsubshard.tcl index d56f36ffaa..e19db211f7 100644 --- a/tests/unit/pubsubshard.tcl +++ b/tests/unit/pubsubshard.tcl @@ -46,6 +46,14 @@ start_server {tags {"pubsubshard external:skip"}} { assert_equal {2} [ssubscribe $rd1 {chan2}] assert_equal {3} [ssubscribe $rd1 {chan3}] sunsubscribe $rd1 + + # wait for the unsubscribe to take effect + wait_for_condition 50 100 { + [r spublish chan1 hello] eq 0 + } else { + fail "unsubscribe did not take effect" + } + assert_equal 0 [r SPUBLISH chan1 hello] assert_equal 0 [r SPUBLISH chan2 hello] assert_equal 0 [r SPUBLISH chan3 hello] diff --git a/tests/unit/querybuf.tcl b/tests/unit/querybuf.tcl index 519743d248..f0f432b38f 100644 --- a/tests/unit/querybuf.tcl +++ b/tests/unit/querybuf.tcl @@ -92,7 +92,7 @@ start_server {tags {"querybuf slow"}} { # Write something smaller, so query buf peak can shrink $rd set x [string repeat A 100] set new_test_client_qbuf [client_query_buffer test_client] - if {$new_test_client_qbuf < $orig_test_client_qbuf} { break } + if {$new_test_client_qbuf < $orig_test_client_qbuf && $new_test_client_qbuf > 0} { break } if {[expr [clock milliseconds] - $t] > 1000} { break } after 10 } diff --git a/tests/unit/type/list.tcl b/tests/unit/type/list.tcl index e6c8bb331f..4773a58820 100644 --- a/tests/unit/type/list.tcl +++ b/tests/unit/type/list.tcl @@ -1100,6 +1100,13 @@ foreach {pop} {BLPOP BLMPOP_LEFT} { $watching_client get somekey{t} $watching_client read $watching_client exec + # wait for exec to be called. + wait_for_condition 50 10 { + [llength [lsearch -all [split [string trim [r client list]] "\r\n"] *cmd=exec*]] == 1 + } else { + fail "$cmd was not called" + } + # Blocked BLPOPLPUSH may create problems, unblock it. r lpush srclist{t} element set res [$watching_client read] diff --git a/tests/unit/type/stream-cgroups.tcl b/tests/unit/type/stream-cgroups.tcl index 2cd812e521..d934e48140 100644 --- a/tests/unit/type/stream-cgroups.tcl +++ b/tests/unit/type/stream-cgroups.tcl @@ -520,7 +520,7 @@ start_server { # Before the fix in #13004, this time would have been 1200+ (i.e. more than 1200ms), # now it should be 1000, but in order to avoid timing issues, we increase the range a bit. - assert_range [expr $end-$start] 1000 1150 + assert_range [expr $end-$start] 1000 1199 $rd1 close $rd2 close @@ -931,14 +931,14 @@ start_server { set reply [r xinfo consumers mystream mygroup] set consumer_info [lindex $reply 0] assert_equal [lindex $consumer_info 1] "Alice" ;# consumer name - assert {[dict get $consumer_info idle] < 80} ;# consumer idle (seen-time) - assert {[dict get $consumer_info inactive] < 80} ;# consumer inactive (active-time) + assert {[dict get $consumer_info idle] < 300} ;# consumer idle (seen-time) + assert {[dict get $consumer_info inactive] < 300} ;# consumer inactive (active-time) after 100 r XREADGROUP GROUP mygroup Alice COUNT 1 STREAMS mystream > set reply [r xinfo consumers mystream mygroup] set consumer_info [lindex $reply 0] - assert {[dict get $consumer_info idle] < 80} ;# consumer idle (seen-time) + assert {[dict get $consumer_info idle] < 300} ;# consumer idle (seen-time) assert {[dict get $consumer_info inactive] >= 100} ;# consumer inactive (active-time) @@ -1324,6 +1324,9 @@ start_server { assert_equal [dict get $group entries-read] 3 assert_equal [dict get $group lag] 0 + # wait for replica offset + wait_for_ofs_sync $master $replica + set reply [$replica XINFO STREAM mystream FULL] set group [lindex [dict get $reply groups] 0] assert_equal [dict get $group entries-read] 3 diff --git a/tests/unit/type/zset.tcl b/tests/unit/type/zset.tcl index f6c643a5ef..b341bbf69d 100644 --- a/tests/unit/type/zset.tcl +++ b/tests/unit/type/zset.tcl @@ -2012,6 +2012,7 @@ start_server {tags {"zset"}} { # Before the fix in #13004, this time would have been 1200+ (i.e. more than 1200ms), # now it should be 1000, but in order to avoid timing issues, we increase the range a bit. assert_range [expr $end-$start] 1000 1150 + puts "Time: [expr $end-$start]" r debug set-active-expire 1 $rd close diff --git a/valkey.conf b/valkey.conf index e4ffd0f8ad..8badf1487a 100644 --- a/valkey.conf +++ b/valkey.conf @@ -1288,9 +1288,8 @@ lazyfree-lazy-user-flush no # to pipelining nor sharding of the instance. # # By default threading is disabled, we suggest enabling it only in machines -# that have at least 4 or more cores, leaving at least one spare core. -# Using more than 8 threads is unlikely to help much. We also recommend using -# threaded I/O only if you actually have performance problems, with +# that have at least 3 or more cores, leaving at least one spare core. +# We also recommend using threaded I/O only if you actually have performance problems, with # instances being able to use a quite big percentage of CPU time, otherwise # there is no point in using this feature. # @@ -1301,19 +1300,9 @@ lazyfree-lazy-user-flush no # io-threads 4 # # Setting io-threads to 1 will just use the main thread as usual. -# When I/O threads are enabled, we only use threads for writes, that is -# to thread the write(2) syscall and transfer the client buffers to the -# socket. However it is also possible to enable threading of reads and -# protocol parsing using the following configuration directive, by setting -# it to yes: -# -# io-threads-do-reads no -# -# Usually threading reads doesn't help much. -# -# NOTE 1: This configuration directive cannot be changed at runtime via -# CONFIG SET. Also, this feature currently does not work when SSL is -# enabled. +# When I/O threads are enabled, we use threads for reads and writes, that is +# to thread the write and read syscall and transfer the client buffers to the +# socket and to enable threading of reads and protocol parsing. # # NOTE 2: If you want to test the server speedup using valkey-benchmark, make # sure you also run the benchmark itself in threaded mode, using the