Add sanitizer support and clean up sanitizer findings (#9601)

- Added sanitizer support. `address`, `undefined` and `thread` sanitizers are available.  
- To build Redis with desired sanitizer : `make SANITIZER=undefined`
- There were some sanitizer findings, cleaned up codebase
- Added tests with address and undefined behavior sanitizers to daily CI.
- Added tests with address sanitizer to the per-PR CI (smoke out mem leaks sooner).

Basically, there are three types of issues : 

**1- Unaligned load/store** : Most probably, this issue may cause a crash on a platform that
does not support unaligned access. Redis does unaligned access only on supported platforms.

**2- Signed integer overflow.** Although, signed overflow issue can be problematic time to time
and change how compiler generates code, current findings mostly about signed shift or simple
addition overflow. For most platforms Redis can be compiled for, this wouldn't cause any issue
as far as I can tell (checked generated code on godbolt.org).

 **3 -Minor leak** (redis-cli), **use-after-free**(just before calling exit());

UB means nothing guaranteed and risky to reason about program behavior but I don't think any
of the fixes here worth backporting. As sanitizers are now part of the CI, preventing new issues
will be the real benefit.
This commit is contained in:
Ozan Tezcan 2021-11-11 14:51:33 +03:00 committed by GitHub
parent cd6b3d558b
commit b91d8b289b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 326 additions and 54 deletions

View File

@ -19,6 +19,19 @@ jobs:
- name: module api test
run: ./runtest-moduleapi --verbose
test-sanitizer-address:
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v2
- name: make
run: make SANITIZER=address
- name: testprep
run: sudo apt-get install tcl8.6 tclx -y
- name: test
run: ./runtest --verbose --tags -slow
- name: module api test
run: ./runtest-moduleapi --verbose
build-debian-old:
runs-on: ubuntu-latest
container: debian:oldoldstable

View File

@ -11,7 +11,7 @@ on:
inputs:
skipjobs:
description: 'jobs to skip (delete the ones you wanna keep, do not leave empty)'
default: 'valgrind,tls,freebsd,macos,alpine,32bit'
default: 'valgrind,sanitizer,tls,freebsd,macos,alpine,32bit'
skiptests:
description: 'tests to skip (delete the ones you wanna keep, do not leave empty)'
default: 'redis,modules,sentinel,cluster'
@ -290,6 +290,86 @@ jobs:
if: true && !contains(github.event.inputs.skiptests, 'modules')
run: ./runtest-moduleapi --valgrind --no-latency --verbose --clients 1 --timeout 2400 --dump-logs ${{github.event.inputs.test_args}}
test-sanitizer-address:
runs-on: ubuntu-latest
if: github.repository == 'redis/redis' && !contains(github.event.inputs.skipjobs, 'sanitizer')
timeout-minutes: 14400
strategy:
matrix:
compiler: [ gcc, clang ]
env:
CC: ${{ matrix.compiler }}
steps:
- name: prep
if: github.event_name == 'workflow_dispatch'
run: |
echo "GITHUB_REPOSITORY=${{github.event.inputs.use_repo}}" >> $GITHUB_ENV
echo "GITHUB_HEAD_REF=${{github.event.inputs.use_git_ref}}" >> $GITHUB_ENV
- uses: actions/checkout@v2
with:
repository: ${{ env.GITHUB_REPOSITORY }}
ref: ${{ env.GITHUB_HEAD_REF }}
- name: make
run: make SANITIZER=address REDIS_CFLAGS='-DREDIS_TEST'
- name: testprep
run: |
sudo apt-get update
sudo apt-get install tcl8.6 tclx -y
- name: test
if: true && !contains(github.event.inputs.skiptests, 'redis')
run: ./runtest --accurate --verbose --dump-logs ${{github.event.inputs.test_args}}
- name: module api test
if: true && !contains(github.event.inputs.skiptests, 'modules')
run: ./runtest-moduleapi --verbose ${{github.event.inputs.test_args}}
- name: sentinel tests
if: true && !contains(github.event.inputs.skiptests, 'sentinel')
run: ./runtest-sentinel ${{github.event.inputs.cluster_test_args}}
- name: cluster tests
if: true && !contains(github.event.inputs.skiptests, 'cluster')
run: ./runtest-cluster ${{github.event.inputs.cluster_test_args}}
- name: unittest
run: ./src/redis-server test all
test-sanitizer-undefined:
runs-on: ubuntu-latest
if: github.repository == 'redis/redis' && !contains(github.event.inputs.skipjobs, 'sanitizer')
timeout-minutes: 14400
strategy:
matrix:
compiler: [ gcc, clang ]
env:
CC: ${{ matrix.compiler }}
steps:
- name: prep
if: github.event_name == 'workflow_dispatch'
run: |
echo "GITHUB_REPOSITORY=${{github.event.inputs.use_repo}}" >> $GITHUB_ENV
echo "GITHUB_HEAD_REF=${{github.event.inputs.use_git_ref}}" >> $GITHUB_ENV
- uses: actions/checkout@v2
with:
repository: ${{ env.GITHUB_REPOSITORY }}
ref: ${{ env.GITHUB_HEAD_REF }}
- name: make
run: make SANITIZER=undefined REDIS_CFLAGS='-DREDIS_TEST'
- name: testprep
run: |
sudo apt-get update
sudo apt-get install tcl8.6 tclx -y
- name: test
if: true && !contains(github.event.inputs.skiptests, 'redis')
run: ./runtest --accurate --verbose --dump-logs ${{github.event.inputs.test_args}}
- name: module api test
if: true && !contains(github.event.inputs.skiptests, 'modules')
run: ./runtest-moduleapi --verbose ${{github.event.inputs.test_args}}
- name: sentinel tests
if: true && !contains(github.event.inputs.skiptests, 'sentinel')
run: ./runtest-sentinel ${{github.event.inputs.cluster_test_args}}
- name: cluster tests
if: true && !contains(github.event.inputs.skiptests, 'cluster')
run: ./runtest-cluster ${{github.event.inputs.cluster_test_args}}
- name: unittest
run: ./src/redis-server test all
test-centos7-jemalloc:
runs-on: ubuntu-latest
if: github.repository == 'redis/redis'

View File

@ -85,6 +85,27 @@ ifeq ($(USE_JEMALLOC),no)
MALLOC=libc
endif
ifdef SANITIZER
ifeq ($(SANITIZER),address)
MALLOC=libc
CFLAGS+=-fsanitize=address -fno-sanitize-recover=all -fno-omit-frame-pointer
LDFLAGS+=-fsanitize=address
else
ifeq ($(SANITIZER),undefined)
MALLOC=libc
CFLAGS+=-fsanitize=undefined -fno-sanitize-recover=all -fno-omit-frame-pointer
LDFLAGS+=-fsanitize=undefined
else
ifeq ($(SANITIZER),thread)
CFLAGS+=-fsanitize=thread -fno-sanitize-recover=all -fno-omit-frame-pointer
LDFLAGS+=-fsanitize=thread
else
$(error "unknown sanitizer=${SANITIZER}")
endif
endif
endif
endif
# Override default settings if possible
-include .make-settings

View File

@ -145,7 +145,11 @@ void aofChildWriteDiffData(aeEventLoop *el, int fd, void *privdata, int mask) {
latencyAddSampleIfNeeded("aof-rewrite-write-data-to-child",latency);
}
/* Append data to the AOF rewrite buffer, allocating new blocks if needed. */
/* Append data to the AOF rewrite buffer, allocating new blocks if needed.
*
* Sanitizer suppression: zmalloc_usable() confuses sanitizer, it generates
* a false positive out-of-bounds error */
REDIS_NO_SANITIZE("bounds")
void aofRewriteBufferAppend(unsigned char *s, unsigned long len) {
listNode *ln = listLast(server.aof_rewrite_buf_blocks);
aofrwblock *block = ln ? ln->value : NULL;

View File

@ -328,8 +328,9 @@ int checkSignedBitfieldOverflow(int64_t value, int64_t incr, uint64_t bits, int
/* Note that maxincr and minincr could overflow, but we use the values
* only after checking 'value' range, so when we use it no overflow
* happens. */
int64_t maxincr = max-value;
* happens. 'uint64_t' cast is there just to prevent undefined behavior on
* overflow */
int64_t maxincr = (uint64_t)max-value;
int64_t minincr = min-value;
if (value > max || (bits != 64 && incr > maxincr) || (value >= 0 && incr > 0 && incr > maxincr))
@ -600,6 +601,7 @@ void getbitCommand(client *c) {
}
/* BITOP op_name target_key src_key1 src_key2 src_key3 ... src_keyN */
REDIS_NO_SANITIZE("alignment")
void bitopCommand(client *c) {
char *opname = c->argv[1]->ptr;
robj *o, *targetkey = c->argv[2];
@ -682,7 +684,6 @@ void bitopCommand(client *c) {
unsigned long *lp[16];
unsigned long *lres = (unsigned long*) res;
/* Note: sds pointer is always aligned to 8 byte boundary. */
memcpy(lp,src,sizeof(unsigned long*)*numkeys);
memcpy(res,src[0],minlen);
@ -1152,10 +1153,9 @@ void bitfieldGeneric(client *c, int flags) {
thisop->bits);
if (thisop->opcode == BITFIELDOP_INCRBY) {
newval = oldval + thisop->i64;
overflow = checkSignedBitfieldOverflow(oldval,
thisop->i64,thisop->bits,thisop->owtype,&wrapped);
if (overflow) newval = wrapped;
newval = overflow ? wrapped : oldval + thisop->i64;
retval = newval;
} else {
newval = thisop->i64;

View File

@ -356,6 +356,7 @@ void parseRedisUri(const char *uri, const char* tool_name, cliConnInfo *connInfo
connInfo->hostport = atoi(port + 1);
host = port - 1;
}
sdsfree(connInfo->hostip);
connInfo->hostip = sdsnewlen(curr, host - curr + 1);
}
curr = path ? path + 1 : end;

View File

@ -2781,7 +2781,13 @@ void clusterBroadcastPong(int target) {
/* Send a PUBLISH message.
*
* If link is NULL, then the message is broadcasted to the whole cluster. */
* If link is NULL, then the message is broadcasted to the whole cluster.
*
* Sanitizer suppression: In clusterMsgDataPublish, sizeof(bulk_data) is 8.
* As all the struct is used as a buffer, when more than 8 bytes are copied into
* the 'bulk_data', sanitizer generates an out-of-bounds error which is a false
* positive in this context. */
REDIS_NO_SANITIZE("bounds")
void clusterSendPublish(clusterLink *link, robj *channel, robj *message) {
unsigned char *payload;
clusterMsg buf[1];

View File

@ -453,6 +453,7 @@ void initConfigValues() {
}
void loadServerConfigFromString(char *config) {
char buf[1024];
const char *err = NULL;
int linenum = 0, totlines, i;
sds *lines;
@ -544,7 +545,6 @@ void loadServerConfigFromString(char *config) {
} else if (!strcasecmp(argv[0],"user") && argc >= 2) {
int argc_err;
if (ACLAppendUserForLoading(argv,argc,&argc_err) == C_ERR) {
char buf[1024];
const char *errmsg = ACLSetUserStringError();
snprintf(buf,sizeof(buf),"Error in user declaration '%s': %s",
argv[argc_err],errmsg);

View File

@ -120,6 +120,15 @@
#define unlikely(x) (x)
#endif
#if defined(__has_attribute)
#if __has_attribute(no_sanitize)
#define REDIS_NO_SANITIZE(sanitizer) __attribute__((no_sanitize(sanitizer)))
#endif
#endif
#if !defined(REDIS_NO_SANITIZE)
#define REDIS_NO_SANITIZE(sanitizer)
#endif
/* Define rdb_fsync_range to sync_file_range() on Linux, otherwise we use
* the plain fsync() call. */
#if (defined(__linux__) && defined(SYNC_FILE_RANGE_WAIT_BEFORE))

View File

@ -39,6 +39,7 @@
#include <signal.h>
#include <dlfcn.h>
#include <fcntl.h>
#include <sys/mman.h>
#include <unistd.h>
#ifdef HAVE_BACKTRACE
@ -485,7 +486,11 @@ NULL
};
addReplyHelp(c, help);
} else if (!strcasecmp(c->argv[1]->ptr,"segfault")) {
*((char*)-1) = 'x';
/* Compiler gives warnings about writing to a random address
* e.g "*((char*)-1) = 'x';". As a workaround, we map a read-only area
* and try to write there to trigger segmentation fault. */
char* p = mmap(NULL, 4096, PROT_READ, MAP_PRIVATE | MAP_ANON, -1, 0);
*p = 'x';
} else if (!strcasecmp(c->argv[1]->ptr,"panic")) {
serverPanic("DEBUG PANIC called at Unix time %lld", (long long)time(NULL));
} else if (!strcasecmp(c->argv[1]->ptr,"restart") ||
@ -1154,6 +1159,7 @@ static void *getMcontextEip(ucontext_t *uc) {
#undef NOT_SUPPORTED
}
REDIS_NO_SANITIZE("address")
void logStackContent(void **sp) {
int i;
for (i = 15; i >= 0; i--) {
@ -1856,7 +1862,9 @@ void dumpX86Calls(void *addr, size_t len) {
for (j = 0; j < len-4; j++) {
if (p[j] != 0xE8) continue; /* Not an E8 CALL opcode. */
unsigned long target = (unsigned long)addr+j+5;
target += *((int32_t*)(p+j+1));
uint32_t tmp;
memcpy(&tmp, p+j+1, sizeof(tmp));
target += tmp;
if (dladdr((void*)target, &info) != 0 && info.dli_sname != NULL) {
if (ht[target&0xff] != target) {
printf("Function at 0x%lx is %s\n",target,info.dli_sname);

View File

@ -541,8 +541,8 @@ void *dictFetchValue(dict *d, const void *key) {
* the fingerprint again when the iterator is released.
* If the two fingerprints are different it means that the user of the iterator
* performed forbidden operations against the dictionary while iterating. */
long long dictFingerprint(dict *d) {
long long integers[6], hash = 0;
unsigned long long dictFingerprint(dict *d) {
unsigned long long integers[6], hash = 0;
int j;
integers[0] = (long) d->ht_table[0];

View File

@ -99,7 +99,7 @@ typedef struct dictIterator {
int table, safe;
dictEntry *entry, *nextEntry;
/* unsafe iterator fingerprint for misuse detection. */
long long fingerprint;
unsigned long long fingerprint;
} dictIterator;
typedef void (dictScanFunction)(void *privdata, const dictEntry *de);

View File

@ -552,15 +552,23 @@ void expireGenericCommand(client *c, long long basetime, int unit) {
if (getLongLongFromObjectOrReply(c, param, &when, NULL) != C_OK)
return;
int negative_when = when < 0;
if (unit == UNIT_SECONDS) when *= 1000;
when += basetime;
if (((when < 0) && !negative_when) || ((when-basetime > 0) && negative_when)) {
/* EXPIRE allows negative numbers, but we can at least detect an
* overflow by either unit conversion or basetime addition. */
/* EXPIRE allows negative numbers, but we can at least detect an
* overflow by either unit conversion or basetime addition. */
if (unit == UNIT_SECONDS) {
if (when > LLONG_MAX / 1000 || when < LLONG_MIN / 1000) {
addReplyErrorFormat(c, "invalid expire time in %s", c->cmd->name);
return;
}
when *= 1000;
}
if (when > LLONG_MAX - basetime) {
addReplyErrorFormat(c, "invalid expire time in %s", c->cmd->name);
return;
}
when += basetime;
/* No key, return zero. */
if (lookupKeyWrite(c->db,key) == NULL) {
addReply(c,shared.czero);

View File

@ -393,6 +393,7 @@ static char *invalid_hll_err = "-INVALIDOBJ Corrupted HLL object detected";
/* Our hash function is MurmurHash2, 64 bit version.
* It was modified for Redis in order to provide the same result in
* big and little endian archs (endian neutral). */
REDIS_NO_SANITIZE("alignment")
uint64_t MurmurHash64A (const void * key, int len, unsigned int seed) {
const uint64_t m = 0xc6a4a7935bd1e995;
const int r = 47;

View File

@ -86,6 +86,16 @@
#define expect_false(expr) expect ((expr) != 0, 0)
#define expect_true(expr) expect ((expr) != 0, 1)
#if defined(__has_attribute)
# if __has_attribute(no_sanitize)
# define NO_SANITIZE(sanitizer) __attribute__((no_sanitize(sanitizer)))
# endif
#endif
#if !defined(NO_SANITIZE)
# define NO_SANITIZE(sanitizer)
#endif
/*
* compressed format
*
@ -94,7 +104,7 @@
* 111ooooo LLLLLLLL oooooooo ; backref L+8 octets, o+1=1..4096 offset
*
*/
NO_SANITIZE("alignment")
unsigned int
lzf_compress (const void *const in_data, unsigned int in_len,
void *out_data, unsigned int out_len

View File

@ -56,6 +56,16 @@
#define ULONG_ZEROONE 0x5555555555555555UL
#endif
#if defined(__has_attribute)
#if __has_attribute(no_sanitize)
#define NO_SANITIZE(sanitizer) __attribute__((no_sanitize(sanitizer)))
#endif
#endif
#if !defined(NO_SANITIZE)
#define NO_SANITIZE(sanitizer)
#endif
static struct winsize ws;
size_t progress_printed; /* Printed chars in screen-wide progress bar. */
size_t progress_full; /* How many chars to write to fill the progress bar. */
@ -277,6 +287,8 @@ int memtest_test(unsigned long *m, size_t bytes, int passes, int interactive) {
* end of the region between fill and compare cycles in order to trash
* the cache. */
#define MEMTEST_DECACHE_SIZE (1024*8)
NO_SANITIZE("undefined")
int memtest_preserving_test(unsigned long *m, size_t bytes, int passes) {
unsigned long backup[MEMTEST_BACKUP_WORDS];
unsigned long *p = m;

View File

@ -294,7 +294,12 @@ int prepareClientToWrite(client *c) {
* -------------------------------------------------------------------------- */
/* Attempts to add the reply to the static buffer in the client struct.
* Returns the length of data that is added to the reply buffer. */
* Returns the length of data that is added to the reply buffer.
*
* Sanitizer suppression: client->buf_usable_size determined by
* zmalloc_usable_size() call. Writing beyond client->buf boundaries confuses
* sanitizer and generates a false positive out-of-bounds error */
REDIS_NO_SANITIZE("bounds")
size_t _addReplyToBuffer(client *c, const char *s, size_t len) {
size_t available = c->buf_usable_size - c->bufpos;

View File

@ -2006,21 +2006,24 @@ int quicklistTest(int argc, char *argv[], int accurate) {
}
TEST("Comprassion Plain node") {
char buf[256];
quicklistisSetPackedThreshold(1);
quicklist *ql = quicklistNew(-2, 1);
for (int i = 0; i < 500; i++)
for (int i = 0; i < 500; i++) {
/* Set to 256 to allow the node to be triggered to compress,
* if it is less than 48(nocompress), the test will be successful. */
quicklistPushHead(ql, genstr("hello", i), 256);
snprintf(buf, sizeof(buf), "hello%d", i);
quicklistPushHead(ql, buf, 256);
}
quicklistIter *iter = quicklistGetIterator(ql, AL_START_TAIL);
quicklistEntry entry;
int i = 0;
while (quicklistNext(iter, &entry)) {
char *h = genstr("hello", i);
if (strcmp((char *)entry.value, h))
snprintf(buf, sizeof(buf), "hello%d", i);
if (strcmp((char *)entry.value, buf))
ERR("value [%s] didn't match [%s] at position %d",
entry.value, h, i);
entry.value, buf, i);
i++;
}
quicklistReleaseIterator(iter);

View File

@ -1270,6 +1270,7 @@ void raxStart(raxIterator *it, rax *rt) {
* is a low level function used to implement the iterator, not callable by
* the user. Returns 0 on out of memory, otherwise 1 is returned. */
int raxIteratorAddChars(raxIterator *it, unsigned char *s, size_t len) {
if (len == 0) return 1;
if (it->key_max < it->key_len+len) {
unsigned char *old = (it->key == it->key_static_string) ? NULL :
it->key;

View File

@ -292,12 +292,16 @@ void *rdbLoadIntegerObject(rio *rdb, int enctype, int flags, size_t *lenptr) {
} else if (enctype == RDB_ENC_INT16) {
uint16_t v;
if (rioRead(rdb,enc,2) == 0) return NULL;
v = enc[0]|(enc[1]<<8);
v = ((uint32_t)enc[0])|
((uint32_t)enc[1]<<8);
val = (int16_t)v;
} else if (enctype == RDB_ENC_INT32) {
uint32_t v;
if (rioRead(rdb,enc,4) == 0) return NULL;
v = enc[0]|(enc[1]<<8)|(enc[2]<<16)|(enc[3]<<24);
v = ((uint32_t)enc[0])|
((uint32_t)enc[1]<<8)|
((uint32_t)enc[2]<<16)|
((uint32_t)enc[3]<<24);
val = (int32_t)v;
} else {
rdbReportCorruptRDB("Unknown RDB integer encoding type %d",enctype);

View File

@ -37,11 +37,12 @@
#include <stdio.h>
#include <stdlib.h>
#include <signal.h>
void _serverAssert(const char *estr, const char *file, int line) {
fprintf(stderr, "=== ASSERTION FAILED ===");
fprintf(stderr, "==> %s:%d '%s' is not true",file,line,estr);
*((char*)-1) = 'x';
raise(SIGSEGV);
}
void _serverPanic(const char *file, int line, const char *msg, ...) {

View File

@ -4916,7 +4916,7 @@ void sentinelFailoverWaitStart(sentinelRedisInstance *ri) {
/* If I'm not the leader, and it is not a forced failover via
* SENTINEL FAILOVER, then I can't continue with the failover. */
if (!isleader && !(ri->flags & SRI_FORCE_FAILOVER)) {
int election_timeout = sentinel_election_timeout;
mstime_t election_timeout = sentinel_election_timeout;
/* The election timeout is the MIN between SENTINEL_ELECTION_TIMEOUT
* and the configured failover timeout. */

View File

@ -45,8 +45,13 @@ void sha256_transform(SHA256_CTX *ctx, const BYTE data[])
{
WORD a, b, c, d, e, f, g, h, i, j, t1, t2, m[64];
for (i = 0, j = 0; i < 16; ++i, j += 4)
m[i] = (data[j] << 24) | (data[j + 1] << 16) | (data[j + 2] << 8) | (data[j + 3]);
for (i = 0, j = 0; i < 16; ++i, j += 4) {
m[i] = ((WORD) data[j + 0] << 24) |
((WORD) data[j + 1] << 16) |
((WORD) data[j + 2] << 8) |
((WORD) data[j + 3]);
}
for ( ; i < 64; ++i)
m[i] = SIG1(m[i - 2]) + m[i - 7] + SIG0(m[i - 15]) + m[i - 16];

View File

@ -55,6 +55,16 @@ int siptlw(int c) {
}
}
#if defined(__has_attribute)
#if __has_attribute(no_sanitize)
#define NO_SANITIZE(sanitizer) __attribute__((no_sanitize(sanitizer)))
#endif
#endif
#if !defined(NO_SANITIZE)
#define NO_SANITIZE(sanitizer)
#endif
/* Test of the CPU is Little Endian and supports not aligned accesses.
* Two interesting conditions to speedup the function that happen to be
* in most of x86 servers. */
@ -113,6 +123,7 @@ int siptlw(int c) {
v2 = ROTL(v2, 32); \
} while (0)
NO_SANITIZE("alignment")
uint64_t siphash(const uint8_t *in, const size_t inlen, const uint8_t *k) {
#ifndef UNALIGNED_LE_CPU
uint64_t hash;
@ -172,6 +183,7 @@ uint64_t siphash(const uint8_t *in, const size_t inlen, const uint8_t *k) {
#endif
}
NO_SANITIZE("alignment")
uint64_t siphash_nocase(const uint8_t *in, const size_t inlen, const uint8_t *k)
{
#ifndef UNALIGNED_LE_CPU

View File

@ -416,10 +416,10 @@ unsigned int zipStoreEntryEncoding(unsigned char *p, unsigned char encoding, uns
(len) = (((ptr)[0] & 0x3f) << 8) | (ptr)[1]; \
} else if ((encoding) == ZIP_STR_32B) { \
(lensize) = 5; \
(len) = ((ptr)[1] << 24) | \
((ptr)[2] << 16) | \
((ptr)[3] << 8) | \
((ptr)[4]); \
(len) = ((uint32_t)(ptr)[1] << 24) | \
((uint32_t)(ptr)[2] << 16) | \
((uint32_t)(ptr)[3] << 8) | \
((uint32_t)(ptr)[4]); \
} else { \
(lensize) = 0; /* bad encoding, should be covered by a previous */ \
(len) = 0; /* ZIP_ASSERT_ENCODING / zipEncodingLenSize, or */ \
@ -557,7 +557,7 @@ void zipSaveInteger(unsigned char *p, int64_t value, unsigned char encoding) {
memcpy(p,&i16,sizeof(i16));
memrev16ifbe(p);
} else if (encoding == ZIP_INT_24B) {
i32 = value<<8;
i32 = ((uint64_t)value)<<8;
memrev32ifbe(&i32);
memcpy(p,((uint8_t*)&i32)+1,sizeof(i32)-sizeof(uint8_t));
} else if (encoding == ZIP_INT_32B) {

View File

@ -181,6 +181,15 @@ proc log_crashes {} {
incr ::failed
}
}
set logs [glob */err.txt]
foreach log $logs {
set res [sanitizer_errors_from_file $log]
if {$res != ""} {
puts $res
incr ::failed
}
}
}
proc is_alive pid {

View File

@ -1,20 +1,20 @@
tags {"external:skip"} {
set system_name [string tolower [exec uname -s]]
set system_supported 0
set backtrace_supported 0
# We only support darwin or Linux with glibc
if {$system_name eq {darwin}} {
set system_supported 1
set backtrace_supported 1
} elseif {$system_name eq {linux}} {
# Avoid the test on libmusl, which does not support backtrace
set ldd [exec ldd src/redis-server]
if {![string match {*libc.musl*} $ldd]} {
set system_supported 1
set backtrace_supported 1
}
}
if {$system_supported} {
if {$backtrace_supported} {
set server_path [tmpdir server.log]
start_server [list overrides [list dir $server_path]] {
test "Server is able to generate a stack trace on selected systems" {
@ -35,21 +35,34 @@ if {$system_supported} {
}
}
}
}
# Valgrind will complain that the process terminated by a signal, skip it.
if {!$::valgrind} {
set server_path [tmpdir server1.log]
start_server [list overrides [list dir $server_path]] {
test "Crash report generated on SIGABRT" {
set pid [s process_id]
exec kill -SIGABRT $pid
set pattern "*STACK TRACE*"
set result [exec tail -1000 < [srv 0 stdout]]
assert {[string match $pattern $result]}
}
# Valgrind will complain that the process terminated by a signal, skip it.
if {!$::valgrind} {
if {$backtrace_supported} {
set crash_pattern "*STACK TRACE*"
} else {
set crash_pattern "*crashed by signal*"
}
set server_path [tmpdir server1.log]
start_server [list overrides [list dir $server_path]] {
test "Crash report generated on SIGABRT" {
set pid [s process_id]
exec kill -SIGABRT $pid
set result [exec tail -1000 < [srv 0 stdout]]
assert {[string match $crash_pattern $result]}
}
}
set server_path [tmpdir server2.log]
start_server [list overrides [list dir $server_path]] {
test "Crash report generated on DEBUG SEGFAULT" {
catch {r debug segfault}
set result [exec tail -1000 < [srv 0 stdout]]
assert {[string match $crash_pattern $result]}
}
}
}
}

View File

@ -19,6 +19,13 @@ proc check_valgrind_errors stderr {
}
}
proc check_sanitizer_errors stderr {
set res [sanitizer_errors_from_file $stderr]
if {$res != ""} {
send_data_packet $::test_server_fd err "Sanitizer error: $res\n"
}
}
proc clean_persistence config {
# we may wanna keep the logs for later, but let's clean the persistence
# files right away, since they can accumulate and take up a lot of space
@ -44,6 +51,8 @@ proc kill_server config {
if {$::valgrind} {
check_valgrind_errors [dict get $config stderr]
}
check_sanitizer_errors [dict get $config stderr]
return
}
set pid [dict get $config pid]
@ -104,6 +113,8 @@ proc kill_server config {
check_valgrind_errors [dict get $config stderr]
}
check_sanitizer_errors [dict get $config stderr]
# Remove this pid from the set of active pids in the test server.
send_data_packet $::test_server_fd server-killed $pid
}
@ -251,7 +262,10 @@ proc spawn_server {config_file stdout stderr} {
} elseif ($::stack_logging) {
set pid [exec /usr/bin/env MallocStackLogging=1 MallocLogFile=/tmp/malloc_log.txt src/redis-server $config_file >> $stdout 2>> $stderr &]
} else {
set pid [exec src/redis-server $config_file >> $stdout 2>> $stderr &]
# ASAN_OPTIONS environment variable is for address sanitizer. If a test
# tries to allocate huge memory area and expects allocator to return
# NULL, address sanitizer throws an error without this setting.
set pid [exec /usr/bin/env ASAN_OPTIONS=allocator_may_return_null=1 src/redis-server $config_file >> $stdout 2>> $stderr &]
}
if {$::wait_server} {
@ -299,6 +313,10 @@ proc dump_server_log {srv} {
puts "\n===== Start of server log (pid $pid) =====\n"
puts [exec cat [dict get $srv "stdout"]]
puts "===== End of server log (pid $pid) =====\n"
puts "\n===== Start of server stderr log (pid $pid) =====\n"
puts [exec cat [dict get $srv "stderr"]]
puts "===== End of server stderr log (pid $pid) =====\n"
}
proc run_external_server_test {code overrides} {
@ -599,6 +617,13 @@ proc start_server {options {code undefined}} {
puts "$crashlog"
puts ""
}
set sanitizerlog [sanitizer_errors_from_file [dict get $srv "stderr"]]
if {[string length $sanitizerlog] > 0} {
puts [format "\nLogged sanitizer errors (pid %d):" [dict get $srv "pid"]]
puts "$sanitizerlog"
puts ""
}
}
if {!$assertion && $::durable} {

View File

@ -50,6 +50,27 @@ proc crashlog_from_file {filename} {
join $result "\n"
}
# Return sanitizer log lines
proc sanitizer_errors_from_file {filename} {
set log [exec cat $filename]
set lines [split [exec cat $filename] "\n"]
foreach line $lines {
# Ignore huge allocation warnings
if ([string match {*WARNING: AddressSanitizer failed to allocate*} $line]) {
continue
}
# GCC UBSAN output does not contain 'Sanitizer' but 'runtime error'.
if {[string match {*runtime error*} $log] ||
[string match {*Sanitizer*} $log]} {
return $log
}
}
return ""
}
proc getInfoProperty {infostr property} {
if {[regexp "\r\n$property:(.*?)\r\n" $infostr _ value]} {
set _ $value