Unverified Commit 41d9eb02 authored by Oran Agra's avatar Oran Agra Committed by GitHub
Browse files

Merge: Fully abstract connection and make TLS dynamically loadable (#9320)

There are many commits in this PR, the detailed changes is described
in each commit message.

### Main changes in this PR

* Fully abstract connection type, and hide connection type specified methods.
  Ex, currently TLS class looks like:
```
static ConnectionType CT_TLS = {
    /* connection type */
    .get_type = connTLSGetType,

    /* connection type initialize & finalize & configure */
    .init = tlsInit,
    .cleanup = tlsCleanup,
    .configure = tlsConfigure,

    /* ae & accept & listen & error & address handler */
    .ae_handler = tlsEventHandler,
    .accept_handler = tlsAcceptHandler,
    .addr = connTLSAddr,
    .listen = connTLSListen,

    /* create/close connection */
    .conn_create = connCreateTLS,
    .conn_create_accepted = connCreateAcceptedTLS,
    .close = connTLSClose,

    /* connect & accept */
    .connect = connTLSConnect,
    .blocking_connect = connTLSBlockingConnect,
    .accept = connTLSAccept,

    /* IO */
    .read = connTLSRead,
    .write = connTLSWrite,
    .writev = connTLSWritev,
    .set_write_handler = connTLSSetWriteHandler,
    .set_read_handler = connTLSSetReadHandler,
    .get_last_error = connTLSGetLastError,
    .sync_write = connTLSSyncWrite,
    .sync_read = connTLSSyncRead,
    .sync_readline = connTLSSyncReadLine,

    /* pending data */
    .has_pending_data = tlsHasPendingData,
    .process_pending_data = tlsProcessPendingData,

    /* TLS specified methods */
    .get_peer_cert = connTLSGetPeerCert,
};

int RedisRegisterConnectionTypeTLS()
{
    return connTypeRegister(&CT_TLS);
}
```

* Also abstract Unix socket class. Currently, the connection framework becomes like:
```
                       uplayer
                          |
                   connection layer
                     /    |     \
                   TCP   Unix   TLS
    
```

* It's possible to build TLS as a shared library (`make BUILD_TLS=module`).
  Loading the shared library(redis-tls.so) into Redis by Redis module subsystem,
  and Redis starts to listen TLS port. Ex:
```
    ./src/redis-server --tls-port 6379 --port 0 \
        --tls-cert-file ./tests/tls/redis.crt \
        --tls-key-file ./tests/tls/redis.key \
        --tls-ca-cert-file ./tests/tls/ca.crt \
        --loadmodule src/redis-tls.so
```

### Interface changes
* RM_GetContextFlags supports a new flag: REDISMODULE_CTX_FLAGS_SERVER_STARTUP
* INFO SERVER includes a list of listeners:
```
listener0:name=tcp,bind=127.0.0.1,port=6380
listener1:name=unix,bind=/run/redis.sock
listener2:name=tls,bind=127.0.0.1,port=6379
```

### Other notes

* Fix wrong signature of RedisModuleDefragFunc, this could break
  compilation of a module, but not the ABI
* Some reordering of initialization order in server.c:
  * Move initialization of listeners to be after loading the modules
  * Config TLS after initialization of listeners
  * Init cluster after initialization of listeners
* Sentinel does not support the TLS module or any connection module
  since it uses hiredis for outbound connections, so when TLS is built as
  a module, sentinel lacks TLS support.
parents 90223759 4faddf18
......@@ -81,6 +81,7 @@ typedef long long ustime_t; /* microsecond time type. */
#include "connection.h" /* Connection abstraction */
#define REDISMODULE_CORE 1
typedef struct redisObject robj;
#include "redismodule.h" /* Redis modules API defines. */
/* Following includes allow test functions to be called from Redis main() */
......@@ -679,9 +680,6 @@ struct RedisModuleIO;
struct RedisModuleDigest;
struct RedisModuleCtx;
struct moduleLoadQueueEntry;
struct redisObject;
struct RedisModuleDefragCtx;
struct RedisModuleInfoCtx;
struct RedisModuleKeyOptCtx;
struct RedisModuleCommand;
......@@ -701,20 +699,12 @@ typedef size_t (*moduleTypeFreeEffortFunc)(struct redisObject *key, const void *
typedef void (*moduleTypeUnlinkFunc)(struct redisObject *key, void *value);
typedef void *(*moduleTypeCopyFunc)(struct redisObject *fromkey, struct redisObject *tokey, const void *value);
typedef int (*moduleTypeDefragFunc)(struct RedisModuleDefragCtx *ctx, struct redisObject *key, void **value);
typedef void (*RedisModuleInfoFunc)(struct RedisModuleInfoCtx *ctx, int for_crash_report);
typedef void (*RedisModuleDefragFunc)(struct RedisModuleDefragCtx *ctx);
typedef size_t (*moduleTypeMemUsageFunc2)(struct RedisModuleKeyOptCtx *ctx, const void *value, size_t sample_size);
typedef void (*moduleTypeFreeFunc2)(struct RedisModuleKeyOptCtx *ctx, void *value);
typedef size_t (*moduleTypeFreeEffortFunc2)(struct RedisModuleKeyOptCtx *ctx, const void *value);
typedef void (*moduleTypeUnlinkFunc2)(struct RedisModuleKeyOptCtx *ctx, void *value);
typedef void *(*moduleTypeCopyFunc2)(struct RedisModuleKeyOptCtx *ctx, const void *value);
/* This callback type is called by moduleNotifyUserChanged() every time
* a user authenticated via the module API is associated with a different
* user or gets disconnected. This needs to be exposed since you can't cast
* a function pointer to (void *). */
typedef void (*RedisModuleUserChangedFunc) (uint64_t client_id, void *privdata);
/* The module type, which is referenced in each value of a given type, defines
* the methods and links to the module exporting the type. */
......@@ -786,7 +776,7 @@ typedef struct RedisModule RedisModule;
/* This is a wrapper for the 'rio' streams used inside rdb.c in Redis, so that
* the user does not have to take the total count of the written bytes nor
* to care about error conditions. */
typedef struct RedisModuleIO {
struct RedisModuleIO {
size_t bytes; /* Bytes read / written so far. */
rio *rio; /* Rio stream. */
moduleType *type; /* Module type doing the operation. */
......@@ -794,7 +784,7 @@ typedef struct RedisModuleIO {
struct RedisModuleCtx *ctx; /* Optional context, see RM_GetContextFromIO()*/
struct redisObject *key; /* Optional name of key processed */
int dbid; /* The dbid of the key being processed, -1 when unknown. */
} RedisModuleIO;
};
/* Macro to initialize an IO context. Note that the 'ver' field is populated
* inside rdb.c according to the version of the value to load. */
......@@ -813,12 +803,12 @@ typedef struct RedisModuleIO {
* a data structure, so that a digest can be created in a way that correctly
* reflects the values. See the DEBUG DIGEST command implementation for more
* background. */
typedef struct RedisModuleDigest {
struct RedisModuleDigest {
unsigned char o[20]; /* Ordered elements. */
unsigned char x[20]; /* Xored elements. */
struct redisObject *key; /* Optional name of key processed */
int dbid; /* The dbid of the key being processed */
} RedisModuleDigest;
};
/* Just start with a digest composed of all zero bytes. */
#define moduleInitDigestContext(mdvar) do { \
......@@ -849,7 +839,7 @@ typedef struct RedisModuleDigest {
#define OBJ_SHARED_REFCOUNT INT_MAX /* Global object never destroyed. */
#define OBJ_STATIC_REFCOUNT (INT_MAX-1) /* Object allocated in the stack. */
#define OBJ_FIRST_SPECIAL_REFCOUNT OBJ_STATIC_REFCOUNT
typedef struct redisObject {
struct redisObject {
unsigned type:4;
unsigned encoding:4;
unsigned lru:LRU_BITS; /* LRU time (relative to global lru_clock) or
......@@ -857,7 +847,7 @@ typedef struct redisObject {
* and most significant 16 bits access time). */
int refcount;
void *ptr;
} robj;
};
/* The a string name for an object's type as listed above
* Native types are checked against the OBJ_STRING, OBJ_LIST, OBJ_* defines,
......@@ -1376,11 +1366,6 @@ struct malloc_stats {
size_t allocator_resident;
};
typedef struct socketFds {
int fd[CONFIG_BINDADDR_MAX];
int count;
} socketFds;
/*-----------------------------------------------------------------------------
* TLS Context Configuration
*----------------------------------------------------------------------------*/
......@@ -1514,11 +1499,9 @@ struct redisServer {
char *bind_source_addr; /* Source address to bind on for outgoing connections */
char *unixsocket; /* UNIX socket path */
unsigned int unixsocketperm; /* UNIX socket permission (see mode_t) */
socketFds ipfd; /* TCP socket file descriptors */
socketFds tlsfd; /* TLS socket file descriptors */
int sofd; /* Unix socket file descriptor */
connListener listeners[CONN_TYPE_MAX]; /* TCP/Unix/TLS even more types */
uint32_t socket_mark_id; /* ID for listen socket marking */
socketFds cfd; /* Cluster bus listening socket */
connListener clistener; /* Cluster bus listener */
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. */
......@@ -2460,9 +2443,7 @@ void setDeferredSetLen(client *c, void *node, long length);
void setDeferredAttributeLen(client *c, void *node, long length);
void setDeferredPushLen(client *c, void *node, long length);
int processInputBuffer(client *c);
void acceptTcpHandler(aeEventLoop *el, int fd, void *privdata, int mask);
void acceptTLSHandler(aeEventLoop *el, int fd, void *privdata, int mask);
void acceptUnixHandler(aeEventLoop *el, int fd, void *privdata, int mask);
void acceptCommonHandler(connection *conn, int flags, char *ip);
void readQueryFromClient(connection *conn);
int prepareClientToWrite(client *c);
void addReplyNull(client *c);
......@@ -2527,7 +2508,7 @@ char *getClientTypeName(int class);
void flushSlavesOutputBuffers(void);
void disconnectSlaves(void);
void evictClients(void);
int listenToPort(int port, socketFds *fds);
int listenToPort(connListener *fds);
void pauseClients(pause_purpose purpose, mstime_t end, pause_type type);
void unpauseClients(pause_purpose purpose);
int areClientsPaused(void);
......@@ -2893,9 +2874,10 @@ int processCommand(client *c);
int processPendingCommandAndInputBuffer(client *c);
void setupSignalHandlers(void);
void removeSignalHandlers(void);
int createSocketAcceptHandler(socketFds *sfd, aeFileProc *accept_handler);
int changeListenPort(int port, socketFds *sfd, aeFileProc *accept_handler);
int changeBindAddr(void);
int createSocketAcceptHandler(connListener *sfd, aeFileProc *accept_handler);
connListener *listenerByType(const char *typename);
int changeListener(connListener *listener);
void closeListener(connListener *listener);
struct redisCommand *lookupSubcommand(struct redisCommand *container, sds sub_name);
struct redisCommand *lookupCommand(robj **argv, int argc);
struct redisCommand *lookupCommandBySdsLogic(dict *commands, sds s);
......@@ -3277,6 +3259,7 @@ void *dictSdsDup(dict *d, const void *key);
char *redisGitSHA1(void);
char *redisGitDirty(void);
uint64_t redisBuildId(void);
const char *redisBuildIdRaw(void);
char *redisBuildIdString(void);
/* Commands prototypes */
......@@ -3579,12 +3562,6 @@ void swapMainDbWithTempDb(redisDb *tempDb);
_serverLog(level, __VA_ARGS__);\
} while(0)
/* TLS stuff */
void tlsInit(void);
void tlsCleanup(void);
int tlsConfigure(redisTLSContextConfig *ctx_config);
int isTlsConfigured(void);
#define redisDebug(fmt, ...) \
printf("DEBUG %s:%d > " fmt "\n", __FILE__, __LINE__, __VA_ARGS__)
#define redisDebugMark() \
......
/*
* Copyright (c) 2019, Redis Labs
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* * Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of Redis nor the names of its contributors may be used
* to endorse or promote products derived from this software without
* specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include "server.h"
#include "connhelpers.h"
/* The connections module provides a lean abstraction of network connections
* to avoid direct socket and async event management across the Redis code base.
*
* It does NOT provide advanced connection features commonly found in similar
* libraries such as complete in/out buffer management, throttling, etc. These
* functions remain in networking.c.
*
* The primary goal is to allow transparent handling of TCP and TLS based
* connections. To do so, connections have the following properties:
*
* 1. A connection may live before its corresponding socket exists. This
* allows various context and configuration setting to be handled before
* establishing the actual connection.
* 2. The caller may register/unregister logical read/write handlers to be
* called when the connection has data to read from/can accept writes.
* These logical handlers may or may not correspond to actual AE events,
* depending on the implementation (for TCP they are; for TLS they aren't).
*/
static ConnectionType CT_Socket;
/* When a connection is created we must know its type already, but the
* underlying socket may or may not exist:
*
* - For accepted connections, it exists as we do not model the listen/accept
* part; So caller calls connCreateSocket() followed by connAccept().
* - For outgoing connections, the socket is created by the connection module
* itself; So caller calls connCreateSocket() followed by connConnect(),
* which registers a connect callback that fires on connected/error state
* (and after any transport level handshake was done).
*
* NOTE: An earlier version relied on connections being part of other structs
* and not independently allocated. This could lead to further optimizations
* like using container_of(), etc. However it was discontinued in favor of
* this approach for these reasons:
*
* 1. In some cases conns are created/handled outside the context of the
* containing struct, in which case it gets a bit awkward to copy them.
* 2. Future implementations may wish to allocate arbitrary data for the
* connection.
* 3. The container_of() approach is anyway risky because connections may
* be embedded in different structs, not just client.
*/
static connection *connCreateSocket(void) {
connection *conn = zcalloc(sizeof(connection));
conn->type = &CT_Socket;
conn->fd = -1;
return conn;
}
/* Create a new socket-type connection that is already associated with
* an accepted connection.
*
* The socket is not ready for I/O until connAccept() was called and
* invoked the connection-level accept handler.
*
* Callers should use connGetState() and verify the created connection
* is not in an error state (which is not possible for a socket connection,
* but could but possible with other protocols).
*/
static connection *connCreateAcceptedSocket(int fd, void *priv) {
UNUSED(priv);
connection *conn = connCreateSocket();
conn->fd = fd;
conn->state = CONN_STATE_ACCEPTING;
return conn;
}
static int connSocketConnect(connection *conn, const char *addr, int port, const char *src_addr,
ConnectionCallbackFunc connect_handler) {
int fd = anetTcpNonBlockBestEffortBindConnect(NULL,addr,port,src_addr);
if (fd == -1) {
conn->state = CONN_STATE_ERROR;
conn->last_errno = errno;
return C_ERR;
}
conn->fd = fd;
conn->state = CONN_STATE_CONNECTING;
conn->conn_handler = connect_handler;
aeCreateFileEvent(server.el, conn->fd, AE_WRITABLE,
conn->type->ae_handler, conn);
return C_OK;
}
/* ------ Pure socket connections ------- */
/* A very incomplete list of implementation-specific calls. Much of the above shall
* move here as we implement additional connection types.
*/
/* Close the connection and free resources. */
static void connSocketClose(connection *conn) {
if (conn->fd != -1) {
aeDeleteFileEvent(server.el,conn->fd, AE_READABLE | AE_WRITABLE);
close(conn->fd);
conn->fd = -1;
}
/* If called from within a handler, schedule the close but
* keep the connection until the handler returns.
*/
if (connHasRefs(conn)) {
conn->flags |= CONN_FLAG_CLOSE_SCHEDULED;
return;
}
zfree(conn);
}
static int connSocketWrite(connection *conn, const void *data, size_t data_len) {
int ret = write(conn->fd, data, data_len);
if (ret < 0 && errno != EAGAIN) {
conn->last_errno = errno;
/* Don't overwrite the state of a connection that is not already
* connected, not to mess with handler callbacks.
*/
if (errno != EINTR && conn->state == CONN_STATE_CONNECTED)
conn->state = CONN_STATE_ERROR;
}
return ret;
}
static int connSocketWritev(connection *conn, const struct iovec *iov, int iovcnt) {
int ret = writev(conn->fd, iov, iovcnt);
if (ret < 0 && errno != EAGAIN) {
conn->last_errno = errno;
/* Don't overwrite the state of a connection that is not already
* connected, not to mess with handler callbacks.
*/
if (errno != EINTR && conn->state == CONN_STATE_CONNECTED)
conn->state = CONN_STATE_ERROR;
}
return ret;
}
static int connSocketRead(connection *conn, void *buf, size_t buf_len) {
int ret = read(conn->fd, buf, buf_len);
if (!ret) {
conn->state = CONN_STATE_CLOSED;
} else if (ret < 0 && errno != EAGAIN) {
conn->last_errno = errno;
/* Don't overwrite the state of a connection that is not already
* connected, not to mess with handler callbacks.
*/
if (errno != EINTR && conn->state == CONN_STATE_CONNECTED)
conn->state = CONN_STATE_ERROR;
}
return ret;
}
static int connSocketAccept(connection *conn, ConnectionCallbackFunc accept_handler) {
int ret = C_OK;
if (conn->state != CONN_STATE_ACCEPTING) return C_ERR;
conn->state = CONN_STATE_CONNECTED;
connIncrRefs(conn);
if (!callHandler(conn, accept_handler)) ret = C_ERR;
connDecrRefs(conn);
return ret;
}
/* Register a write handler, to be called when the connection is writable.
* If NULL, the existing handler is removed.
*
* The barrier flag indicates a write barrier is requested, resulting with
* CONN_FLAG_WRITE_BARRIER set. This will ensure that the write handler is
* always called before and not after the read handler in a single event
* loop.
*/
static int connSocketSetWriteHandler(connection *conn, ConnectionCallbackFunc func, int barrier) {
if (func == conn->write_handler) return C_OK;
conn->write_handler = func;
if (barrier)
conn->flags |= CONN_FLAG_WRITE_BARRIER;
else
conn->flags &= ~CONN_FLAG_WRITE_BARRIER;
if (!conn->write_handler)
aeDeleteFileEvent(server.el,conn->fd,AE_WRITABLE);
else
if (aeCreateFileEvent(server.el,conn->fd,AE_WRITABLE,
conn->type->ae_handler,conn) == AE_ERR) return C_ERR;
return C_OK;
}
/* Register a read handler, to be called when the connection is readable.
* If NULL, the existing handler is removed.
*/
static int connSocketSetReadHandler(connection *conn, ConnectionCallbackFunc func) {
if (func == conn->read_handler) return C_OK;
conn->read_handler = func;
if (!conn->read_handler)
aeDeleteFileEvent(server.el,conn->fd,AE_READABLE);
else
if (aeCreateFileEvent(server.el,conn->fd,
AE_READABLE,conn->type->ae_handler,conn) == AE_ERR) return C_ERR;
return C_OK;
}
static const char *connSocketGetLastError(connection *conn) {
return strerror(conn->last_errno);
}
static void connSocketEventHandler(struct aeEventLoop *el, int fd, void *clientData, int mask)
{
UNUSED(el);
UNUSED(fd);
connection *conn = clientData;
if (conn->state == CONN_STATE_CONNECTING &&
(mask & AE_WRITABLE) && conn->conn_handler) {
int conn_error = anetGetError(conn->fd);
if (conn_error) {
conn->last_errno = conn_error;
conn->state = CONN_STATE_ERROR;
} else {
conn->state = CONN_STATE_CONNECTED;
}
if (!conn->write_handler) aeDeleteFileEvent(server.el,conn->fd,AE_WRITABLE);
if (!callHandler(conn, conn->conn_handler)) return;
conn->conn_handler = NULL;
}
/* Normally we execute the readable event first, and the writable
* event later. This is useful as sometimes we may be able
* to serve the reply of a query immediately after processing the
* query.
*
* However if WRITE_BARRIER is set in the mask, our application is
* asking us to do the reverse: never fire the writable event
* after the readable. In such a case, we invert the calls.
* This is useful when, for instance, we want to do things
* in the beforeSleep() hook, like fsync'ing a file to disk,
* before replying to a client. */
int invert = conn->flags & CONN_FLAG_WRITE_BARRIER;
int call_write = (mask & AE_WRITABLE) && conn->write_handler;
int call_read = (mask & AE_READABLE) && conn->read_handler;
/* Handle normal I/O flows */
if (!invert && call_read) {
if (!callHandler(conn, conn->read_handler)) return;
}
/* Fire the writable event. */
if (call_write) {
if (!callHandler(conn, conn->write_handler)) return;
}
/* If we have to invert the call, fire the readable event now
* after the writable one. */
if (invert && call_read) {
if (!callHandler(conn, conn->read_handler)) return;
}
}
static void connSocketAcceptHandler(aeEventLoop *el, int fd, void *privdata, int mask) {
int cport, cfd, max = MAX_ACCEPTS_PER_CALL;
char cip[NET_IP_STR_LEN];
UNUSED(el);
UNUSED(mask);
UNUSED(privdata);
while(max--) {
cfd = anetTcpAccept(server.neterr, fd, cip, sizeof(cip), &cport);
if (cfd == ANET_ERR) {
if (errno != EWOULDBLOCK)
serverLog(LL_WARNING,
"Accepting client connection: %s", server.neterr);
return;
}
serverLog(LL_VERBOSE,"Accepted %s:%d", cip, cport);
acceptCommonHandler(connCreateAcceptedSocket(cfd, NULL),0,cip);
}
}
static int connSocketAddr(connection *conn, char *ip, size_t ip_len, int *port, int remote) {
if (anetFdToString(conn->fd, ip, ip_len, port, remote) == 0)
return C_OK;
conn->last_errno = errno;
return C_ERR;
}
static int connSocketListen(connListener *listener) {
return listenToPort(listener);
}
static int connSocketBlockingConnect(connection *conn, const char *addr, int port, long long timeout) {
int fd = anetTcpNonBlockConnect(NULL,addr,port);
if (fd == -1) {
conn->state = CONN_STATE_ERROR;
conn->last_errno = errno;
return C_ERR;
}
if ((aeWait(fd, AE_WRITABLE, timeout) & AE_WRITABLE) == 0) {
conn->state = CONN_STATE_ERROR;
conn->last_errno = ETIMEDOUT;
}
conn->fd = fd;
conn->state = CONN_STATE_CONNECTED;
return C_OK;
}
/* Connection-based versions of syncio.c functions.
* NOTE: This should ideally be refactored out in favor of pure async work.
*/
static ssize_t connSocketSyncWrite(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncWrite(conn->fd, ptr, size, timeout);
}
static ssize_t connSocketSyncRead(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncRead(conn->fd, ptr, size, timeout);
}
static ssize_t connSocketSyncReadLine(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncReadLine(conn->fd, ptr, size, timeout);
}
static const char *connSocketGetType(connection *conn) {
(void) conn;
return CONN_TYPE_SOCKET;
}
static ConnectionType CT_Socket = {
/* connection type */
.get_type = connSocketGetType,
/* connection type initialize & finalize & configure */
.init = NULL,
.cleanup = NULL,
.configure = NULL,
/* ae & accept & listen & error & address handler */
.ae_handler = connSocketEventHandler,
.accept_handler = connSocketAcceptHandler,
.addr = connSocketAddr,
.listen = connSocketListen,
/* create/close connection */
.conn_create = connCreateSocket,
.conn_create_accepted = connCreateAcceptedSocket,
.close = connSocketClose,
/* connect & accept */
.connect = connSocketConnect,
.blocking_connect = connSocketBlockingConnect,
.accept = connSocketAccept,
/* IO */
.write = connSocketWrite,
.writev = connSocketWritev,
.read = connSocketRead,
.set_write_handler = connSocketSetWriteHandler,
.set_read_handler = connSocketSetReadHandler,
.get_last_error = connSocketGetLastError,
.sync_write = connSocketSyncWrite,
.sync_read = connSocketSyncRead,
.sync_readline = connSocketSyncReadLine,
/* pending data */
.has_pending_data = NULL,
.process_pending_data = NULL,
};
int connBlock(connection *conn) {
if (conn->fd == -1) return C_ERR;
return anetBlock(NULL, conn->fd);
}
int connNonBlock(connection *conn) {
if (conn->fd == -1) return C_ERR;
return anetNonBlock(NULL, conn->fd);
}
int connEnableTcpNoDelay(connection *conn) {
if (conn->fd == -1) return C_ERR;
return anetEnableTcpNoDelay(NULL, conn->fd);
}
int connDisableTcpNoDelay(connection *conn) {
if (conn->fd == -1) return C_ERR;
return anetDisableTcpNoDelay(NULL, conn->fd);
}
int connKeepAlive(connection *conn, int interval) {
if (conn->fd == -1) return C_ERR;
return anetKeepAlive(NULL, conn->fd, interval);
}
int connSendTimeout(connection *conn, long long ms) {
return anetSendTimeout(NULL, conn->fd, ms);
}
int connRecvTimeout(connection *conn, long long ms) {
return anetRecvTimeout(NULL, conn->fd, ms);
}
int RedisRegisterConnectionTypeSocket()
{
return connTypeRegister(&CT_Socket);
}
......@@ -27,12 +27,13 @@
* POSSIBILITY OF SUCH DAMAGE.
*/
#define REDISMODULE_CORE_MODULE /* A module that's part of the redis core, uses server.h too. */
#include "server.h"
#include "connhelpers.h"
#include "adlist.h"
#ifdef USE_OPENSSL
#if (USE_OPENSSL == 1 /* BUILD_YES */ ) || ((USE_OPENSSL == 2 /* BUILD_MODULE */) && (BUILD_TLS_MODULE == 2))
#include <openssl/conf.h>
#include <openssl/ssl.h>
......@@ -56,8 +57,6 @@
#define REDIS_TLS_PROTO_DEFAULT (REDIS_TLS_PROTO_TLSv1_2)
#endif
extern ConnectionType CT_Socket;
SSL_CTX *redis_tls_ctx = NULL;
SSL_CTX *redis_tls_client_ctx = NULL;
......@@ -141,7 +140,7 @@ static void initCryptoLocks(void) {
}
#endif /* USE_CRYPTO_LOCKS */
void tlsInit(void) {
static void tlsInit(void) {
/* Enable configuring OpenSSL using the standard openssl.cnf
* OPENSSL_config()/OPENSSL_init_crypto() should be the first
* call to the OpenSSL* library.
......@@ -169,7 +168,7 @@ void tlsInit(void) {
pending_list = listCreate();
}
void tlsCleanup(void) {
static void tlsCleanup(void) {
if (redis_tls_ctx) {
SSL_CTX_free(redis_tls_ctx);
redis_tls_ctx = NULL;
......@@ -281,12 +280,20 @@ error:
/* Attempt to configure/reconfigure TLS. This operation is atomic and will
* leave the SSL_CTX unchanged if fails.
* @priv: config of redisTLSContextConfig.
* @reconfigure: if true, ignore the previous configure; if false, only
* configure from @ctx_config if redis_tls_ctx is NULL.
*/
int tlsConfigure(redisTLSContextConfig *ctx_config) {
static int tlsConfigure(void *priv, int reconfigure) {
redisTLSContextConfig *ctx_config = (redisTLSContextConfig *)priv;
char errbuf[256];
SSL_CTX *ctx = NULL;
SSL_CTX *client_ctx = NULL;
if (!reconfigure && redis_tls_ctx) {
return C_OK;
}
if (!ctx_config->cert_file) {
serverLog(LL_WARNING, "No tls-cert-file configured!");
goto error;
......@@ -406,12 +413,6 @@ error:
return C_ERR;
}
/* Return 1 if TLS was already configured, 0 otherwise.
*/
int isTlsConfigured(void) {
return redis_tls_ctx != NULL;
}
#ifdef TLS_DEBUGGING
#define TLSCONN_DEBUG(fmt, ...) \
serverLog(LL_DEBUG, "TLSCONN: " fmt, __VA_ARGS__)
......@@ -419,7 +420,7 @@ int isTlsConfigured(void) {
#define TLSCONN_DEBUG(fmt, ...)
#endif
ConnectionType CT_TLS;
static ConnectionType CT_TLS;
/* Normal socket connections have a simple events/handler correlation.
*
......@@ -464,7 +465,7 @@ static connection *createTLSConnection(int client_side) {
return (connection *) conn;
}
connection *connCreateTLS(void) {
static connection *connCreateTLS(void) {
return createTLSConnection(1);
}
......@@ -485,7 +486,8 @@ static void updateTLSError(tls_connection *conn) {
* Callers should use connGetState() and verify the created connection
* is not in an error state.
*/
connection *connCreateAcceptedTLS(int fd, int require_auth) {
static connection *connCreateAcceptedTLS(int fd, void *priv) {
int require_auth = *(int *)priv;
tls_connection *conn = (tls_connection *) createTLSConnection(0);
conn->c.fd = fd;
conn->c.state = CONN_STATE_ACCEPTING;
......@@ -548,7 +550,7 @@ static int handleSSLReturnCode(tls_connection *conn, int ret_value, WantIOType *
return 0;
}
void registerSSLEvent(tls_connection *conn, WantIOType want) {
static void registerSSLEvent(tls_connection *conn, WantIOType want) {
int mask = aeGetFileEvents(server.el, conn->c.fd);
switch (want) {
......@@ -568,7 +570,7 @@ void registerSSLEvent(tls_connection *conn, WantIOType want) {
}
}
void updateSSLEvent(tls_connection *conn) {
static void updateSSLEvent(tls_connection *conn) {
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);
......@@ -595,7 +597,7 @@ static void tlsHandleEvent(tls_connection *conn, int mask) {
switch (conn->c.state) {
case CONN_STATE_CONNECTING:
conn_error = connGetSocketError((connection *) conn);
conn_error = anetGetError(conn->c.fd);
if (conn_error) {
conn->c.last_errno = conn_error;
conn->c.state = CONN_STATE_ERROR;
......@@ -718,6 +720,34 @@ static void tlsEventHandler(struct aeEventLoop *el, int fd, void *clientData, in
tlsHandleEvent(conn, mask);
}
static void tlsAcceptHandler(aeEventLoop *el, int fd, void *privdata, int mask) {
int cport, cfd, max = MAX_ACCEPTS_PER_CALL;
char cip[NET_IP_STR_LEN];
UNUSED(el);
UNUSED(mask);
UNUSED(privdata);
while(max--) {
cfd = anetTcpAccept(server.neterr, fd, cip, sizeof(cip), &cport);
if (cfd == ANET_ERR) {
if (errno != EWOULDBLOCK)
serverLog(LL_WARNING,
"Accepting client connection: %s", server.neterr);
return;
}
serverLog(LL_VERBOSE,"Accepted %s:%d", cip, cport);
acceptCommonHandler(connCreateAcceptedTLS(cfd, &server.tls_auth_clients),0,cip);
}
}
static int connTLSAddr(connection *conn, char *ip, size_t ip_len, int *port, int remote) {
return anetFdToString(conn->fd, ip, ip_len, port, remote);
}
static int connTLSListen(connListener *listener) {
return listenToPort(listener);
}
static void connTLSClose(connection *conn_) {
tls_connection *conn = (tls_connection *) conn_;
......@@ -738,7 +768,7 @@ static void connTLSClose(connection *conn_) {
conn->pending_list_node = NULL;
}
CT_Socket.close(conn_);
connectionTypeTcp()->close(conn_);
}
static int connTLSAccept(connection *_conn, ConnectionCallbackFunc accept_handler) {
......@@ -777,7 +807,7 @@ static int connTLSConnect(connection *conn_, const char *addr, int port, const c
ERR_clear_error();
/* Initiate Socket connection first */
if (CT_Socket.connect(conn_, addr, port, src_addr, connect_handler) == C_ERR) return C_ERR;
if (connectionTypeTcp()->connect(conn_, addr, port, src_addr, connect_handler) == C_ERR) return C_ERR;
/* Return now, once the socket is connected we'll initiate
* TLS connection from the event handler.
......@@ -905,7 +935,7 @@ static const char *connTLSGetLastError(connection *conn_) {
return NULL;
}
int connTLSSetWriteHandler(connection *conn, ConnectionCallbackFunc func, int barrier) {
static int connTLSSetWriteHandler(connection *conn, ConnectionCallbackFunc func, int barrier) {
conn->write_handler = func;
if (barrier)
conn->flags |= CONN_FLAG_WRITE_BARRIER;
......@@ -915,7 +945,7 @@ int connTLSSetWriteHandler(connection *conn, ConnectionCallbackFunc func, int ba
return C_OK;
}
int connTLSSetReadHandler(connection *conn, ConnectionCallbackFunc func) {
static int connTLSSetReadHandler(connection *conn, ConnectionCallbackFunc func) {
conn->read_handler = func;
updateSSLEvent((tls_connection *) conn);
return C_OK;
......@@ -940,7 +970,7 @@ static int connTLSBlockingConnect(connection *conn_, const char *addr, int port,
if (conn->c.state != CONN_STATE_NONE) return C_ERR;
/* Initiate socket blocking connect first */
if (CT_Socket.blocking_connect(conn_, addr, port, timeout) == C_ERR) return C_ERR;
if (connectionTypeTcp()->blocking_connect(conn_, addr, port, timeout) == C_ERR) return C_ERR;
/* Initiate TLS connection now. We set up a send/recv timeout on the socket,
* which means the specified timeout will not be enforced accurately. */
......@@ -1009,37 +1039,19 @@ exit:
return nread;
}
static int connTLSGetType(connection *conn_) {
static const char *connTLSGetType(connection *conn_) {
(void) conn_;
return CONN_TYPE_TLS;
}
ConnectionType CT_TLS = {
.ae_handler = tlsEventHandler,
.accept = connTLSAccept,
.connect = connTLSConnect,
.blocking_connect = connTLSBlockingConnect,
.read = connTLSRead,
.write = connTLSWrite,
.writev = connTLSWritev,
.close = connTLSClose,
.set_write_handler = connTLSSetWriteHandler,
.set_read_handler = connTLSSetReadHandler,
.get_last_error = connTLSGetLastError,
.sync_write = connTLSSyncWrite,
.sync_read = connTLSSyncRead,
.sync_readline = connTLSSyncReadLine,
.get_type = connTLSGetType
};
int tlsHasPendingData() {
static int tlsHasPendingData() {
if (!pending_list)
return 0;
return listLength(pending_list) > 0;
}
int tlsProcessPendingData() {
static int tlsProcessPendingData() {
listIter li;
listNode *ln;
......@@ -1055,9 +1067,9 @@ int tlsProcessPendingData() {
/* Fetch the peer certificate used for authentication on the specified
* connection and return it as a PEM-encoded sds.
*/
sds connTLSGetPeerCert(connection *conn_) {
static sds connTLSGetPeerCert(connection *conn_) {
tls_connection *conn = (tls_connection *) conn_;
if (conn_->type->get_type(conn_) != CONN_TYPE_TLS || !conn->ssl) return NULL;
if ((conn_->type != connectionTypeTls()) || !conn->ssl) return NULL;
X509 *cert = SSL_get_peer_certificate(conn->ssl);
if (!cert) return NULL;
......@@ -1076,41 +1088,97 @@ sds connTLSGetPeerCert(connection *conn_) {
return cert_pem;
}
#else /* USE_OPENSSL */
static ConnectionType CT_TLS = {
/* connection type */
.get_type = connTLSGetType,
void tlsInit(void) {
}
/* connection type initialize & finalize & configure */
.init = tlsInit,
.cleanup = tlsCleanup,
.configure = tlsConfigure,
void tlsCleanup(void) {
}
/* ae & accept & listen & error & address handler */
.ae_handler = tlsEventHandler,
.accept_handler = tlsAcceptHandler,
.addr = connTLSAddr,
.listen = connTLSListen,
int tlsConfigure(redisTLSContextConfig *ctx_config) {
UNUSED(ctx_config);
return C_OK;
}
/* create/close connection */
.conn_create = connCreateTLS,
.conn_create_accepted = connCreateAcceptedTLS,
.close = connTLSClose,
connection *connCreateTLS(void) {
return NULL;
}
/* connect & accept */
.connect = connTLSConnect,
.blocking_connect = connTLSBlockingConnect,
.accept = connTLSAccept,
connection *connCreateAcceptedTLS(int fd, int require_auth) {
UNUSED(fd);
UNUSED(require_auth);
/* IO */
.read = connTLSRead,
.write = connTLSWrite,
.writev = connTLSWritev,
.set_write_handler = connTLSSetWriteHandler,
.set_read_handler = connTLSSetReadHandler,
.get_last_error = connTLSGetLastError,
.sync_write = connTLSSyncWrite,
.sync_read = connTLSSyncRead,
.sync_readline = connTLSSyncReadLine,
return NULL;
}
/* pending data */
.has_pending_data = tlsHasPendingData,
.process_pending_data = tlsProcessPendingData,
int tlsHasPendingData() {
return 0;
/* TLS specified methods */
.get_peer_cert = connTLSGetPeerCert,
};
int RedisRegisterConnectionTypeTLS() {
return connTypeRegister(&CT_TLS);
}
int tlsProcessPendingData() {
return 0;
#else /* USE_OPENSSL */
int RedisRegisterConnectionTypeTLS() {
serverLog(LL_VERBOSE, "Connection type %s not builtin", CONN_TYPE_TLS);
return C_ERR;
}
sds connTLSGetPeerCert(connection *conn_) {
(void) conn_;
return NULL;
#endif
#if BUILD_TLS_MODULE == 2 /* BUILD_MODULE */
#include "release.h"
int RedisModule_OnLoad(void *ctx, RedisModuleString **argv, int argc) {
UNUSED(argv);
UNUSED(argc);
/* Connection modules must be part of the same build as redis. */
if (strcmp(REDIS_BUILD_ID_RAW, redisBuildIdRaw())) {
serverLog(LL_NOTICE, "Connection type %s was not built together with the redis-server used.", CONN_TYPE_TLS);
return REDISMODULE_ERR;
}
if (RedisModule_Init(ctx,"tls",1,REDISMODULE_APIVER_1) == REDISMODULE_ERR)
return REDISMODULE_ERR;
/* Connection modules is available only bootup. */
if ((RedisModule_GetContextFlags(ctx) & REDISMODULE_CTX_FLAGS_SERVER_STARTUP) == 0) {
serverLog(LL_NOTICE, "Connection type %s can be loaded only during bootup", CONN_TYPE_TLS);
return REDISMODULE_ERR;
}
RedisModule_SetModuleOptions(ctx, REDISMODULE_OPTIONS_HANDLE_REPL_ASYNC_LOAD);
if(connTypeRegister(&CT_TLS) != C_OK)
return REDISMODULE_ERR;
return REDISMODULE_OK;
}
int RedisModule_OnUnload(void *arg) {
UNUSED(arg);
serverLog(LL_NOTICE, "Connection type %s can not be unloaded", CONN_TYPE_TLS);
return REDISMODULE_ERR;
}
#endif
/* ==========================================================================
* unix.c - unix socket connection implementation
* --------------------------------------------------------------------------
* Copyright (C) 2022 zhenwei pi
*
* Permission is hereby granted, free of charge, to any person obtaining a
* copy of this software and associated documentation files (the
* "Software"), to deal in the Software without restriction, including
* without limitation the rights to use, copy, modify, merge, publish,
* distribute, sublicense, and/or sell copies of the Software, and to permit
* persons to whom the Software is furnished to do so, subject to the
* following conditions:
*
* The above copyright notice and this permission notice shall be included
* in all copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
* OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
* MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
* NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
* DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
* OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE
* USE OR OTHER DEALINGS IN THE SOFTWARE.
* ==========================================================================
*/
#include "server.h"
#include "connection.h"
static ConnectionType CT_Unix;
static const char *connUnixGetType(connection *conn) {
UNUSED(conn);
return CONN_TYPE_UNIX;
}
static void connUnixEventHandler(struct aeEventLoop *el, int fd, void *clientData, int mask) {
connectionTypeTcp()->ae_handler(el, fd, clientData, mask);
}
static int connUnixAddr(connection *conn, char *ip, size_t ip_len, int *port, int remote) {
return connectionTypeTcp()->addr(conn, ip, ip_len, port, remote);
}
static int connUnixListen(connListener *listener) {
int fd;
mode_t *perm = (mode_t *)listener->priv;
if (listener->bindaddr_count == 0)
return C_OK;
/* currently listener->bindaddr_count is always 1, we still use a loop here in case Redis supports multi Unix socket in the future */
for (int j = 0; j < listener->bindaddr_count; j++) {
char *addr = listener->bindaddr[j];
unlink(addr); /* don't care if this fails */
fd = anetUnixServer(server.neterr, addr, *perm, server.tcp_backlog);
if (fd == ANET_ERR) {
serverLog(LL_WARNING, "Failed opening Unix socket: %s", server.neterr);
exit(1);
}
anetNonBlock(NULL, fd);
anetCloexec(fd);
listener->fd[listener->count++] = fd;
}
return C_OK;
}
static connection *connCreateUnix(void) {
connection *conn = zcalloc(sizeof(connection));
conn->type = &CT_Unix;
conn->fd = -1;
return conn;
}
static connection *connCreateAcceptedUnix(int fd, void *priv) {
UNUSED(priv);
connection *conn = connCreateUnix();
conn->fd = fd;
conn->state = CONN_STATE_ACCEPTING;
return conn;
}
static void connUnixAcceptHandler(aeEventLoop *el, int fd, void *privdata, int mask) {
int cfd, max = MAX_ACCEPTS_PER_CALL;
UNUSED(el);
UNUSED(mask);
UNUSED(privdata);
while(max--) {
cfd = anetUnixAccept(server.neterr, fd);
if (cfd == ANET_ERR) {
if (errno != EWOULDBLOCK)
serverLog(LL_WARNING,
"Accepting client connection: %s", server.neterr);
return;
}
serverLog(LL_VERBOSE,"Accepted connection to %s", server.unixsocket);
acceptCommonHandler(connCreateAcceptedUnix(cfd, NULL),CLIENT_UNIX_SOCKET,NULL);
}
}
static void connUnixClose(connection *conn) {
connectionTypeTcp()->close(conn);
}
static int connUnixAccept(connection *conn, ConnectionCallbackFunc accept_handler) {
return connectionTypeTcp()->accept(conn, accept_handler);
}
static int connUnixWrite(connection *conn, const void *data, size_t data_len) {
return connectionTypeTcp()->write(conn, data, data_len);
}
static int connUnixWritev(connection *conn, const struct iovec *iov, int iovcnt) {
return connectionTypeTcp()->writev(conn, iov, iovcnt);
}
static int connUnixRead(connection *conn, void *buf, size_t buf_len) {
return connectionTypeTcp()->read(conn, buf, buf_len);
}
static int connUnixSetWriteHandler(connection *conn, ConnectionCallbackFunc func, int barrier) {
return connectionTypeTcp()->set_write_handler(conn, func, barrier);
}
static int connUnixSetReadHandler(connection *conn, ConnectionCallbackFunc func) {
return connectionTypeTcp()->set_read_handler(conn, func);
}
static const char *connUnixGetLastError(connection *conn) {
return strerror(conn->last_errno);
}
static ssize_t connUnixSyncWrite(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncWrite(conn->fd, ptr, size, timeout);
}
static ssize_t connUnixSyncRead(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncRead(conn->fd, ptr, size, timeout);
}
static ssize_t connUnixSyncReadLine(connection *conn, char *ptr, ssize_t size, long long timeout) {
return syncReadLine(conn->fd, ptr, size, timeout);
}
static ConnectionType CT_Unix = {
/* connection type */
.get_type = connUnixGetType,
/* connection type initialize & finalize & configure */
.init = NULL,
.cleanup = NULL,
.configure = NULL,
/* ae & accept & listen & error & address handler */
.ae_handler = connUnixEventHandler,
.accept_handler = connUnixAcceptHandler,
.addr = connUnixAddr,
.listen = connUnixListen,
/* create/close connection */
.conn_create = connCreateUnix,
.conn_create_accepted = connCreateAcceptedUnix,
.close = connUnixClose,
/* connect & accept */
.connect = NULL,
.blocking_connect = NULL,
.accept = connUnixAccept,
/* IO */
.write = connUnixWrite,
.writev = connUnixWritev,
.read = connUnixRead,
.set_write_handler = connUnixSetWriteHandler,
.set_read_handler = connUnixSetReadHandler,
.get_last_error = connUnixGetLastError,
.sync_write = connUnixSyncWrite,
.sync_read = connUnixSyncRead,
.sync_readline = connUnixSyncReadLine,
/* pending data */
.has_pending_data = NULL,
.process_pending_data = NULL,
};
int RedisRegisterConnectionTypeUnix()
{
return connTypeRegister(&CT_Unix);
}
......@@ -19,6 +19,7 @@ source ../support/test.tcl
set ::verbose 0
set ::valgrind 0
set ::tls 0
set ::tls_module 0
set ::pause_on_error 0
set ::dont_clean 0
set ::simulate_error 0
......@@ -85,6 +86,10 @@ proc spawn_instance {type base_port count {conf {}} {base_conf_file ""}} {
}
if {$::tls} {
if {$::tls_module} {
puts $cfg [format "loadmodule %s/../../../src/redis-tls.so" [pwd]]
}
puts $cfg "tls-port $port"
puts $cfg "tls-replication yes"
puts $cfg "tls-cluster yes"
......@@ -271,13 +276,16 @@ proc parse_options {} {
} elseif {$opt eq {--host}} {
incr j
set ::host ${val}
} elseif {$opt eq {--tls}} {
} elseif {$opt eq {--tls} || $opt eq {--tls-module}} {
package require tls 1.6
::tls::init \
-cafile "$::tlsdir/ca.crt" \
-certfile "$::tlsdir/client.crt" \
-keyfile "$::tlsdir/client.key"
set ::tls 1
if {$opt eq {--tls-module}} {
set ::tls_module 1
}
} elseif {$opt eq {--config}} {
set val2 [lindex $::argv [expr $j+2]]
dict set ::global_config $val $val2
......@@ -293,6 +301,7 @@ proc parse_options {} {
puts "--fail Simulate a test failure."
puts "--valgrind Run with valgrind."
puts "--tls Run tests in TLS mode."
puts "--tls-module Run tests in TLS mode with Redis module."
puts "--host <host> Use hostname instead of 127.0.0.1."
puts "--config <k> <v> Extra config argument(s)."
puts "--stop Blocks once the first test fails."
......
......@@ -35,7 +35,7 @@ static void createGlobalStrings(RedisModuleCtx *ctx, int count)
}
}
static int defragGlobalStrings(RedisModuleDefragCtx *ctx)
static void defragGlobalStrings(RedisModuleDefragCtx *ctx)
{
for (int i = 0; i < global_strings_len; i++) {
RedisModuleString *new = RedisModule_DefragRedisModuleString(ctx, global_strings[i]);
......@@ -45,8 +45,6 @@ static int defragGlobalStrings(RedisModuleDefragCtx *ctx)
global_defragged++;
}
}
return 0;
}
static void FragInfo(RedisModuleInfoCtx *ctx, int for_crash_report) {
......
......@@ -300,7 +300,7 @@ proc wait_server_started {config_file stdout pid} {
set maxiter [expr {120*1000/$checkperiod}] ; # Wait up to 2 minutes.
set port_busy 0
while 1 {
if {[regexp -- " PID: $pid" [exec cat $stdout]]} {
if {[regexp -- " PID: $pid.*Server initialized" [exec cat $stdout]]} {
break
}
after $checkperiod
......@@ -464,6 +464,9 @@ proc start_server {options {code undefined}} {
set data [split [exec cat "tests/assets/$baseconfig"] "\n"]
set config {}
if {$::tls} {
if {$::tls_module} {
lappend config_lines [list "loadmodule" [format "%s/src/redis-tls.so" [pwd]]]
}
dict set config "tls-cert-file" [format "%s/tests/tls/server.crt" [pwd]]
dict set config "tls-key-file" [format "%s/tests/tls/server.key" [pwd]]
dict set config "tls-client-cert-file" [format "%s/tests/tls/client.crt" [pwd]]
......
......@@ -1039,3 +1039,25 @@ proc memory_usage {key} {
}
return $usage
}
# forward compatibility, lmap missing in TCL 8.5
proc lmap args {
set body [lindex $args end]
set args [lrange $args 0 end-1]
set n 0
set pairs [list]
foreach {varnames listval} $args {
set varlist [list]
foreach varname $varnames {
upvar 1 $varname var$n
lappend varlist var$n
incr n
}
lappend pairs $varlist $listval
}
set temp [list]
foreach {*}$pairs {
lappend temp [uplevel 1 $body]
}
set temp
}
......@@ -111,6 +111,7 @@ set ::traceleaks 0
set ::valgrind 0
set ::durable 0
set ::tls 0
set ::tls_module 0
set ::stack_logging 0
set ::verbose 0
set ::quiet 0
......@@ -611,6 +612,7 @@ proc print_help_screen {} {
"--wait-server Wait after server is started (so that you can attach a debugger)."
"--dump-logs Dump server log on test failure."
"--tls Run tests in TLS mode."
"--tls-module Run tests in TLS mode with Redis module."
"--host <addr> Run tests against an external host."
"--port <port> TCP port to use against external host."
"--baseport <port> Initial port number for spawned redis servers."
......@@ -659,13 +661,16 @@ for {set j 0} {$j < [llength $argv]} {incr j} {
}
} elseif {$opt eq {--quiet}} {
set ::quiet 1
} elseif {$opt eq {--tls}} {
} elseif {$opt eq {--tls} || $opt eq {--tls-module}} {
package require tls 1.6
set ::tls 1
::tls::init \
-cafile "$::tlsdir/ca.crt" \
-certfile "$::tlsdir/client.crt" \
-keyfile "$::tlsdir/client.key"
if {$opt eq {--tls-module}} {
set ::tls_module 1
}
} elseif {$opt eq {--host}} {
set ::external 1
set ::host $arg
......
......@@ -5,18 +5,21 @@ test {modules config rewrite} {
start_server {tags {"modules"}} {
r module load $testmodule
assert_equal [lindex [lindex [r module list] 0] 1] infotest
set modules [lmap x [r module list] {dict get $x name}]
assert_not_equal [lsearch $modules infotest] -1
r config rewrite
restart_server 0 true false
assert_equal [lindex [lindex [r module list] 0] 1] infotest
set modules [lmap x [r module list] {dict get $x name}]
assert_not_equal [lsearch $modules infotest] -1
assert_equal {OK} [r module unload infotest]
r config rewrite
restart_server 0 true false
assert_equal [llength [r module list]] 0
set modules [lmap x [r module list] {dict get $x name}]
assert_equal [lsearch $modules infotest] -1
}
}
......@@ -5,7 +5,7 @@ start_server {tags {"modules"}} {
r module load $testmodule
test {Config get commands work} {
# Make sure config get module config works
assert_equal [lindex [lindex [r module list] 0] 1] moduleconfigs
assert_not_equal [lsearch [lmap x [r module list] {dict get $x name}] moduleconfigs] -1
assert_equal [r config get moduleconfigs.mutable_bool] "moduleconfigs.mutable_bool yes"
assert_equal [r config get moduleconfigs.immutable_bool] "moduleconfigs.immutable_bool no"
assert_equal [r config get moduleconfigs.memory_numeric] "moduleconfigs.memory_numeric 1024"
......@@ -94,7 +94,7 @@ start_server {tags {"modules"}} {
test {test loadex functionality} {
r module loadex $testmodule CONFIG moduleconfigs.mutable_bool no CONFIG moduleconfigs.immutable_bool yes CONFIG moduleconfigs.memory_numeric 2mb CONFIG moduleconfigs.string tclortickle
assert_equal [lindex [lindex [r module list] 0] 1] moduleconfigs
assert_not_equal [lsearch [lmap x [r module list] {dict get $x name}] moduleconfigs] -1
assert_equal [r config get moduleconfigs.mutable_bool] "moduleconfigs.mutable_bool no"
assert_equal [r config get moduleconfigs.immutable_bool] "moduleconfigs.immutable_bool yes"
assert_equal [r config get moduleconfigs.memory_numeric] "moduleconfigs.memory_numeric 2097152"
......@@ -157,7 +157,7 @@ start_server {tags {"modules"}} {
test {test config rewrite with dynamic load} {
#translates to: super \0secret password
r module loadex $testmodule CONFIG moduleconfigs.string \x73\x75\x70\x65\x72\x20\x00\x73\x65\x63\x72\x65\x74\x20\x70\x61\x73\x73\x77\x6f\x72\x64 ARGS
assert_equal [lindex [lindex [r module list] 0] 1] moduleconfigs
assert_not_equal [lsearch [lmap x [r module list] {dict get $x name}] moduleconfigs] -1
assert_equal [r config get moduleconfigs.string] "moduleconfigs.string {super \0secret password}"
r config set moduleconfigs.mutable_bool yes
r config set moduleconfigs.memory_numeric 750
......@@ -207,7 +207,7 @@ start_server {tags {"modules"}} {
test {test 1.module load 2.config rewrite 3.module unload 4.config rewrite works} {
# Configs need to be removed from the old config file in this case.
r module loadex $testmodule CONFIG moduleconfigs.memory_numeric 500 ARGS
assert_equal [lindex [lindex [r module list] 0] 1] moduleconfigs
assert_not_equal [lsearch [lmap x [r module list] {dict get $x name}] moduleconfigs] -1
r config rewrite
r module unload moduleconfigs
r config rewrite
......@@ -217,34 +217,18 @@ start_server {tags {"modules"}} {
}
test {startup moduleconfigs} {
# No loadmodule directive
set nomodload [start_server [list overrides [list moduleconfigs.string "hello"]]]
wait_for_condition 100 50 {
! [is_alive $nomodload]
} else {
fail "startup should've failed with no load and module configs supplied"
}
set stdout [dict get $nomodload stdout]
assert_equal [count_message_lines $stdout "Module Configuration detected without loadmodule directive or no ApplyConfig call: aborting"] 1
catch {exec src/redis-server --moduleconfigs.string "hello"} err
assert_match {*Module Configuration detected without loadmodule directive or no ApplyConfig call: aborting*} $err
# Bad config value
set badconfig [start_server [list overrides [list loadmodule "$testmodule" moduleconfigs.string "rejectisfreed"]]]
wait_for_condition 100 50 {
! [is_alive $badconfig]
} else {
fail "startup with bad moduleconfigs should've failed"
}
set stdout [dict get $badconfig stdout]
assert_equal [count_message_lines $stdout "Issue during loading of configuration moduleconfigs.string : Cannot set string to 'rejectisfreed'"] 1
catch {exec src/redis-server --loadmodule "$testmodule" --moduleconfigs.string "rejectisfreed"} err
assert_match {*Issue during loading of configuration moduleconfigs.string : Cannot set string to 'rejectisfreed'*} $err
set noload [start_server [list overrides [list loadmodule "$testmodule noload" moduleconfigs.string "hello"]]]
wait_for_condition 100 50 {
! [is_alive $noload]
} else {
fail "startup with moduleconfigs and no loadconfigs call should've failed"
}
set stdout [dict get $noload stdout]
assert_equal [count_message_lines $stdout "Module Configurations were not set, likely a missing LoadConfigs call. Unloading the module."] 1
# missing LoadConfigs call
catch {exec src/redis-server --loadmodule "$testmodule" noload --moduleconfigs.string "hello"} err
assert_match {*Module Configurations were not set, likely a missing LoadConfigs call. Unloading the module.*} $err
# successful
start_server [list overrides [list loadmodule "$testmodule" moduleconfigs.string "bootedup" moduleconfigs.enum two moduleconfigs.flags "two four"]] {
assert_equal [r config get moduleconfigs.string] "moduleconfigs.string bootedup"
assert_equal [r config get moduleconfigs.mutable_bool] "moduleconfigs.mutable_bool yes"
......
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment