Merge branch 'mdb-master' at 2015-01-01.

This commit is contained in:
Leo Yuriev 2015-01-07 18:46:52 +03:00
commit 37a8a96733
3 changed files with 327 additions and 164 deletions

27
lmdb.h
View File

@ -49,11 +49,15 @@
* stale locks can block further operation.
*
* Fix: Check for stale readers periodically, using the
* #mdb_reader_check function or the \ref mdb_stat_1 "mdb_stat" tool. Or just
* make all programs using the database close it; the lockfile
* is always reset on first open of the environment.
* #mdb_reader_check function or the \ref mdb_stat_1 "mdb_stat" tool.
* Stale writers will be cleared automatically on most systems:
* - Windows - automatic
* - BSD, systems using SysV semaphores - automatic
* - Linux, systems using POSIX mutexes with Robust option - automatic
* Otherwise just make all programs using the database close it;
* the lockfile is always reset on first open of the environment.
*
* - On BSD systems or others configured with MDB_USE_POSIX_SEM,
* - On BSD systems or others configured with MDB_USE_SYSV_SEM,
* startup can fail due to semaphores owned by another userid.
*
* Fix: Open and close the database as the user which owns the
@ -105,6 +109,7 @@
* The transaction becomes "long-lived" as above until a check
* for stale readers is performed or the lockfile is reset,
* since the process may not remove it from the lockfile.
* Except write-transactions on Unix with MDB_ROBUST or on Windows.
*
* - If you do that anyway, do a periodic check for stale readers. Or
* close the environment once in a while, so the lockfile can get reset.
@ -391,7 +396,7 @@ typedef enum MDB_cursor_op {
#define MDB_PAGE_NOTFOUND (-30797)
/** Located page was wrong type */
#define MDB_CORRUPTED (-30796)
/** Update of meta page failed, probably I/O error */
/** Update of meta page failed or environment had fatal error */
#define MDB_PANIC (-30795)
/** Environment version mismatch */
#define MDB_VERSION_MISMATCH (-30794)
@ -727,6 +732,7 @@ void mdb_env_close(MDB_env *env);
* This may be used to set some flags in addition to those from
* #mdb_env_open(), or to unset these flags. If several threads
* change the flags at the same time, the result is undefined.
* Most flags cannot be changed after #mdb_env_open().
* @param[in] env An environment handle returned by #mdb_env_create()
* @param[in] flags The flags to change, bitwise OR'ed together
* @param[in] onoff A non-zero value sets the flags, zero clears them.
@ -944,6 +950,17 @@ int mdb_txn_begin(MDB_env *env, MDB_txn *parent, unsigned int flags, MDB_txn **
*/
MDB_env *mdb_txn_env(MDB_txn *txn);
/** @brief Return the transaction's ID.
*
* This returns the identifier associated with this transaction. For a
* read-only transaction, this corresponds to the snapshot being read;
* concurrent readers will frequently have the same transaction ID.
*
* @param[in] txn A transaction handle returned by #mdb_txn_begin()
* @return A transaction ID, valid if input is an active transaction.
*/
size_t mdb_txn_id(MDB_txn *txn);
/** @brief Commit all the operations of a transaction into the database.
*
* The transaction handle is freed. It and its cursors must not be used

456
mdb.c
View File

@ -102,7 +102,7 @@ extern int cacheflush(char *addr, int nbytes, int cache);
#endif
#if defined(__APPLE__) || defined (BSD)
# define MDB_USE_POSIX_SEM 1
# define MDB_USE_SYSV_SEM 1
# define MDB_FDATASYNC fsync
#elif defined(ANDROID)
# define MDB_FDATASYNC fsync
@ -110,11 +110,18 @@ extern int cacheflush(char *addr, int nbytes, int cache);
#ifndef _WIN32
#include <pthread.h>
#ifdef MDB_USE_POSIX_SEM
# define MDB_USE_HASH 1
#include <semaphore.h>
#endif
#endif
#ifdef MDB_USE_SYSV_SEM
#include <sys/ipc.h>
#include <sys/sem.h>
#ifdef _SEM_SEMUN_UNDEFINED
union semun {
int val;
struct semid_ds *buf;
unsigned short *array;
};
#endif /* _SEM_SEMUN_UNDEFINED */
#endif /* MDB_USE_SYSV_SEM */
#endif /* !_WIN32 */
#ifdef USE_VALGRIND
#include <valgrind/memcheck.h>
@ -191,6 +198,10 @@ extern int cacheflush(char *addr, int nbytes, int cache);
/** Features under development */
#ifndef MDB_DEVEL
#define MDB_DEVEL 0
#endif
#if defined(_WIN32) || (defined(EOWNERDEAD) && !defined(MDB_USE_SYSV_SEM))
#define MDB_ROBUST_SUPPORTED 1
#endif
/** Wrapper around __func__, which is a C99 feature */
@ -210,6 +221,7 @@ extern int cacheflush(char *addr, int nbytes, int cache);
#define pthread_t HANDLE
#define pthread_mutex_t HANDLE
#define pthread_cond_t HANDLE
typedef HANDLE mdb_mutex_t;
#define pthread_key_t DWORD
#define pthread_self() GetCurrentThreadId()
#define pthread_key_create(x,y) \
@ -217,16 +229,16 @@ extern int cacheflush(char *addr, int nbytes, int cache);
#define pthread_key_delete(x) TlsFree(x)
#define pthread_getspecific(x) TlsGetValue(x)
#define pthread_setspecific(x,y) (TlsSetValue(x,y) ? 0 : ErrCode())
#define pthread_mutex_consistent(mutex) 0
#define pthread_mutex_unlock(x) ReleaseMutex(*x)
#define pthread_mutex_lock(x) WaitForSingleObject(*x, INFINITE)
#define pthread_cond_signal(x) SetEvent(*x)
#define pthread_cond_wait(cond,mutex) do{SignalObjectAndWait(*mutex, *cond, INFINITE, FALSE); WaitForSingleObject(*mutex, INFINITE);}while(0)
#define THREAD_CREATE(thr,start,arg) thr=CreateThread(NULL,0,start,arg,0,NULL)
#define THREAD_FINISH(thr) WaitForSingleObject(thr, INFINITE)
#define LOCK_MUTEX_R(env) pthread_mutex_lock(&(env)->me_rmutex)
#define UNLOCK_MUTEX_R(env) pthread_mutex_unlock(&(env)->me_rmutex)
#define LOCK_MUTEX_W(env) pthread_mutex_lock(&(env)->me_wmutex)
#define UNLOCK_MUTEX_W(env) pthread_mutex_unlock(&(env)->me_wmutex)
#define MDB_MUTEX(env, rw) ((env)->me_##rw##mutex)
#define LOCK_MUTEX0(mutex) WaitForSingleObject(mutex, INFINITE)
#define UNLOCK_MUTEX(mutex) ReleaseMutex(mutex)
#define getpid() GetCurrentProcessId()
#define MDB_FDATASYNC(fd) (!FlushFileBuffers(fd))
#define MDB_MSYNC(addr,len,flags) (!FlushViewOfFile(addr,len))
@ -249,38 +261,46 @@ extern int cacheflush(char *addr, int nbytes, int cache);
/** For MDB_LOCK_FORMAT: True if readers take a pid lock in the lockfile */
#define MDB_PIDLOCK 1
#ifdef MDB_USE_POSIX_SEM
#ifdef MDB_USE_SYSV_SEM
#define LOCK_MUTEX_R(env) mdb_sem_wait((env)->me_rmutex)
#define UNLOCK_MUTEX_R(env) sem_post((env)->me_rmutex)
#define LOCK_MUTEX_W(env) mdb_sem_wait((env)->me_wmutex)
#define UNLOCK_MUTEX_W(env) sem_post((env)->me_wmutex)
typedef struct mdb_mutex {
int semid;
int semnum;
} mdb_mutex_t;
#define MDB_MUTEX(env, rw) (&(env)->me_##rw##mutex)
#define LOCK_MUTEX0(mutex) mdb_sem_wait(mutex)
#define UNLOCK_MUTEX(mutex) do { \
struct sembuf sb = { 0, 1, SEM_UNDO }; \
sb.sem_num = (mutex)->semnum; \
semop((mutex)->semid, &sb, 1); \
} while(0)
static int
mdb_sem_wait(sem_t *sem)
mdb_sem_wait(mdb_mutex_t *sem)
{
int rc;
while ((rc = sem_wait(sem)) && (rc = errno) == EINTR) ;
struct sembuf sb = { 0, -1, SEM_UNDO };
sb.sem_num = sem->semnum;
while ((rc = semop(sem->semid, &sb, 1)) && (rc = errno) == EINTR) ;
return rc;
}
#else
/** Lock the reader mutex.
/** Pointer/HANDLE type of shared mutex/semaphore.
*/
#define LOCK_MUTEX_R(env) pthread_mutex_lock(&(env)->me_txns->mti_mutex)
/** Unlock the reader mutex.
typedef pthread_mutex_t mdb_mutex_t;
/** Mutex for the reader table (rw = r) or write transaction (rw = w).
*/
#define UNLOCK_MUTEX_R(env) pthread_mutex_unlock(&(env)->me_txns->mti_mutex)
/** Lock the writer mutex.
* Only a single write transaction is allowed at a time. Other writers
* will block waiting for this mutex.
#define MDB_MUTEX(env, rw) (&(env)->me_txns->mti_##rw##mutex)
/** Lock the reader or writer mutex.
* Returns 0 or a code to give #mdb_mutex_failed(), as in #LOCK_MUTEX().
*/
#define LOCK_MUTEX_W(env) pthread_mutex_lock(&(env)->me_txns->mti_wmutex)
/** Unlock the writer mutex.
#define LOCK_MUTEX0(mutex) pthread_mutex_lock(mutex)
/** Unlock the reader or writer mutex.
*/
#define UNLOCK_MUTEX_W(env) pthread_mutex_unlock(&(env)->me_txns->mti_wmutex)
#endif /* MDB_USE_POSIX_SEM */
#define UNLOCK_MUTEX(mutex) pthread_mutex_unlock(mutex)
#endif /* MDB_USE_SYSV_SEM */
/** Get the error code for the last failed system function.
*/
@ -305,14 +325,29 @@ mdb_sem_wait(sem_t *sem)
#define GET_PAGESIZE(x) ((x) = sysconf(_SC_PAGE_SIZE))
#endif
#if defined(_WIN32) || defined(MDB_USE_POSIX_SEM)
#if defined(_WIN32)
#define MNAME_LEN 32
#elif defined(MDB_USE_SYSV_SEM)
#define MNAME_LEN 0
#else
#define MNAME_LEN (sizeof(pthread_mutex_t))
#endif
/** @} */
#ifdef MDB_ROBUST_SUPPORTED
/** Lock mutex, handle any error, set rc = result.
* Return 0 on success, nonzero (not rc) on error.
*/
#define LOCK_MUTEX(rc, env, mutex) \
(((rc) = LOCK_MUTEX0(mutex)) && \
((rc) = mdb_mutex_failed(env, mutex, rc)))
static int mdb_mutex_failed(MDB_env *env, mdb_mutex_t *mutex, int rc);
#else
#define LOCK_MUTEX(rc, env, mutex) ((rc) = LOCK_MUTEX0(mutex))
#define mdb_mutex_failed(env, mutex, rc) (rc)
#endif
#ifndef _WIN32
/** A flag for opening a file and requesting synchronous data writes.
* This is only used when writing a meta page. It's not strictly needed;
@ -437,7 +472,7 @@ static txnid_t mdb_debug_start;
/** The version number for a database's datafile format. */
#define MDB_DATA_VERSION ((MDB_DEVEL) ? 999 : 1)
/** The version number for a database's lockfile format. */
#define MDB_LOCK_VERSION 1
#define MDB_LOCK_VERSION ((MDB_DEVEL) ? 999 : 1)
/** @brief The max size of a key we can write, or 0 for dynamic max.
*
@ -581,11 +616,11 @@ typedef struct MDB_rxbody {
* started from so we can avoid overwriting any data used in that
* particular version.
*/
txnid_t mrb_txnid;
volatile txnid_t mrb_txnid;
/** The process ID of the process owning this reader txn. */
MDB_PID_T mrb_pid;
volatile MDB_PID_T mrb_pid;
/** The thread ID of the thread owning this txn. */
MDB_THR_T mrb_tid;
volatile MDB_THR_T mrb_tid;
} MDB_rxbody;
/** The actual reader record, with cacheline padding. */
@ -621,24 +656,26 @@ typedef struct MDB_txbody {
uint32_t mtb_magic;
/** Format of this lock file. Must be set to #MDB_LOCK_FORMAT. */
uint32_t mtb_format;
#if defined(_WIN32) || defined(MDB_USE_POSIX_SEM)
#if defined(_WIN32)
char mtb_rmname[MNAME_LEN];
#elif defined(MDB_USE_SYSV_SEM)
int mtb_semid;
#else
/** Mutex protecting access to this table.
* This is the reader lock that #LOCK_MUTEX_R acquires.
* This is the #MDB_MUTEX(env,r) reader table lock.
*/
pthread_mutex_t mtb_mutex;
pthread_mutex_t mtb_rmutex;
#endif
/** The ID of the last transaction committed to the database.
* This is recorded here only for convenience; the value can always
* be determined by reading the main database meta pages.
*/
txnid_t mtb_txnid;
volatile txnid_t mtb_txnid;
/** The number of slots that have been used in the reader table.
* This always records the maximum count, it is not decremented
* when readers release their slots.
*/
unsigned mtb_numreaders;
volatile unsigned mtb_numreaders;
} MDB_txbody;
/** The actual reader table definition. */
@ -647,14 +684,17 @@ typedef struct MDB_txninfo {
MDB_txbody mtb;
#define mti_magic mt1.mtb.mtb_magic
#define mti_format mt1.mtb.mtb_format
#define mti_mutex mt1.mtb.mtb_mutex
#define mti_rmutex mt1.mtb.mtb_rmutex
#define mti_rmname mt1.mtb.mtb_rmname
#define mti_txnid mt1.mtb.mtb_txnid
#define mti_numreaders mt1.mtb.mtb_numreaders
char pad[(sizeof(MDB_txbody)+CACHELINE-1) & ~(CACHELINE-1)];
} mt1;
#ifdef MDB_USE_SYSV_SEM
#define mti_semid mt1.mtb.mtb_semid
#else
union {
#if defined(_WIN32) || defined(MDB_USE_POSIX_SEM)
#if defined(_WIN32)
char mt2_wmname[MNAME_LEN];
#define mti_wmname mt2.mt2_wmname
#else
@ -663,6 +703,7 @@ typedef struct MDB_txninfo {
#endif
char pad[(MNAME_LEN+CACHELINE-1) & ~(CACHELINE-1)];
} mt2;
#endif
MDB_reader mti_readers[1];
} MDB_txninfo;
@ -671,6 +712,7 @@ typedef struct MDB_txninfo {
((uint32_t) \
((MDB_LOCK_VERSION) \
/* Flags which describe functionality */ \
+ (((MNAME_LEN) == 0) << 18) /* MDB_USE_SYSV_SEM */ \
+ (((MDB_PIDLOCK) != 0) << 16)))
/** @} */
@ -909,7 +951,7 @@ typedef struct MDB_meta {
/** Any persistent environment flags. @ref mdb_env */
#define mm_flags mm_dbs[0].md_flags
pgno_t mm_last_pg; /**< last used page in file */
txnid_t mm_txnid; /**< txnid that committed this page */
volatile txnid_t mm_txnid; /**< txnid that committed this page */
} MDB_meta;
/** Buffer for a stack-allocated meta page.
@ -1138,11 +1180,11 @@ struct MDB_env {
int me_live_reader; /**< have liveness lock in reader table */
#ifdef _WIN32
int me_pidquery; /**< Used in OpenProcess */
HANDLE me_rmutex; /* Windows mutexes don't reside in shared mem */
HANDLE me_wmutex;
#elif defined(MDB_USE_POSIX_SEM)
sem_t *me_rmutex; /* Shared mutexes are not supported */
sem_t *me_wmutex;
#endif
#if defined(_WIN32) || defined(MDB_USE_SYSV_SEM)
/* Windows mutexes/SysV semaphores do not reside in shared mem */
mdb_mutex_t me_rmutex;
mdb_mutex_t me_wmutex;
#endif
void *me_userctx; /**< User-settable context */
MDB_assert_func *me_assert_func; /**< Callback for assertion failures */
@ -1194,7 +1236,7 @@ static int mdb_page_split(MDB_cursor *mc, MDB_val *newkey, MDB_val *newdata,
static int mdb_env_read_header(MDB_env *env, MDB_meta *meta);
static int mdb_env_pick_meta(const MDB_env *env);
static int mdb_env_write_meta(MDB_txn *txn);
#if !(defined(_WIN32) || defined(MDB_USE_POSIX_SEM)) /* Drop unused excl arg */
#if !(defined(_WIN32) || defined(MDB_USE_SYSV_SEM)) /* Drop unused excl arg */
# define mdb_env_close0(env, excl) mdb_env_close1(env)
#endif
static void mdb_env_close0(MDB_env *env, int excl);
@ -1231,6 +1273,7 @@ static void mdb_xcursor_init1(MDB_cursor *mc, MDB_node *node);
static int mdb_drop0(MDB_cursor *mc, int subs);
static void mdb_default_cmp(MDB_txn *txn, MDB_dbi dbi);
static int mdb_reader_check0(MDB_env *env, int rlocked, int *dead);
/** @cond */
static MDB_cmp_func mdb_cmp_memn, mdb_cmp_memnr, mdb_cmp_int, mdb_cmp_cint, mdb_cmp_long;
@ -1258,7 +1301,7 @@ static char *const mdb_errstr[] = {
"MDB_NOTFOUND: No matching key/data pair found",
"MDB_PAGE_NOTFOUND: Requested page not found",
"MDB_CORRUPTED: Located page was wrong type",
"MDB_PANIC: Update of meta page failed",
"MDB_PANIC: Update of meta page failed or environment had fatal error",
"MDB_VERSION_MISMATCH: Database environment version mismatch",
"MDB_INVALID: File is not an LMDB file",
"MDB_MAP_FULL: Environment mapsize limit reached",
@ -2511,6 +2554,7 @@ mdb_txn_renew0(MDB_txn *txn)
} else {
MDB_PID_T pid = env->me_pid;
MDB_THR_T tid = pthread_self();
mdb_mutex_t *rmutex = MDB_MUTEX(env, r);
if (!env->me_live_reader) {
rc = mdb_reader_pid(env, Pidset, pid);
@ -2519,40 +2563,54 @@ mdb_txn_renew0(MDB_txn *txn)
env->me_live_reader = 1;
}
LOCK_MUTEX_R(env);
if (LOCK_MUTEX(rc, env, rmutex))
return rc;
nr = ti->mti_numreaders;
for (i=0; i<nr; i++)
if (ti->mti_readers[i].mr_pid == 0)
break;
if (i == env->me_maxreaders) {
UNLOCK_MUTEX_R(env);
UNLOCK_MUTEX(rmutex);
return MDB_READERS_FULL;
}
ti->mti_readers[i].mr_pid = pid;
ti->mti_readers[i].mr_tid = tid;
r = &ti->mti_readers[i];
r->mr_txnid = (txnid_t)-1;
r->mr_tid = tid;
r->mr_pid = pid; /* should be written last, see ITS#7971. */
if (i == nr)
ti->mti_numreaders = ++nr;
/* Save numreaders for un-mutexed mdb_env_close() */
env->me_numreaders = nr;
UNLOCK_MUTEX_R(env);
UNLOCK_MUTEX(rmutex);
r = &ti->mti_readers[i];
new_notls = (env->me_flags & MDB_NOTLS);
if (!new_notls && (rc=pthread_setspecific(env->me_txkey, r))) {
r->mr_pid = 0;
return rc;
}
}
txn->mt_txnid = r->mr_txnid = ti->mti_txnid;
do /* LY: Retry on a race, ITS#7970. */
r->mr_txnid = ti->mti_txnid;
while(r->mr_txnid != ti->mti_txnid);
txn->mt_txnid = r->mr_txnid;
txn->mt_u.reader = r;
meta = env->me_metas[txn->mt_txnid & 1];
}
} else {
if (ti) {
LOCK_MUTEX_W(env);
if (LOCK_MUTEX(rc, env, MDB_MUTEX(env, w)))
return rc;
#ifdef MDB_USE_SYSV_SEM
meta = env->me_metas[ mdb_env_pick_meta(env) ];
txn->mt_txnid = meta->mm_txnid;
/* Update mti_txnid like mdb_mutex_failed() would,
* in case last writer crashed before updating it.
*/
ti->mti_txnid = txn->mt_txnid;
#else
txn->mt_txnid = ti->mti_txnid;
meta = env->me_metas[txn->mt_txnid & 1];
#endif
} else {
meta = env->me_metas[ mdb_env_pick_meta(env) ];
txn->mt_txnid = meta->mm_txnid;
@ -2744,6 +2802,13 @@ mdb_txn_env(MDB_txn *txn)
return txn->mt_env;
}
size_t
mdb_txn_id(MDB_txn *txn)
{
if(!txn) return 0;
return txn->mt_txnid;
}
/** Export or close DBI handles opened in this txn. */
static void
mdb_dbis_update(MDB_txn *txn, int keep)
@ -2816,7 +2881,7 @@ mdb_txn_reset0(MDB_txn *txn, const char *act)
env->me_txn = NULL;
/* The writer mutex was locked in mdb_txn_begin. */
if (env->me_txns)
UNLOCK_MUTEX_W(env);
UNLOCK_MUTEX(MDB_MUTEX(env, w));
} else {
txn->mt_parent->mt_child = NULL;
env->me_pgstate = ((MDB_ntxn *)txn)->mnt_pgstate;
@ -3416,7 +3481,7 @@ done:
mdb_dbis_update(txn, 1);
if (env->me_txns)
UNLOCK_MUTEX_W(env);
UNLOCK_MUTEX(MDB_MUTEX(env, w));
if (txn != env->me_txn0)
free(txn);
@ -3491,6 +3556,7 @@ mdb_env_read_header(MDB_env *env, MDB_meta *meta)
return 0;
}
/** Fill in most of the zeroed #MDB_meta for an empty database environment */
static void ESECT
mdb_env_init_meta0(MDB_env *env, MDB_meta *meta)
{
@ -3507,7 +3573,7 @@ mdb_env_init_meta0(MDB_env *env, MDB_meta *meta)
/** Write the environment parameters of a freshly created DB environment.
* @param[in] env the environment handle
* @param[out] meta address of where to store the meta information
* @param[in] meta the #MDB_meta to write
* @return 0 on success, non-zero on failure.
*/
static int ESECT
@ -3534,8 +3600,6 @@ mdb_env_init_meta(MDB_env *env, MDB_meta *meta)
psize = env->me_psize;
mdb_env_init_meta0(env, meta);
p = calloc(2, psize);
p->mp_pgno = 0;
p->mp_flags = P_META;
@ -3593,6 +3657,10 @@ mdb_env_write_meta(MDB_txn *txn)
mp->mm_dbs[0] = txn->mt_dbs[0];
mp->mm_dbs[1] = txn->mt_dbs[1];
mp->mm_last_pg = txn->mt_next_pgno - 1;
#if !(defined(_MSC_VER) || defined(__i386__) || defined(__x86_64__))
/* LY: issue a memory barrier, if not x86. ITS#7969 */
__sync_synchronize();
#endif
mp->mm_txnid = txn->mt_txnid;
if (!(env->me_flags & (MDB_NOMETASYNC|MDB_NOSYNC))) {
unsigned meta_size = env->me_psize;
@ -3702,9 +3770,9 @@ mdb_env_create(MDB_env **env)
e->me_fd = INVALID_HANDLE_VALUE;
e->me_lfd = INVALID_HANDLE_VALUE;
e->me_mfd = INVALID_HANDLE_VALUE;
#ifdef MDB_USE_POSIX_SEM
e->me_rmutex = SEM_FAILED;
e->me_wmutex = SEM_FAILED;
#ifdef MDB_USE_SYSV_SEM
e->me_rmutex.semid = -1;
e->me_wmutex.semid = -1;
#endif
e->me_pid = getpid();
GET_PAGESIZE(e->me_os_psize);
@ -3805,16 +3873,16 @@ mdb_env_set_mapsize(MDB_env *env, size_t size)
*/
if (env->me_map) {
int rc;
MDB_meta *meta;
void *old;
if (env->me_txn)
return EINVAL;
meta = env->me_metas[mdb_env_pick_meta(env)];
if (!size)
size = env->me_metas[mdb_env_pick_meta(env)]->mm_mapsize;
else if (size < env->me_mapsize) {
/* If the configured size is smaller, make sure it's
* still big enough. Silently round up to minimum if not.
*/
size_t minsize = (env->me_metas[mdb_env_pick_meta(env)]->mm_last_pg + 1) * env->me_psize;
size = meta->mm_mapsize;
{
/* Silently round up to minimum if the size is too small */
size_t minsize = (meta->mm_last_pg + 1) * env->me_psize;
if (size < minsize)
size = minsize;
}
@ -3897,8 +3965,6 @@ mdb_env_open2(MDB_env *env)
env->me_pidquery = PROCESS_QUERY_INFORMATION;
#endif /* _WIN32 */
memset(&meta, 0, sizeof(meta));
if ((i = mdb_env_read_header(env, &meta)) != 0) {
if (i != ENOENT)
return i;
@ -3907,24 +3973,40 @@ mdb_env_open2(MDB_env *env)
env->me_psize = env->me_os_psize;
if (env->me_psize > MAX_PAGESIZE)
env->me_psize = MAX_PAGESIZE;
memset(&meta, 0, sizeof(meta));
mdb_env_init_meta0(env, &meta);
meta.mm_mapsize = DEFAULT_MAPSIZE;
} else {
env->me_psize = meta.mm_psize;
}
/* Was a mapsize configured? */
if (!env->me_mapsize) {
/* If this is a new environment, take the default,
* else use the size recorded in the existing env.
*/
env->me_mapsize = newenv ? DEFAULT_MAPSIZE : meta.mm_mapsize;
} else if (env->me_mapsize < meta.mm_mapsize) {
/* If the configured size is smaller, make sure it's
* still big enough. Silently round up to minimum if not.
env->me_mapsize = meta.mm_mapsize;
}
{
/* Make sure mapsize >= committed data size. Even when using
* mm_mapsize, which could be broken in old files (ITS#7789).
*/
size_t minsize = (meta.mm_last_pg + 1) * meta.mm_psize;
if (env->me_mapsize < minsize)
env->me_mapsize = minsize;
}
meta.mm_mapsize = env->me_mapsize;
if (newenv && !(flags & MDB_FIXEDMAP)) {
/* mdb_env_map() may grow the datafile. Write the metapages
* first, so the file will be valid if initialization fails.
* Except with FIXEDMAP, since we do not yet know mm_address.
* We could fill in mm_address later, but then a different
* program might end up doing that - one with a memory layout
* and map address which does not suit the main program.
*/
rc = mdb_env_init_meta(env, &meta);
if (rc)
return rc;
newenv = 0;
}
rc = mdb_fsize(env->me_fd, &env->me_size);
if (rc)
@ -4112,8 +4194,8 @@ mdb_env_excl_lock(MDB_env *env, int *excl)
if (!rc) {
*excl = 1;
} else
# ifdef MDB_USE_POSIX_SEM
if (*excl < 0) /* always true when !MDB_USE_POSIX_SEM */
# ifdef MDB_USE_SYSV_SEM
if (*excl < 0) /* always true when !MDB_USE_SYSV_SEM */
# endif
{
lock_info.l_type = F_RDLCK;
@ -4351,50 +4433,35 @@ mdb_env_setup_locks(MDB_env *env, char *lpath, int mode, int *excl)
if (!env->me_rmutex) goto fail_errno;
env->me_wmutex = CreateMutex(&mdb_all_sa, FALSE, env->me_txns->mti_wmname);
if (!env->me_wmutex) goto fail_errno;
#elif defined(MDB_USE_POSIX_SEM)
struct stat stbuf;
struct {
dev_t dev;
ino_t ino;
} idbuf;
MDB_val val;
char encbuf[11];
#elif defined(MDB_USE_SYSV_SEM)
union semun semu;
unsigned short vals[2] = {1, 1};
int semid = semget(IPC_PRIVATE, 2, mode);
if (semid < 0)
goto fail_errno;
#if defined(__NetBSD__)
#define MDB_SHORT_SEMNAMES 1 /* limited to 14 chars */
#endif
if (fstat(env->me_lfd, &stbuf)) goto fail_errno;
idbuf.dev = stbuf.st_dev;
idbuf.ino = stbuf.st_ino;
val.mv_data = &idbuf;
val.mv_size = sizeof(idbuf);
mdb_hash_enc(&val, encbuf);
#ifdef MDB_SHORT_SEMNAMES
encbuf[9] = '\0'; /* drop name from 15 chars to 14 chars */
#endif
sprintf(env->me_txns->mti_rmname, "/MDBr%s", encbuf);
sprintf(env->me_txns->mti_wmname, "/MDBw%s", encbuf);
/* Clean up after a previous run, if needed: Try to
* remove both semaphores before doing anything else.
*/
sem_unlink(env->me_txns->mti_rmname);
sem_unlink(env->me_txns->mti_wmname);
env->me_rmutex = sem_open(env->me_txns->mti_rmname,
O_CREAT|O_EXCL, mode, 1);
if (env->me_rmutex == SEM_FAILED) goto fail_errno;
env->me_wmutex = sem_open(env->me_txns->mti_wmname,
O_CREAT|O_EXCL, mode, 1);
if (env->me_wmutex == SEM_FAILED) goto fail_errno;
#else /* MDB_USE_POSIX_SEM */
env->me_rmutex.semid = semid;
env->me_wmutex.semid = semid;
env->me_rmutex.semnum = 0;
env->me_wmutex.semnum = 1;
semu.array = vals;
if (semctl(semid, 0, SETALL, semu) < 0)
goto fail_errno;
env->me_txns->mti_semid = semid;
#else /* MDB_USE_SYSV_SEM */
pthread_mutexattr_t mattr;
if ((rc = pthread_mutexattr_init(&mattr))
|| (rc = pthread_mutexattr_setpshared(&mattr, PTHREAD_PROCESS_SHARED))
|| (rc = pthread_mutex_init(&env->me_txns->mti_mutex, &mattr))
#ifdef MDB_ROBUST_SUPPORTED
|| (rc = pthread_mutexattr_setrobust(&mattr, PTHREAD_MUTEX_ROBUST))
#endif
|| (rc = pthread_mutex_init(&env->me_txns->mti_rmutex, &mattr))
|| (rc = pthread_mutex_init(&env->me_txns->mti_wmutex, &mattr)))
goto fail;
pthread_mutexattr_destroy(&mattr);
#endif /* _WIN32 || MDB_USE_POSIX_SEM */
#endif /* _WIN32 || MDB_USE_SYSV_SEM */
env->me_txns->mti_magic = MDB_MAGIC;
env->me_txns->mti_format = MDB_LOCK_FORMAT;
@ -4402,6 +4469,11 @@ mdb_env_setup_locks(MDB_env *env, char *lpath, int mode, int *excl)
env->me_txns->mti_numreaders = 0;
} else {
#ifdef MDB_USE_SYSV_SEM
struct semid_ds buf;
union semun semu;
int semid;
#endif
if (env->me_txns->mti_magic != MDB_MAGIC) {
DPUTS("lock region has invalid magic");
rc = MDB_INVALID;
@ -4422,11 +4494,21 @@ mdb_env_setup_locks(MDB_env *env, char *lpath, int mode, int *excl)
if (!env->me_rmutex) goto fail_errno;
env->me_wmutex = OpenMutex(SYNCHRONIZE, FALSE, env->me_txns->mti_wmname);
if (!env->me_wmutex) goto fail_errno;
#elif defined(MDB_USE_POSIX_SEM)
env->me_rmutex = sem_open(env->me_txns->mti_rmname, 0);
if (env->me_rmutex == SEM_FAILED) goto fail_errno;
env->me_wmutex = sem_open(env->me_txns->mti_wmname, 0);
if (env->me_wmutex == SEM_FAILED) goto fail_errno;
#elif defined(MDB_USE_SYSV_SEM)
semid = env->me_txns->mti_semid;
semu.buf = &buf;
/* check for read access */
if (semctl(semid, 0, IPC_STAT, semu) < 0)
goto fail_errno;
/* check for write access */
if (semctl(semid, 0, IPC_SET, semu) < 0)
goto fail_errno;
env->me_rmutex.semid = semid;
env->me_wmutex.semid = semid;
env->me_rmutex.semnum = 0;
env->me_wmutex.semnum = 1;
#endif
}
return MDB_SUCCESS;
@ -4448,8 +4530,8 @@ fail:
* environment and re-opening it with the new flags.
*/
#define CHANGEABLE (MDB_NOSYNC|MDB_NOMETASYNC|MDB_MAPASYNC|MDB_NOMEMINIT)
#define CHANGELESS (MDB_FIXEDMAP|MDB_NOSUBDIR|MDB_RDONLY|MDB_WRITEMAP| \
MDB_NOTLS|MDB_NOLOCK|MDB_NORDAHEAD)
#define CHANGELESS (MDB_FIXEDMAP|MDB_NOSUBDIR|MDB_RDONLY| \
MDB_WRITEMAP|MDB_NOTLS|MDB_NOLOCK|MDB_NORDAHEAD)
#if VALID_FLAGS & PERSISTENT_FLAGS & (CHANGEABLE|CHANGELESS)
# error "Persistent DB flags & env flags overlap, but both go in mm_flags"
@ -4658,20 +4740,15 @@ mdb_env_close0(MDB_env *env, int excl)
/* Windows automatically destroys the mutexes when
* the last handle closes.
*/
#elif defined(MDB_USE_POSIX_SEM)
if (env->me_rmutex != SEM_FAILED) {
sem_close(env->me_rmutex);
if (env->me_wmutex != SEM_FAILED)
sem_close(env->me_wmutex);
#elif defined(MDB_USE_SYSV_SEM)
if (env->me_rmutex.semid != -1) {
/* If we have the filelock: If we are the
* only remaining user, clean up semaphores.
*/
if (excl == 0)
mdb_env_excl_lock(env, &excl);
if (excl > 0) {
sem_unlink(env->me_txns->mti_rmname);
sem_unlink(env->me_txns->mti_wmname);
}
if (excl > 0)
semctl(env->me_rmutex.semid, 0, IPC_RMID);
}
#endif
munmap((void *)env->me_txns, (env->me_maxreaders-1)*sizeof(MDB_reader)+sizeof(MDB_txninfo));
@ -4691,7 +4768,6 @@ mdb_env_close0(MDB_env *env, int excl)
env->me_flags &= ~(MDB_ENV_ACTIVE|MDB_ENV_TXKEY);
}
void ESECT
mdb_env_close(MDB_env *env)
{
@ -8685,6 +8761,7 @@ static int ESECT
mdb_env_copyfd0(MDB_env *env, HANDLE fd)
{
MDB_txn *txn = NULL;
mdb_mutex_t *wmutex = NULL;
int rc;
size_t wsize;
char *ptr;
@ -8709,11 +8786,13 @@ mdb_env_copyfd0(MDB_env *env, HANDLE fd)
mdb_txn_reset0(txn, "reset-stage1");
/* Temporarily block writers until we snapshot the meta pages */
LOCK_MUTEX_W(env);
wmutex = MDB_MUTEX(env, w);
if (LOCK_MUTEX(rc, env, wmutex))
goto leave;
rc = mdb_txn_renew0(txn);
if (rc) {
UNLOCK_MUTEX_W(env);
UNLOCK_MUTEX(wmutex);
goto leave;
}
}
@ -8737,8 +8816,8 @@ mdb_env_copyfd0(MDB_env *env, HANDLE fd)
break;
}
}
if (env->me_txns)
UNLOCK_MUTEX_W(env);
if (wmutex)
UNLOCK_MUTEX(wmutex);
if (rc)
goto leave;
@ -8861,7 +8940,7 @@ mdb_env_copy(MDB_env *env, const char *path)
int ESECT
mdb_env_set_flags(MDB_env *env, unsigned int flag, int onoff)
{
if ((flag & CHANGEABLE) != flag)
if (flag & (env->me_map ? ~CHANGEABLE : ~(CHANGEABLE|CHANGELESS)))
return EINVAL;
if (onoff)
env->me_flags |= flag;
@ -9415,17 +9494,22 @@ mdb_pid_insert(MDB_PID_T *ids, MDB_PID_T pid)
int ESECT
mdb_reader_check(MDB_env *env, int *dead)
{
unsigned int i, j, rdrs;
MDB_reader *mr;
MDB_PID_T *pids, pid;
int count = 0;
if (!env)
return EINVAL;
if (dead)
*dead = 0;
if (!env->me_txns)
return MDB_SUCCESS;
return env->me_txns ? mdb_reader_check0(env, 0, dead) : MDB_SUCCESS;
}
/** As #mdb_reader_check(). rlocked = <caller locked the reader mutex>. */
static int mdb_reader_check0(MDB_env *env, int rlocked, int *dead)
{
mdb_mutex_t *rmutex = rlocked ? NULL : MDB_MUTEX(env, r);
unsigned int i, j, rdrs;
MDB_reader *mr;
MDB_PID_T *pids, pid;
int rc = MDB_SUCCESS, count = 0;
rdrs = env->me_txns->mti_numreaders;
pids = malloc((rdrs+1) * sizeof(MDB_PID_T));
if (!pids)
@ -9433,22 +9517,32 @@ mdb_reader_check(MDB_env *env, int *dead)
pids[0] = 0;
mr = env->me_txns->mti_readers;
for (i=0; i<rdrs; i++) {
if (mr[i].mr_pid && mr[i].mr_pid != env->me_pid) {
pid = mr[i].mr_pid;
if (pid && pid != env->me_pid) {
if (mdb_pid_insert(pids, pid) == 0) {
if (!mdb_reader_pid(env, Pidcheck, pid)) {
LOCK_MUTEX_R(env);
/* Stale reader found */
j = i;
if (rmutex) {
if ((rc = LOCK_MUTEX0(rmutex)) != 0) {
if ((rc = mdb_mutex_failed(env, rmutex, rc)))
break;
rdrs = 0; /* the above checked all readers */
} else {
/* Recheck, a new process may have reused pid */
if (!mdb_reader_pid(env, Pidcheck, pid)) {
for (j=i; j<rdrs; j++)
if (mdb_reader_pid(env, Pidcheck, pid))
j = rdrs;
}
}
for (; j<rdrs; j++)
if (mr[j].mr_pid == pid) {
DPRINTF(("clear stale reader pid %u txn %"Z"d",
(unsigned) pid, mr[j].mr_txnid));
mr[j].mr_pid = 0;
count++;
}
}
UNLOCK_MUTEX_R(env);
if (rmutex)
UNLOCK_MUTEX(rmutex);
}
}
}
@ -9456,6 +9550,58 @@ mdb_reader_check(MDB_env *env, int *dead)
free(pids);
if (dead)
*dead = count;
return MDB_SUCCESS;
return rc;
}
#ifdef MDB_ROBUST_SUPPORTED
/** Handle #LOCK_MUTEX0() failure.
* With #MDB_ROBUST, try to repair the lock file if the mutex owner died.
* @param[in] env the environment handle
* @param[in] mutex LOCK_MUTEX0() mutex
* @param[in] rc LOCK_MUTEX0() error (nonzero)
* @return 0 on success with the mutex locked, or an error code on failure.
*/
static int mdb_mutex_failed(MDB_env *env, mdb_mutex_t *mutex, int rc)
{
int toggle, rlocked, rc2;
#ifndef _WIN32
enum { WAIT_ABANDONED = EOWNERDEAD };
#endif
if (rc == (int) WAIT_ABANDONED) {
/* We own the mutex. Clean up after dead previous owner. */
rc = MDB_SUCCESS;
rlocked = (mutex == MDB_MUTEX(env, r));
if (!rlocked) {
/* Keep mti_txnid updated, otherwise next writer can
* overwrite data which latest meta page refers to.
*/
toggle = mdb_env_pick_meta(env);
env->me_txns->mti_txnid = env->me_metas[toggle]->mm_txnid;
/* env is hosed if the dead thread was ours */
if (env->me_txn) {
env->me_flags |= MDB_FATAL_ERROR;
env->me_txn = NULL;
rc = MDB_PANIC;
}
}
DPRINTF(("%cmutex owner died, %s", (rlocked ? 'r' : 'w'),
(rc ? "this process' env is hosed" : "recovering")));
rc2 = mdb_reader_check0(env, rlocked, NULL);
if (rc2 == 0)
rc2 = pthread_mutex_consistent(mutex);
if (rc || (rc = rc2)) {
DPRINTF(("LOCK_MUTEX recovery failed, %s", mdb_strerror(rc)));
UNLOCK_MUTEX(mutex);
}
} else {
#ifdef _WIN32
rc = ErrCode();
#endif
DPRINTF(("LOCK_MUTEX failed, %s", mdb_strerror(rc)));
}
return rc;
}
#endif /* MDB_ROBUST_SUPPORTED */
/** @} */

View File

@ -64,7 +64,7 @@ int main(int argc,char * argv[])
act = "opening environment";
rc = mdb_env_create(&env);
if (rc == MDB_SUCCESS) {
rc = mdb_env_open(env, argv[1], flags, 0664);
rc = mdb_env_open(env, argv[1], flags, 0640);
}
if (rc == MDB_SUCCESS) {
act = "copying";