diff --git a/include/splinterdb/splinterdb.h b/include/splinterdb/splinterdb.h index f8206591..87586a1a 100644 --- a/include/splinterdb/splinterdb.h +++ b/include/splinterdb/splinterdb.h @@ -183,16 +183,16 @@ splinterdb_close(splinterdb **kvs); // Insert a key and value. // Relies on data_config->encode_message int -splinterdb_insert(const splinterdb *kvsb, slice key, slice value); +splinterdb_insert(splinterdb *kvsb, slice key, slice value); // Delete a given key and any associated value / messages int -splinterdb_delete(const splinterdb *kvsb, slice key); +splinterdb_delete(splinterdb *kvsb, slice key); // Insert a key and value. // Relies on data_config->encode_message int -splinterdb_update(const splinterdb *kvsb, slice key, slice delta); +splinterdb_update(splinterdb *kvsb, slice key, slice delta); // Lookups @@ -254,7 +254,7 @@ splinterdb_lookup_result_value(const splinterdb_lookup_result *result, // IN // // result must have first been initialized using splinterdb_lookup_result_init int -splinterdb_lookup(const splinterdb *kvs, // IN +splinterdb_lookup(splinterdb *kvs, // IN slice key, // IN splinterdb_lookup_result *result // IN/OUT ); @@ -318,7 +318,7 @@ typedef struct splinterdb_iterator splinterdb_iterator; // // If start_key is NULL_SLICE, the iterator will start before the minimum key int -splinterdb_iterator_init(const splinterdb *kvs, // IN +splinterdb_iterator_init(splinterdb *kvs, // IN splinterdb_iterator **iter, // OUT slice start_key // IN ); @@ -395,7 +395,7 @@ void splinterdb_stats_print_insertion(const splinterdb *kvs); void -splinterdb_stats_print_lookup(const splinterdb *kvs); +splinterdb_stats_print_lookup(splinterdb *kvs); void splinterdb_stats_reset(splinterdb *kvs); diff --git a/src/clockcache.c b/src/clockcache.c index 3ea5e4c2..1188b66c 100644 --- a/src/clockcache.c +++ b/src/clockcache.c @@ -3127,29 +3127,29 @@ clockcache_init(clockcache *cc, // OUT cc->heap_id = hid; /* lookup maps addrs to entries, entry contains the entries themselves */ - cc->lookup = - TYPED_ARRAY_MALLOC(cc->heap_id, cc->lookup, allocator_page_capacity); - if (!cc->lookup) { + platform_status rc = platform_buffer_init( + &cc->lookup_bh, allocator_page_capacity * sizeof(cc->lookup[0])); + if (!SUCCESS(rc)) { goto alloc_error; } + cc->lookup = platform_buffer_getaddr(&cc->lookup_bh); for (i = 0; i < allocator_page_capacity; i++) { cc->lookup[i] = CC_UNMAPPED_ENTRY; } - cc->entry = - TYPED_ARRAY_ZALLOC(cc->heap_id, cc->entry, cc->cfg->page_capacity); - if (!cc->entry) { + rc = platform_buffer_init(&cc->entry_bh, + cc->cfg->page_capacity * sizeof(cc->entry[0])); + if (!SUCCESS(rc)) { goto alloc_error; } - - platform_status rc = STATUS_NO_MEMORY; + cc->entry = platform_buffer_getaddr(&cc->entry_bh); /* data must be aligned because of O_DIRECT */ - rc = platform_buffer_init(&cc->bh, cc->cfg->capacity); + rc = platform_buffer_init(&cc->data_bh, cc->cfg->capacity); if (!SUCCESS(rc)) { goto alloc_error; } - cc->data = platform_buffer_getaddr(&cc->bh); + cc->data = platform_buffer_getaddr(&cc->data_bh); /* Set up the entries */ for (i = 0; i < cc->cfg->page_capacity; i++) { @@ -3172,11 +3172,12 @@ clockcache_init(clockcache *cc, // OUT cc->refcount = platform_buffer_getaddr(&cc->rc_bh); /* Separate ref counts for pins */ - cc->pincount = - TYPED_ARRAY_ZALLOC(cc->heap_id, cc->pincount, cc->cfg->page_capacity); - if (!cc->pincount) { + rc = platform_buffer_init(&cc->pincount_bh, + cc->cfg->page_capacity * sizeof(cc->pincount[0])); + if (!SUCCESS(rc)) { goto alloc_error; } + cc->pincount = platform_buffer_getaddr(&cc->pincount_bh); /* The hands and associated page */ cc->free_hand = 0; @@ -3185,13 +3186,14 @@ clockcache_init(clockcache *cc, // OUT cc->per_thread[thr_i].free_hand = CC_UNMAPPED_ENTRY; cc->per_thread[thr_i].enable_sync_get = TRUE; } - cc->batch_busy = - TYPED_ARRAY_ZALLOC(cc->heap_id, - cc->batch_busy, - cc->cfg->page_capacity / CC_ENTRIES_PER_BATCH); - if (!cc->batch_busy) { + + rc = platform_buffer_init(&cc->batch_bh, + cc->cfg->page_capacity / CC_ENTRIES_PER_BATCH + * sizeof(cc->batch_busy[0])); + if (!SUCCESS(rc)) { goto alloc_error; } + cc->batch_busy = platform_buffer_getaddr(&cc->batch_bh); return STATUS_OK; @@ -3209,6 +3211,7 @@ clockcache_init(clockcache *cc, // OUT void clockcache_deinit(clockcache *cc) // IN/OUT { + platform_status rc; platform_assert(cc != NULL); if (cc->logfile) { @@ -3219,35 +3222,61 @@ clockcache_deinit(clockcache *cc) // IN/OUT } if (cc->lookup) { - platform_free(cc->heap_id, cc->lookup); + rc = platform_buffer_deinit(&cc->lookup_bh); + if (!SUCCESS(rc)) { + platform_error_log("platform_buffer_deinit(&cc->lookup_bh) failed: %s", + platform_status_to_string(rc)); + } + cc->lookup = NULL; } if (cc->entry) { for (int i = 0; i < cc->cfg->page_capacity; i++) { async_wait_queue_deinit(&cc->entry[i].waiters); } - platform_free(cc->heap_id, cc->entry); + rc = platform_buffer_deinit(&cc->entry_bh); + if (!SUCCESS(rc)) { + platform_error_log("platform_buffer_deinit(&cc->entry_bh) failed: %s", + platform_status_to_string(rc)); + } + cc->entry = NULL; } - debug_only platform_status rc = STATUS_TEST_FAILED; if (cc->data) { - rc = platform_buffer_deinit(&cc->bh); + rc = platform_buffer_deinit(&cc->data_bh); // We expect above to succeed. Anyway, we are in the process of // dismantling the clockcache, hence, for now, can't do much by way // of reporting errors further upstream. - debug_assert(SUCCESS(rc), "rc=%s", platform_status_to_string(rc)); + if (!SUCCESS(rc)) { + platform_error_log("platform_buffer_deinit(&cc->data_bh) failed: %s", + platform_status_to_string(rc)); + } cc->data = NULL; } if (cc->refcount) { rc = platform_buffer_deinit(&cc->rc_bh); - debug_assert(SUCCESS(rc), "rc=%s", platform_status_to_string(rc)); + if (!SUCCESS(rc)) { + platform_error_log("platform_buffer_deinit(&cc->rc_bh) failed: %s", + platform_status_to_string(rc)); + } cc->refcount = NULL; } if (cc->pincount) { - platform_free_volatile(cc->heap_id, cc->pincount); + rc = platform_buffer_deinit(&cc->pincount_bh); + if (!SUCCESS(rc)) { + platform_error_log( + "platform_buffer_deinit(&cc->pincount_bh) failed: %s", + platform_status_to_string(rc)); + } + cc->pincount = NULL; } if (cc->batch_busy) { - platform_free_volatile(cc->heap_id, cc->batch_busy); + rc = platform_buffer_deinit(&cc->batch_bh); + if (!SUCCESS(rc)) { + platform_error_log("platform_buffer_deinit(&cc->batch_bh) failed: %s", + platform_status_to_string(rc)); + } + cc->batch_busy = NULL; } } diff --git a/src/clockcache.h b/src/clockcache.h index 7f5bbdbb..814cde6a 100644 --- a/src/clockcache.h +++ b/src/clockcache.h @@ -117,22 +117,26 @@ struct clockcache { allocator *al; io_handle *io; - uint32 *lookup; - clockcache_entry *entry; - buffer_handle bh; // actual memory for pages - char *data; // convenience pointer for bh + buffer_handle lookup_bh; + uint32 *lookup; // Convenience pointer for lookup_bh + buffer_handle entry_bh; + clockcache_entry *entry; // Convenience pointer for entry_bh + buffer_handle data_bh; // actual memory for pages + char *data; // convenience pointer for bh platform_log_handle *logfile; platform_heap_id heap_id; // Distributed locks (the write bit is in the status uint32 of the entry) buffer_handle rc_bh; volatile uint16 *refcount; - volatile uint8 *pincount; + buffer_handle pincount_bh; + volatile uint8 *pincount; // Convenience pointer for pincount_bh // Clock hands and related metadata volatile uint32 evict_hand; volatile uint32 free_hand; - volatile bool32 *batch_busy; + buffer_handle batch_bh; + volatile bool32 *batch_busy; // Convenience pointer for batch_bh uint64 cleaner_gap; volatile struct { diff --git a/src/core.c b/src/core.c index b8f96e96..29f48f0a 100644 --- a/src/core.c +++ b/src/core.c @@ -11,6 +11,7 @@ #include "data_internal.h" #include "platform_sleep.h" #include "platform_time.h" +#include "platform_util.h" #include "poison.h" #define LATENCYHISTO_SIZE 15 @@ -40,6 +41,8 @@ static const int64 latency_histo_buckets[LATENCYHISTO_SIZE] = { * limit. */ #define CORE_NUM_MEMTABLES (4) +_Static_assert(CORE_NUM_MEMTABLES <= MAX_MEMTABLES, + "CORE_NUM_MEMTABLES <= MAX_MEMTABLES"); /* * For a "small" range query, you don't want to prefetch pages. @@ -252,7 +255,7 @@ static memtable * core_try_get_memtable(core_handle *spl, uint64 generation) { uint64 memtable_idx = generation % CORE_NUM_MEMTABLES; - memtable *mt = &spl->mt_ctxt->mt[memtable_idx]; + memtable *mt = &spl->mt_ctxt.mt[memtable_idx]; if (mt->generation != generation) { mt = NULL; } @@ -267,13 +270,13 @@ static memtable * core_get_memtable(core_handle *spl, uint64 generation) { uint64 memtable_idx = generation % CORE_NUM_MEMTABLES; - memtable *mt = &spl->mt_ctxt->mt[memtable_idx]; + memtable *mt = &spl->mt_ctxt.mt[memtable_idx]; platform_assert(mt->generation == generation, "mt->generation=%lu, mt_ctxt->generation=%lu, " "mt_ctxt->generation_retired=%lu, generation=%lu\n", mt->generation, - spl->mt_ctxt->generation, - spl->mt_ctxt->generation_retired, + spl->mt_ctxt.generation, + spl->mt_ctxt.generation_retired, generation); return mt; } @@ -302,7 +305,7 @@ static void core_memtable_dec_ref(core_handle *spl, uint64 generation) { memtable *mt = core_get_memtable(spl, generation); - memtable_dec_ref_maybe_recycle(spl->mt_ctxt, mt); + memtable_dec_ref_maybe_recycle(&spl->mt_ctxt, mt); // the branch in the compacted memtable is now in the tree, so don't zap it, // we don't try to zero out the cmt because that would introduce a race. @@ -367,12 +370,12 @@ core_memtable_insert(core_handle *spl, key tuple_key, message msg) uint64 generation; platform_status rc = - memtable_maybe_rotate_and_begin_insert(spl->mt_ctxt, &generation); + memtable_maybe_rotate_and_begin_insert(&spl->mt_ctxt, &generation); while (STATUS_IS_EQ(rc, STATUS_BUSY)) { // Memtable isn't ready, do a task if available; may be required to // incorporate memtable that we're waiting on task_perform_one_if_needed(spl->ts, 0); - rc = memtable_maybe_rotate_and_begin_insert(spl->mt_ctxt, &generation); + rc = memtable_maybe_rotate_and_begin_insert(&spl->mt_ctxt, &generation); } if (!SUCCESS(rc)) { goto out; @@ -382,7 +385,7 @@ core_memtable_insert(core_handle *spl, key tuple_key, message msg) memtable *mt = core_get_memtable(spl, generation); uint64 leaf_generation; // used for ordering the log rc = memtable_insert( - spl->mt_ctxt, mt, spl->heap_id, tuple_key, msg, &leaf_generation); + &spl->mt_ctxt, mt, spl->heap_id, tuple_key, msg, &leaf_generation); if (!SUCCESS(rc)) { goto unlock_insert_lock; } @@ -395,7 +398,7 @@ core_memtable_insert(core_handle *spl, key tuple_key, message msg) } unlock_insert_lock: - memtable_end_insert(spl->mt_ctxt); + memtable_end_insert(&spl->mt_ctxt); out: return rc; } @@ -494,10 +497,10 @@ core_try_start_incorporate(core_handle *spl, uint64 generation) { bool32 should_start = FALSE; - memtable_lock_incorporation_lock(spl->mt_ctxt); + memtable_lock_incorporation_lock(&spl->mt_ctxt); memtable *mt = core_try_get_memtable(spl, generation); if ((mt == NULL) - || (generation != memtable_generation_to_incorporate(spl->mt_ctxt))) + || (generation != memtable_generation_to_incorporate(&spl->mt_ctxt))) { should_start = FALSE; goto unlock_incorp_lock; @@ -506,7 +509,7 @@ core_try_start_incorporate(core_handle *spl, uint64 generation) mt, MEMTABLE_STATE_COMPACTED, MEMTABLE_STATE_INCORPORATION_ASSIGNED); unlock_incorp_lock: - memtable_unlock_incorporation_lock(spl->mt_ctxt); + memtable_unlock_incorporation_lock(&spl->mt_ctxt); return should_start; } @@ -515,7 +518,7 @@ core_try_continue_incorporate(core_handle *spl, uint64 next_generation) { bool32 should_continue = FALSE; - memtable_lock_incorporation_lock(spl->mt_ctxt); + memtable_lock_incorporation_lock(&spl->mt_ctxt); memtable *mt = core_try_get_memtable(spl, next_generation); if (mt == NULL) { should_continue = FALSE; @@ -523,11 +526,11 @@ core_try_continue_incorporate(core_handle *spl, uint64 next_generation) } should_continue = memtable_try_transition( mt, MEMTABLE_STATE_COMPACTED, MEMTABLE_STATE_INCORPORATION_ASSIGNED); - memtable_increment_to_generation_to_incorporate(spl->mt_ctxt, + memtable_increment_to_generation_to_incorporate(&spl->mt_ctxt, next_generation); unlock_incorp_lock: - memtable_unlock_incorporation_lock(spl->mt_ctxt); + memtable_unlock_incorporation_lock(&spl->mt_ctxt); return should_continue; } @@ -569,18 +572,19 @@ core_memtable_incorporate(core_handle *spl, * lookups from accessing the memtable that's being incorporated). * And switch to the new root of the trunk. */ - memtable_block_lookups(spl->mt_ctxt); + memtable_block_lookups(&spl->mt_ctxt); memtable *mt = core_get_memtable(spl, generation); // Normally need to hold incorp_mutex, but debug code and also guaranteed no // one is changing gen_to_incorp (we are the only thread that would try) - debug_assert(generation == memtable_generation_to_incorporate(spl->mt_ctxt)); + debug_assert(generation + == memtable_generation_to_incorporate(&spl->mt_ctxt)); memtable_transition( mt, MEMTABLE_STATE_INCORPORATION_ASSIGNED, MEMTABLE_STATE_INCORPORATING); memtable_transition( mt, MEMTABLE_STATE_INCORPORATING, MEMTABLE_STATE_INCORPORATED); - memtable_increment_to_generation_retired(spl->mt_ctxt, generation); + memtable_increment_to_generation_retired(&spl->mt_ctxt, generation); trunk_incorporate_commit(&spl->trunk_context); - memtable_unblock_lookups(spl->mt_ctxt); + memtable_unblock_lookups(&spl->mt_ctxt); trunk_incorporate_cleanup(&spl->trunk_context); @@ -590,7 +594,7 @@ core_memtable_incorporate(core_handle *spl, * Decrement the now-incorporated memtable ref count and recycle if no * references */ - memtable_dec_ref_maybe_recycle(spl->mt_ctxt, mt); + memtable_dec_ref_maybe_recycle(&spl->mt_ctxt, mt); if (spl->cfg.use_stats) { const threadid tid = platform_get_tid(); @@ -848,13 +852,13 @@ core_range_iterator_init(core_handle *spl, ZERO_ARRAY(range_itor->compacted); // grab the lookup lock - memtable_begin_lookup(spl->mt_ctxt); + memtable_begin_lookup(&spl->mt_ctxt); // memtables ZERO_ARRAY(range_itor->branch); // Note this iteration is in descending generation order - range_itor->memtable_start_gen = memtable_generation(spl->mt_ctxt); - range_itor->memtable_end_gen = memtable_generation_retired(spl->mt_ctxt); + range_itor->memtable_start_gen = memtable_generation(&spl->mt_ctxt); + range_itor->memtable_end_gen = memtable_generation_retired(&spl->mt_ctxt); range_itor->num_memtable_branches = range_itor->memtable_start_gen - range_itor->memtable_end_gen; for (uint64 mt_gen = range_itor->memtable_start_gen; @@ -886,7 +890,7 @@ core_range_iterator_init(core_handle *spl, trunk_ondisk_node_handle root_handle; rc = trunk_init_root_handle(&spl->trunk_context, &root_handle); - memtable_end_lookup(spl->mt_ctxt); + memtable_end_lookup(&spl->mt_ctxt); if (!SUCCESS(rc)) { core_range_iterator_deinit(range_itor); return rc; @@ -1289,9 +1293,9 @@ core_lookup(core_handle *spl, key target, merge_accumulator *result) merge_accumulator_set_to_null(result); - memtable_begin_lookup(spl->mt_ctxt); - uint64 mt_gen_start = memtable_generation(spl->mt_ctxt); - uint64 mt_gen_end = memtable_generation_retired(spl->mt_ctxt); + memtable_begin_lookup(&spl->mt_ctxt); + uint64 mt_gen_start = memtable_generation(&spl->mt_ctxt); + uint64 mt_gen_end = memtable_generation_retired(&spl->mt_ctxt); platform_assert(mt_gen_start - mt_gen_end <= CORE_NUM_MEMTABLES); for (uint64 mt_gen = mt_gen_start; mt_gen != mt_gen_end; mt_gen--) { @@ -1299,7 +1303,7 @@ core_lookup(core_handle *spl, key target, merge_accumulator *result) rc = core_memtable_lookup(spl, mt_gen, target, result); platform_assert_status_ok(rc); if (merge_accumulator_is_definitive(result)) { - memtable_end_lookup(spl->mt_ctxt); + memtable_end_lookup(&spl->mt_ctxt); goto found_final_answer_early; } } @@ -1308,7 +1312,7 @@ core_lookup(core_handle *spl, key target, merge_accumulator *result) platform_status rc; rc = trunk_init_root_handle(&spl->trunk_context, &root_handle); // release memtable lookup lock before we handle any errors - memtable_end_lookup(spl->mt_ctxt); + memtable_end_lookup(&spl->mt_ctxt); if (!SUCCESS(rc)) { return rc; } @@ -1361,9 +1365,9 @@ core_lookup_async(core_lookup_async_state *state) merge_accumulator_set_to_null(state->result); - memtable_begin_lookup(state->spl->mt_ctxt); - uint64 mt_gen_start = memtable_generation(state->spl->mt_ctxt); - uint64 mt_gen_end = memtable_generation_retired(state->spl->mt_ctxt); + memtable_begin_lookup(&state->spl->mt_ctxt); + uint64 mt_gen_start = memtable_generation(&state->spl->mt_ctxt); + uint64 mt_gen_end = memtable_generation_retired(&state->spl->mt_ctxt); platform_assert(mt_gen_start - mt_gen_end <= CORE_NUM_MEMTABLES); for (uint64 mt_gen = mt_gen_start; mt_gen != mt_gen_end; mt_gen--) { @@ -1372,7 +1376,7 @@ core_lookup_async(core_lookup_async_state *state) core_memtable_lookup(state->spl, mt_gen, state->target, state->result); platform_assert_status_ok(rc); if (merge_accumulator_is_definitive(state->result)) { - memtable_end_lookup(state->spl->mt_ctxt); + memtable_end_lookup(&state->spl->mt_ctxt); goto found_final_answer_early; } } @@ -1380,7 +1384,7 @@ core_lookup_async(core_lookup_async_state *state) platform_status rc; rc = trunk_init_root_handle(&state->spl->trunk_context, &state->root_handle); // release memtable lookup lock before we handle any errors - memtable_end_lookup(state->spl->mt_ctxt); + memtable_end_lookup(&state->spl->mt_ctxt); if (!SUCCESS(rc)) { async_return(state, rc); } @@ -1468,22 +1472,17 @@ core_apply_to_range(core_handle *spl, } -/* - *----------------------------------------------------------------------------- - * Create/destroy - * XXX Fix this api to return platform_status - *----------------------------------------------------------------------------- - */ -core_handle * -core_create(core_config *cfg, - allocator *al, - cache *cc, - task_system *ts, - allocator_root_id id, - platform_heap_id hid) +/* Format the disk and mount the database */ +platform_status +core_mkfs(core_handle *spl, + core_config *cfg, + allocator *al, + cache *cc, + task_system *ts, + allocator_root_id id, + platform_heap_id hid) { - core_handle *spl = TYPED_FLEXIBLE_STRUCT_ZALLOC( - hid, spl, compacted_memtable, CORE_NUM_MEMTABLES); + ZERO_CONTENTS(spl); memmove(&spl->cfg, cfg, sizeof(*cfg)); spl->al = al; @@ -1495,8 +1494,15 @@ core_create(core_config *cfg, // set up the memtable context memtable_config *mt_cfg = &spl->cfg.mt_cfg; - spl->mt_ctxt = memtable_context_create( - spl->heap_id, cc, mt_cfg, core_memtable_flush_virtual, spl); + platform_status rc = memtable_context_init(&spl->mt_ctxt, + spl->heap_id, + cc, + mt_cfg, + core_memtable_flush_virtual, + spl); + if (!SUCCESS(rc)) { + return rc; + } // set up the log if (spl->cfg.use_log) { @@ -1532,22 +1538,22 @@ core_create(core_config *cfg, } } - return spl; + return STATUS_OK; } /* * Open (mount) an existing splinter database */ -core_handle * -core_mount(core_config *cfg, +platform_status +core_mount(core_handle *spl, + core_config *cfg, allocator *al, cache *cc, task_system *ts, allocator_root_id id, platform_heap_id hid) { - core_handle *spl = TYPED_FLEXIBLE_STRUCT_ZALLOC( - hid, spl, compacted_memtable, CORE_NUM_MEMTABLES); + ZERO_CONTENTS(spl); memmove(&spl->cfg, cfg, sizeof(*cfg)); spl->al = al; @@ -1571,8 +1577,15 @@ core_mount(core_config *cfg, } memtable_config *mt_cfg = &spl->cfg.mt_cfg; - spl->mt_ctxt = memtable_context_create( - spl->heap_id, cc, mt_cfg, core_memtable_flush_virtual, spl); + platform_status rc = memtable_context_init(&spl->mt_ctxt, + spl->heap_id, + cc, + mt_cfg, + core_memtable_flush_virtual, + spl); + if (!SUCCESS(rc)) { + return rc; + } if (spl->cfg.use_log) { spl->log = log_create(cc, spl->cfg.log_cfg, spl->heap_id); @@ -1605,7 +1618,7 @@ core_mount(core_config *cfg, platform_assert_status_ok(rc); } } - return spl; + return STATUS_OK; } /* @@ -1617,13 +1630,13 @@ core_prepare_for_shutdown(core_handle *spl) // write current memtable to disk // (any others must already be flushing/flushed) - if (!memtable_is_empty(spl->mt_ctxt)) { + if (!memtable_is_empty(&spl->mt_ctxt)) { /* * memtable_force_finalize is not thread safe. Note also, we do not hold * the insert lock or rotate while flushing the memtable. */ - uint64 generation = memtable_force_finalize(spl->mt_ctxt); + uint64 generation = memtable_force_finalize(&spl->mt_ctxt); core_memtable_flush(spl, generation); } @@ -1632,7 +1645,7 @@ core_prepare_for_shutdown(core_handle *spl) platform_assert_status_ok(rc); // destroy memtable context (and its memtables) - memtable_context_destroy(spl->heap_id, spl->mt_ctxt); + memtable_context_deinit(&spl->mt_ctxt); // release the log if (spl->cfg.use_log) { @@ -1644,16 +1657,15 @@ core_prepare_for_shutdown(core_handle *spl) } /* - * Destroy a database such that it cannot be re-opened later + * Close (unmount) a database without destroying it. + * It can be re-opened later with core_mount(). */ -void -core_destroy(core_handle *spl) +platform_status +core_unmount(core_handle *spl) { core_prepare_for_shutdown(spl); + core_set_super_block(spl, FALSE, TRUE, FALSE); trunk_context_deinit(&spl->trunk_context); - // clear out this splinter table from the meta page. - allocator_remove_super_addr(spl->al, spl->id); - if (spl->cfg.use_stats) { for (uint64 i = 0; i < MAX_THREADS; i++) { histogram_destroy(spl->heap_id, &spl->stats[i].insert_latency_histo); @@ -1662,20 +1674,20 @@ core_destroy(core_handle *spl) } platform_free(spl->heap_id, spl->stats); } - platform_free(spl->heap_id, spl); + return STATUS_OK; } /* - * Close (unmount) a database without destroying it. - * It can be re-opened later with core_mount(). + * Destroy a database such that it cannot be re-opened later */ void -core_unmount(core_handle **spl_in) +core_destroy(core_handle *spl) { - core_handle *spl = *spl_in; core_prepare_for_shutdown(spl); - core_set_super_block(spl, FALSE, TRUE, FALSE); trunk_context_deinit(&spl->trunk_context); + // clear out this splinter table from the meta page. + allocator_remove_super_addr(spl->al, spl->id); + if (spl->cfg.use_stats) { for (uint64 i = 0; i < MAX_THREADS; i++) { histogram_destroy(spl->heap_id, &spl->stats[i].insert_latency_histo); @@ -1684,10 +1696,9 @@ core_unmount(core_handle **spl_in) } platform_free(spl->heap_id, spl->stats); } - platform_free(spl->heap_id, spl); - *spl_in = (core_handle *)NULL; } + /* *----------------------------------------------------------------------------- * core_perform_task @@ -1742,7 +1753,7 @@ core_print_super_block(platform_log_handle *log_handle, core_handle *spl) // clang-format off void -core_print_insertion_stats(platform_log_handle *log_handle, core_handle *spl) +core_print_insertion_stats(platform_log_handle *log_handle, const core_handle *spl) { if (!spl->cfg.use_stats) { platform_log(log_handle, "Statistics are not enabled\n"); @@ -1922,8 +1933,8 @@ core_print_lookup(core_handle *spl, key target, platform_log_handle *log_handle) platform_stream_handle stream; platform_open_log_stream(&stream); - uint64 mt_gen_start = memtable_generation(spl->mt_ctxt); - uint64 mt_gen_end = memtable_generation_retired(spl->mt_ctxt); + uint64 mt_gen_start = memtable_generation(&spl->mt_ctxt); + uint64 mt_gen_end = memtable_generation_retired(&spl->mt_ctxt); for (uint64 mt_gen = mt_gen_start; mt_gen != mt_gen_end; mt_gen--) { bool32 memtable_is_compacted; uint64 root_addr = core_memtable_root_addr_for_lookup( diff --git a/src/core.h b/src/core.h index 65e2a066..6c7de97e 100644 --- a/src/core.h +++ b/src/core.h @@ -103,16 +103,16 @@ struct core_handle { uint64 super_block_idx; allocator_root_id id; - allocator *al; - cache *cc; - task_system *ts; - log_handle *log; - trunk_context trunk_context; - memtable_context *mt_ctxt; + allocator *al; + cache *cc; + task_system *ts; + log_handle *log; + trunk_context trunk_context; + memtable_context mt_ctxt; core_stats *stats; - core_compacted_memtable compacted_memtable[/*cfg.mt_cfg.max_memtables*/]; + core_compacted_memtable compacted_memtable[MAX_MEMTABLES]; }; typedef struct core_range_iterator { @@ -192,30 +192,38 @@ core_apply_to_range(core_handle *spl, tuple_function func, void *arg); -core_handle * -core_create(core_config *cfg, - allocator *al, - cache *cc, - task_system *ts, - allocator_root_id id, - platform_heap_id hid); -void -core_destroy(core_handle *spl); -core_handle * -core_mount(core_config *cfg, +/* Format the disk and mount the database */ +platform_status +core_mkfs(core_handle *spl, + core_config *cfg, + allocator *al, + cache *cc, + task_system *ts, + allocator_root_id id, + platform_heap_id hid); + +platform_status +core_mount(core_handle *spl, + core_config *cfg, allocator *al, cache *cc, task_system *ts, allocator_root_id id, platform_heap_id hid); + +platform_status +core_unmount(core_handle *spl); + +/* Unmount the database and erase it from the disk */ void -core_unmount(core_handle **spl); +core_destroy(core_handle *spl); void core_perform_tasks(core_handle *spl); void -core_print_insertion_stats(platform_log_handle *log_handle, core_handle *spl); +core_print_insertion_stats(platform_log_handle *log_handle, + const core_handle *spl); void core_print_lookup_stats(platform_log_handle *log_handle, core_handle *spl); diff --git a/src/memtable.c b/src/memtable.c index 59211991..4c4abc76 100644 --- a/src/memtable.c +++ b/src/memtable.c @@ -46,70 +46,70 @@ memtable_process(memtable_context *ctxt, uint64 generation) static inline void memtable_begin_insert(memtable_context *ctxt) { - batch_rwlock_get(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_get(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); } void memtable_end_insert(memtable_context *ctxt) { - batch_rwlock_unget(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_unget(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); } static inline bool32 memtable_try_begin_insert_rotation(memtable_context *ctxt) { - if (!batch_rwlock_try_claim(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX)) { + if (!batch_rwlock_try_claim(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX)) { return FALSE; } - batch_rwlock_lock(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_lock(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); return TRUE; } static inline void memtable_end_insert_rotation(memtable_context *ctxt) { - batch_rwlock_unlock(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); - batch_rwlock_unclaim(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_unlock(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_unclaim(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); } static inline void memtable_begin_raw_rotation(memtable_context *ctxt) { - batch_rwlock_get(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); - batch_rwlock_claim_loop(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); - batch_rwlock_lock(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_get(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_claim_loop(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_lock(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); } static inline void memtable_end_raw_rotation(memtable_context *ctxt) { - batch_rwlock_full_unlock(ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); + batch_rwlock_full_unlock(&ctxt->rwlock, MEMTABLE_INSERT_LOCK_IDX); } void memtable_begin_lookup(memtable_context *ctxt) { - batch_rwlock_get(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_get(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); } void memtable_end_lookup(memtable_context *ctxt) { - batch_rwlock_unget(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_unget(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); } void memtable_block_lookups(memtable_context *ctxt) { - batch_rwlock_get(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); - batch_rwlock_claim_loop(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); - batch_rwlock_lock(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_get(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_claim_loop(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_lock(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); } void memtable_unblock_lookups(memtable_context *ctxt) { - batch_rwlock_full_unlock(ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); + batch_rwlock_full_unlock(&ctxt->rwlock, MEMTABLE_LOOKUP_LOCK_IDX); } @@ -299,22 +299,27 @@ memtable_deinit(cache *cc, memtable *mt) debug_assert(freed); } -memtable_context * -memtable_context_create(platform_heap_id hid, - cache *cc, - memtable_config *cfg, - process_fn process, - void *process_ctxt) +platform_status +memtable_context_init(memtable_context *ctxt, + platform_heap_id hid, + cache *cc, + memtable_config *cfg, + process_fn process, + void *process_ctxt) { - memtable_context *ctxt = - TYPED_FLEXIBLE_STRUCT_ZALLOC(hid, ctxt, mt, cfg->max_memtables); + platform_status rc; + ZERO_CONTENTS(ctxt); ctxt->cc = cc; ctxt->cfg = *cfg; - platform_mutex_init( + rc = platform_mutex_init( &ctxt->incorporation_mutex, platform_get_module_id(), hid); - ctxt->rwlock = TYPED_MALLOC(hid, ctxt->rwlock); - batch_rwlock_init(ctxt->rwlock); + if (!SUCCESS(rc)) { + return rc; + } + batch_rwlock_init(&ctxt->rwlock); + + platform_assert(cfg->max_memtables <= MAX_MEMTABLES); for (uint64 mt_no = 0; mt_no < cfg->max_memtables; mt_no++) { uint64 generation = mt_no; @@ -330,11 +335,11 @@ memtable_context_create(platform_heap_id hid, ctxt->process = process; ctxt->process_ctxt = process_ctxt; - return ctxt; + return STATUS_OK; } void -memtable_context_destroy(platform_heap_id hid, memtable_context *ctxt) +memtable_context_deinit(memtable_context *ctxt) { cache *cc = ctxt->cc; for (uint64 mt_no = 0; mt_no < ctxt->cfg.max_memtables; mt_no++) { @@ -342,9 +347,7 @@ memtable_context_destroy(platform_heap_id hid, memtable_context *ctxt) } platform_mutex_destroy(&ctxt->incorporation_mutex); - platform_free(hid, ctxt->rwlock); - - platform_free(hid, ctxt); + batch_rwlock_deinit(&ctxt->rwlock); } void @@ -354,6 +357,7 @@ memtable_config_init(memtable_config *cfg, uint64 memtable_capacity) { ZERO_CONTENTS(cfg); + platform_assert(max_memtables <= MAX_MEMTABLES); cfg->btree_cfg = btree_cfg; cfg->max_memtables = max_memtables; cfg->max_extents_per_memtable = diff --git a/src/memtable.h b/src/memtable.h index 39d3e6b1..ff745660 100644 --- a/src/memtable.h +++ b/src/memtable.h @@ -15,6 +15,7 @@ #include "btree.h" #include "batch_rwlock.h" +#define MAX_MEMTABLES 128 #define MEMTABLE_SPACE_OVERHEAD_FACTOR (2) typedef enum memtable_state { @@ -118,7 +119,7 @@ typedef struct memtable_context { // batch distributed read/write locks protect the generation and // generation_retired counters - batch_rwlock *rwlock; + batch_rwlock rwlock; // Protected by the MEMTABLE_INSERT_LOCK_IDX'th lock of rwlock. Can read // without lock. Must get read lock to freeze and write lock to modify. @@ -137,7 +138,7 @@ typedef struct memtable_context { // Effectively thread local, no locking at all: btree_scratch scratch[MAX_THREADS]; - memtable mt[]; + memtable mt[MAX_MEMTABLES]; } memtable_context; platform_status @@ -179,15 +180,16 @@ memtable_init(memtable *mt, cache *cc, memtable_config *cfg, uint64 generation); void memtable_deinit(cache *cc, memtable *mt); -memtable_context * -memtable_context_create(platform_heap_id hid, - cache *cc, - memtable_config *cfg, - process_fn process, - void *process_ctxt); +platform_status +memtable_context_init(memtable_context *ctxt, + platform_heap_id hid, + cache *cc, + memtable_config *cfg, + process_fn process, + void *process_ctxt); void -memtable_context_destroy(platform_heap_id hid, memtable_context *ctxt); +memtable_context_deinit(memtable_context *ctxt); void memtable_config_init(memtable_config *cfg, diff --git a/src/splinterdb.c b/src/splinterdb.c index 1e12b5df..55db337a 100644 --- a/src/splinterdb.c +++ b/src/splinterdb.c @@ -52,7 +52,7 @@ typedef struct splinterdb { btree_config btree_cfg; trunk_config trunk_node_cfg; core_config trunk_cfg; - core_handle *spl; + core_handle spl; platform_heap_id heap_id; data_config *data_cfg; bool we_created_heap; @@ -354,21 +354,23 @@ splinterdb_create_or_open(const splinterdb_config *kvs_cfg, // IN kvs->trunk_id = 1; if (open_existing) { - kvs->spl = core_mount(&kvs->trunk_cfg, - (allocator *)&kvs->allocator_handle, - (cache *)&kvs->cache_handle, - &kvs->task_sys, - kvs->trunk_id, - kvs->heap_id); + status = core_mount(&kvs->spl, + &kvs->trunk_cfg, + (allocator *)&kvs->allocator_handle, + (cache *)&kvs->cache_handle, + &kvs->task_sys, + kvs->trunk_id, + kvs->heap_id); } else { - kvs->spl = core_create(&kvs->trunk_cfg, - (allocator *)&kvs->allocator_handle, - (cache *)&kvs->cache_handle, - &kvs->task_sys, - kvs->trunk_id, - kvs->heap_id); - } - if (kvs->spl == NULL || !SUCCESS(status)) { + status = core_mkfs(&kvs->spl, + &kvs->trunk_cfg, + (allocator *)&kvs->allocator_handle, + (cache *)&kvs->cache_handle, + &kvs->task_sys, + kvs->trunk_id, + kvs->heap_id); + } + if (!SUCCESS(status)) { platform_error_log("Failed to %s SplinterDB instance.\n", (open_existing ? "mount existing" : "initialize")); @@ -486,32 +488,32 @@ splinterdb_close(splinterdb **kvs_in) // IN *----------------------------------------------------------------------------- */ static int -splinterdb_insert_message(const splinterdb *kvs, // IN - slice user_key, // IN - message msg // IN +splinterdb_insert_message(splinterdb *kvs, // IN + slice user_key, // IN + message msg // IN ) { key tuple_key = key_create_from_slice(user_key); platform_assert(kvs != NULL); - platform_status status = core_insert(kvs->spl, tuple_key, msg); + platform_status status = core_insert(&kvs->spl, tuple_key, msg); return platform_status_to_int(status); } int -splinterdb_insert(const splinterdb *kvsb, slice user_key, slice value) +splinterdb_insert(splinterdb *kvsb, slice user_key, slice value) { message msg = message_create(MESSAGE_TYPE_INSERT, value); return splinterdb_insert_message(kvsb, user_key, msg); } int -splinterdb_delete(const splinterdb *kvsb, slice user_key) +splinterdb_delete(splinterdb *kvsb, slice user_key) { return splinterdb_insert_message(kvsb, user_key, DELETE_MESSAGE); } int -splinterdb_update(const splinterdb *kvsb, slice user_key, slice update) +splinterdb_update(splinterdb *kvsb, slice user_key, slice update) { message msg = message_create(MESSAGE_TYPE_UPDATE, update); platform_assert(kvsb->data_cfg->merge_tuples); @@ -600,7 +602,7 @@ splinterdb_lookup_result_value(const splinterdb_lookup_result *result, // IN *----------------------------------------------------------------------------- */ int -splinterdb_lookup(const splinterdb *kvs, // IN +splinterdb_lookup(splinterdb *kvs, // IN slice user_key, splinterdb_lookup_result *result) // IN/OUT { @@ -609,7 +611,7 @@ splinterdb_lookup(const splinterdb *kvs, // IN key target = key_create_from_slice(user_key); platform_assert(kvs != NULL); - status = core_lookup(kvs->spl, target, &_result->value); + status = core_lookup(&kvs->spl, target, &_result->value); return platform_status_to_int(status); } @@ -621,12 +623,12 @@ struct splinterdb_iterator { }; int -splinterdb_iterator_init(const splinterdb *kvs, // IN +splinterdb_iterator_init(splinterdb *kvs, // IN splinterdb_iterator **iter, // OUT slice user_start_key // IN ) { - splinterdb_iterator *it = TYPED_MALLOC(kvs->spl->heap_id, it); + splinterdb_iterator *it = TYPED_MALLOC(kvs->spl.heap_id, it); if (it == NULL) { platform_error_log("TYPED_MALLOC error\n"); return platform_status_to_int(STATUS_NO_MEMORY); @@ -642,7 +644,7 @@ splinterdb_iterator_init(const splinterdb *kvs, // IN start_key = key_create_from_slice(user_start_key); } - platform_status rc = core_range_iterator_init(kvs->spl, + platform_status rc = core_range_iterator_init(&kvs->spl, range_itor, NEGATIVE_INFINITY_KEY, POSITIVE_INFINITY_KEY, @@ -650,7 +652,7 @@ splinterdb_iterator_init(const splinterdb *kvs, // IN greater_than_or_equal, UINT64_MAX); if (!SUCCESS(rc)) { - platform_free(kvs->spl->heap_id, *iter); + platform_free(kvs->spl.heap_id, *iter); return platform_status_to_int(rc); } it->parent = kvs; @@ -737,19 +739,19 @@ splinterdb_iterator_get_current(splinterdb_iterator *iter, // IN void splinterdb_stats_print_insertion(const splinterdb *kvs) { - core_print_insertion_stats(Platform_default_log_handle, kvs->spl); + core_print_insertion_stats(Platform_default_log_handle, &kvs->spl); } void -splinterdb_stats_print_lookup(const splinterdb *kvs) +splinterdb_stats_print_lookup(splinterdb *kvs) { - core_print_lookup_stats(Platform_default_log_handle, kvs->spl); + core_print_lookup_stats(Platform_default_log_handle, &kvs->spl); } void splinterdb_stats_reset(splinterdb *kvs) { - core_reset_stats(kvs->spl); + core_reset_stats(&kvs->spl); } static void @@ -766,9 +768,9 @@ splinterdb_close_print_stats(splinterdb *kvs) * ------------------------------------------------------------------------- */ void -splinterdb_cache_flush(const splinterdb *kvs) +splinterdb_cache_flush(splinterdb *kvs) { - cache_flush(kvs->spl->cc); + cache_flush(kvs->spl.cc); } platform_heap_id @@ -804,11 +806,11 @@ splinterdb_get_cache_handle(const splinterdb *kvs) const core_handle * splinterdb_get_trunk_handle(const splinterdb *kvs) { - return kvs->spl; + return &kvs->spl; } const memtable_context * splinterdb_get_memtable_context_handle(const splinterdb *kvs) { - return kvs->spl->mt_ctxt; + return &kvs->spl.mt_ctxt; } \ No newline at end of file diff --git a/src/splinterdb_tests_private.h b/src/splinterdb_tests_private.h index bc7d2cde..bcdbc528 100644 --- a/src/splinterdb_tests_private.h +++ b/src/splinterdb_tests_private.h @@ -19,7 +19,7 @@ // External APIs provided -ONLY- for use as a testing hook. void -splinterdb_cache_flush(const splinterdb *kvs); +splinterdb_cache_flush(splinterdb *kvs); platform_heap_id splinterdb_get_heap_id(const splinterdb *kvs); diff --git a/tests/functional/btree_test.c b/tests/functional/btree_test.c index 134d5eb5..0926d1f2 100644 --- a/tests/functional/btree_test.c +++ b/tests/functional/btree_test.c @@ -36,7 +36,7 @@ typedef struct test_memtable_context { cache *cc; test_btree_config *cfg; platform_heap_id heap_id; - memtable_context *mt_ctxt; + memtable_context mt_ctxt; uint64 max_generation; } test_memtable_context; @@ -60,11 +60,15 @@ test_memtable_context_create(cache *cc, { test_memtable_context *ctxt = TYPED_ZALLOC(hid, ctxt); platform_assert(ctxt); - ctxt->cc = cc; - ctxt->cfg = cfg; - ctxt->heap_id = hid; - ctxt->mt_ctxt = memtable_context_create( - hid, cc, cfg->mt_cfg, test_btree_process_noop, NULL); + ctxt->cc = cc; + ctxt->cfg = cfg; + ctxt->heap_id = hid; + platform_status rc = memtable_context_init( + &ctxt->mt_ctxt, hid, cc, cfg->mt_cfg, test_btree_process_noop, NULL); + if (!SUCCESS(rc)) { + platform_free(hid, ctxt); + return NULL; + } ctxt->max_generation = num_mt; return ctxt; } @@ -72,7 +76,7 @@ test_memtable_context_create(cache *cc, void test_memtable_context_destroy(test_memtable_context *ctxt, platform_heap_id hid) { - memtable_context_destroy(hid, ctxt->mt_ctxt); + memtable_context_deinit(&ctxt->mt_ctxt); platform_free(hid, ctxt); } @@ -81,7 +85,7 @@ test_btree_insert(test_memtable_context *ctxt, key tuple_key, message data) { uint64 generation; platform_status rc = - memtable_maybe_rotate_and_begin_insert(ctxt->mt_ctxt, &generation); + memtable_maybe_rotate_and_begin_insert(&ctxt->mt_ctxt, &generation); if (!SUCCESS(rc)) { return rc; } @@ -93,15 +97,15 @@ test_btree_insert(test_memtable_context *ctxt, key tuple_key, message data) // dummy leaf generation (unused in this test) uint64 dummy_leaf_generation; - rc = memtable_insert(ctxt->mt_ctxt, - &ctxt->mt_ctxt->mt[generation], + rc = memtable_insert(&ctxt->mt_ctxt, + &ctxt->mt_ctxt.mt[generation], ctxt->heap_id, tuple_key, data, &dummy_leaf_generation); out: - memtable_end_insert(ctxt->mt_ctxt); + memtable_end_insert(&ctxt->mt_ctxt); return rc; } @@ -142,7 +146,7 @@ test_memtable_lookup(test_memtable_context *ctxt, message expected_data) { btree_config *btree_cfg = test_memtable_context_btree_config(ctxt); - uint64 root_addr = ctxt->mt_ctxt->mt[mt_no].root_addr; + uint64 root_addr = ctxt->mt_ctxt.mt[mt_no].root_addr; cache *cc = ctxt->cc; return test_btree_lookup(cc, btree_cfg, @@ -277,7 +281,7 @@ test_btree_perf(cache *cc, platform_default_log("btree total insertion rate: %lu insertions/second\n", SEC_TO_NSEC(total_inserts) / average_time); - memtable_print_stats(Platform_default_log_handle, cc, &ctxt->mt_ctxt->mt[0]); + memtable_print_stats(Platform_default_log_handle, cc, &ctxt->mt_ctxt.mt[0]); // for (i = 0; i < num_trees; i++) { // if (!btree_verify_tree(cc, cfg, root_addr[i])) // btree_print_tree(cc, cfg, root_addr[i], PAGE_TYPE_BRANCH); @@ -506,7 +510,7 @@ test_memtable_async_lookup(test_memtable_context *ctxt, bool32 expected_found, bool32 *correct) { - memtable *mt = &ctxt->mt_ctxt->mt[mt_no]; + memtable *mt = &ctxt->mt_ctxt.mt[mt_no]; btree_config *btree_cfg = mt->cfg; cache *cc = ctxt->cc; return test_btree_async_lookup(cc, @@ -528,7 +532,7 @@ test_btree_basic(cache *cc, platform_default_log("btree_test: btree basic test started\n"); test_memtable_context *ctxt = test_memtable_context_create(cc, cfg, 1, hid); - memtable *mt = &ctxt->mt_ctxt->mt[0]; + memtable *mt = &ctxt->mt_ctxt.mt[0]; data_config *data_cfg = mt->cfg->data_cfg; btree_test_async_lookup *async_lookup = TYPED_MALLOC(hid, async_lookup); @@ -819,7 +823,7 @@ test_btree_create_packed_trees(cache *cc, btree_config *btree_cfg = test_memtable_context_btree_config(ctxt); for (uint64 tree_no = 0; tree_no < num_trees; tree_no++) { - memtable *mt = &ctxt->mt_ctxt->mt[tree_no]; + memtable *mt = &ctxt->mt_ctxt.mt[tree_no]; btree_iterator itor; btree_iterator_init(cc, btree_cfg, diff --git a/tests/functional/splinter_test.c b/tests/functional/splinter_test.c index 64078c7f..4b5ebb04 100644 --- a/tests/functional/splinter_test.c +++ b/tests/functional/splinter_test.c @@ -49,7 +49,7 @@ typedef struct stats_insert { typedef struct test_splinter_thread_params { platform_thread thread; - core_handle **spl; + core_handle *spl; test_config *test_cfg; uint64 *total_ops; uint64 *curr_op; @@ -116,7 +116,7 @@ test_trunk_insert_thread(void *arg) { test_splinter_thread_params *params = (test_splinter_thread_params *)arg; - core_handle **spl_tables = params->spl; + core_handle *spl_tables = params->spl; const test_config *test_cfg = params->test_cfg; const uint64 *total_ops = params->total_ops; uint64 *curr_op = params->curr_op; @@ -166,7 +166,7 @@ test_trunk_insert_thread(void *arg) if (test_is_done(done, spl_idx)) { continue; } - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; timestamp ts; if (spl->cfg.use_stats) { @@ -214,7 +214,7 @@ test_trunk_insert_thread(void *arg) params->rc = STATUS_OK; platform_free(platform_get_heap_id(), insert_base); for (uint64 i = 0; i < num_tables; i++) { - core_handle *spl = spl_tables[i]; + core_handle *spl = &spl_tables[i]; core_perform_tasks(spl); } } @@ -227,7 +227,7 @@ test_trunk_lookup_thread(void *arg) { test_splinter_thread_params *params = (test_splinter_thread_params *)arg; - core_handle **spl_tables = params->spl; + core_handle *spl_tables = params->spl; const test_config *test_cfg = params->test_cfg; const uint64 *total_ops = params->total_ops; uint64 *curr_op = params->curr_op; @@ -276,7 +276,7 @@ test_trunk_lookup_thread(void *arg) if (test_is_done(done, spl_idx)) { continue; } - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; test_async_lookup *async_lookup = params->async_lookup[spl_idx]; test_async_ctxt *ctxt; uint64 lookup_num = lookup_base[spl_idx] + op_offset; @@ -328,7 +328,7 @@ test_trunk_lookup_thread(void *arg) if (test_is_done(done, spl_idx)) { continue; } - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; test_async_lookup *async_lookup = params->async_lookup[spl_idx]; test_wait_for_inflight(spl, async_lookup, &vtarg); } @@ -352,7 +352,7 @@ test_trunk_range_thread(void *arg) { test_splinter_thread_params *params = (test_splinter_thread_params *)arg; - core_handle **spl_tables = params->spl; + core_handle *spl_tables = params->spl; const test_config *test_cfg = params->test_cfg; const uint64 *total_ops = params->total_ops; uint64 *curr_op = params->curr_op; @@ -425,7 +425,7 @@ test_trunk_range_thread(void *arg) if (test_is_done(done, spl_idx)) { continue; } - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; uint64 range_num = range_base[spl_idx] + op_offset; test_key(&start_key, @@ -561,7 +561,7 @@ do_operation(test_splinter_thread_params *params, const uint8 *done, bool32 is_insert) { - core_handle **spl_tables = params->spl; + core_handle *spl_tables = params->spl; const test_config *test_cfg = params->test_cfg; uint64 op_granularity = params->op_granularity; uint64 thread_number = params->thread_number; @@ -583,7 +583,7 @@ do_operation(test_splinter_thread_params *params, if (test_is_done(*done, spl_idx)) { continue; } - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; uint64 op_num = base[spl_idx] + op_idx; timestamp ts; @@ -673,10 +673,10 @@ test_trunk_insert_lookup_thread(void *arg) { test_splinter_thread_params *params = (test_splinter_thread_params *)arg; - core_handle **spl_tables = params->spl; - uint8 num_tables = params->num_tables; - uint64 op_granularity = params->op_granularity; - uint64 seed = params->seed; + core_handle *spl_tables = params->spl; + uint8 num_tables = params->num_tables; + uint64 op_granularity = params->op_granularity; + uint64 seed = params->seed; platform_assert(num_tables <= 8); @@ -752,7 +752,7 @@ test_trunk_insert_lookup_thread(void *arg) out: for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; verify_tuple_arg vtarg = {.stats_only = TRUE, .stats = ¶ms->lookup_stats[ASYNC_LU]}; test_async_lookup *async_lookup = params->async_lookup[spl_idx]; @@ -767,7 +767,7 @@ test_trunk_insert_lookup_thread(void *arg) static platform_status -test_trunk_create_tables(core_handle ***spl_handles, +test_trunk_create_tables(core_handle **spl_handles, system_config *cfg, allocator *al, cache *cc[], @@ -776,25 +776,26 @@ test_trunk_create_tables(core_handle ***spl_handles, uint8 num_tables, uint8 num_caches) { - core_handle **spl_tables = TYPED_ARRAY_ZALLOC(hid, spl_tables, num_tables); + core_handle *spl_tables = TYPED_ARRAY_ZALLOC(hid, spl_tables, num_tables); if (spl_tables == NULL) { return STATUS_NO_MEMORY; } for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - cache *cache_to_use = num_caches > 1 ? cc[spl_idx] : *cc; - spl_tables[spl_idx] = core_create(&cfg[spl_idx].splinter_cfg, - al, - cache_to_use, - ts, - test_generate_allocator_root_id(), - hid); - if (spl_tables[spl_idx] == NULL) { + cache *cache_to_use = num_caches > 1 ? cc[spl_idx] : *cc; + platform_status rc = core_mkfs(&spl_tables[spl_idx], + &cfg[spl_idx].splinter_cfg, + al, + cache_to_use, + ts, + test_generate_allocator_root_id(), + hid); + if (!SUCCESS(rc)) { for (uint8 del_idx = 0; del_idx < spl_idx; del_idx++) { - core_destroy(spl_tables[del_idx]); + core_destroy(&spl_tables[del_idx]); } platform_free(hid, spl_tables); - return STATUS_NO_MEMORY; + return rc; } } *spl_handles = spl_tables; @@ -802,12 +803,12 @@ test_trunk_create_tables(core_handle ***spl_handles, } static void -test_trunk_destroy_tables(core_handle **spl_tables, +test_trunk_destroy_tables(core_handle *spl_tables, platform_heap_id hid, uint8 num_tables) { for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_destroy(spl_tables[spl_idx]); + core_destroy(&spl_tables[spl_idx]); } platform_free(hid, spl_tables); } @@ -853,7 +854,7 @@ compute_per_table_inserts(uint64 *per_table_inserts, // OUT */ static void load_thread_params(test_splinter_thread_params *params, - core_handle **spl_tables, + core_handle *spl_tables, test_config *test_cfg, uint64 *per_table_inserts, uint64 *curr_op, @@ -959,7 +960,7 @@ static platform_status splinter_perf_inserts(platform_heap_id hid, system_config *cfg, test_config *test_cfg, - core_handle **spl_tables, + core_handle *spl_tables, cache *cc[], task_system *ts, test_splinter_thread_params *params, @@ -1056,7 +1057,7 @@ splinter_perf_inserts(platform_heap_id hid, } for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_insertion_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1079,7 +1080,7 @@ static platform_status splinter_perf_lookups(platform_heap_id hid, system_config *cfg, test_config *test_cfg, - core_handle **spl_tables, + core_handle *spl_tables, task_system *ts, test_splinter_thread_params *params, uint64 num_lookup_threads, @@ -1159,7 +1160,7 @@ splinter_perf_lookups(platform_heap_id hid, sync_lookup_latency_max, async_lookup_latency_max); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1180,7 +1181,7 @@ splinter_perf_lookups(platform_heap_id hid, static platform_status splinter_perf_range_lookups(platform_heap_id hid, test_config *test_cfg, - core_handle **spl_tables, + core_handle *spl_tables, task_system *ts, test_splinter_thread_params *params, uint64 *per_table_inserts, @@ -1298,7 +1299,7 @@ splinter_perf_range_lookups(platform_heap_id hid, (total_time ? SEC_TO_NSEC(total_ranges) / total_time : 0)); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1341,7 +1342,7 @@ test_splinter_perf(system_config *cfg, platform_default_log("splinter_test: SplinterDB performance test started " "with %d tables\n", num_tables); - core_handle **spl_tables; + core_handle *spl_tables; platform_status rc; rc = test_trunk_create_tables( @@ -1466,7 +1467,7 @@ test_splinter_periodic(system_config *cfg, "splinter_test: SplinterDB performance test (periodic) started with " "%d tables\n", num_tables); - core_handle **spl_tables; + core_handle *spl_tables; platform_status rc; rc = test_trunk_create_tables( @@ -1570,7 +1571,7 @@ test_splinter_periodic(system_config *cfg, } for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_insertion_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1635,7 +1636,7 @@ test_splinter_periodic(system_config *cfg, } for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_insertion_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1715,7 +1716,7 @@ test_splinter_periodic(system_config *cfg, sync_lookup_latency_max, async_lookup_latency_max); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1771,7 +1772,7 @@ test_splinter_periodic(system_config *cfg, platform_default_log("splinter total range rate: %lu ops/second\n", SEC_TO_NSEC(total_ranges) / total_time); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1822,7 +1823,7 @@ test_splinter_periodic(system_config *cfg, platform_default_log("splinter total range rate: %lu ops/second\n", SEC_TO_NSEC(total_ranges) / total_time); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1873,7 +1874,7 @@ test_splinter_periodic(system_config *cfg, platform_default_log("splinter total range rate: %lu ops/second\n", SEC_TO_NSEC(total_ranges) / total_time); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; cache_assert_free(spl->cc); core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); @@ -1927,7 +1928,7 @@ test_splinter_parallel_perf(system_config *cfg, "splinter_test: SplinterDB parallel performance test started with " "%d tables\n", num_tables); - core_handle **spl_tables; + core_handle *spl_tables; platform_status rc; platform_assert(num_inserts_per_thread <= num_lookups_per_thread); @@ -2056,7 +2057,7 @@ test_splinter_parallel_perf(system_config *cfg, } for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; core_print_insertion_stats(Platform_default_log_handle, spl); } @@ -2084,7 +2085,7 @@ test_splinter_parallel_perf(system_config *cfg, sync_lookup_latency_max, async_lookup_latency_max); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); cache_reset_stats(spl->cc); @@ -2120,7 +2121,7 @@ test_splinter_delete(system_config *cfg, platform_default_log("splinter_test: SplinterDB deletion test started with " "%d tables\n", num_tables); - core_handle **spl_tables; + core_handle *spl_tables; platform_status rc; rc = test_trunk_create_tables( @@ -2190,7 +2191,7 @@ test_splinter_delete(system_config *cfg, SEC_TO_NSEC(total_inserts) / total_time); platform_default_log("After inserts:\n"); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; core_print_insertion_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); } @@ -2227,7 +2228,7 @@ test_splinter_delete(system_config *cfg, SEC_TO_NSEC(total_inserts) / total_time); platform_default_log("After deletes:\n"); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; core_print_insertion_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); } @@ -2292,7 +2293,7 @@ test_splinter_delete(system_config *cfg, platform_default_log("%lu%% lookups were async\n", num_async_lookups * 100 / total_inserts); for (uint8 spl_idx = 0; spl_idx < num_tables; spl_idx++) { - core_handle *spl = spl_tables[spl_idx]; + core_handle *spl = &spl_tables[spl_idx]; core_print_lookup_stats(Platform_default_log_handle, spl); cache_print_stats(Platform_default_log_handle, spl->cc); } diff --git a/tests/functional/test_functionality.c b/tests/functional/test_functionality.c index 4fd98f99..b49971eb 100644 --- a/tests/functional/test_functionality.c +++ b/tests/functional/test_functionality.c @@ -648,7 +648,7 @@ test_functionality(allocator *al, platform_error_log("Functional test started with %d tables\n", num_tables); platform_assert(cc != NULL); - core_handle **spl_tables = TYPED_ARRAY_ZALLOC(hid, spl_tables, num_tables); + core_handle *spl_tables = TYPED_ARRAY_ZALLOC(hid, spl_tables, num_tables); platform_assert(spl_tables != NULL); test_splinter_shadow_tree **shadows = @@ -683,18 +683,24 @@ test_functionality(allocator *al, } splinters[idx] = test_generate_allocator_root_id(); - spl_tables[idx] = core_create( - &cfg[idx].splinter_cfg, al, cache_to_use, state, splinters[idx], hid); - if (spl_tables[idx] == NULL) { - status = STATUS_NO_MEMORY; - platform_error_log("splinter_create() failed for index=%d.\n", idx); + status = core_mkfs(&spl_tables[idx], + &cfg[idx].splinter_cfg, + al, + cache_to_use, + state, + splinters[idx], + hid); + if (!SUCCESS(status)) { + platform_error_log("core_mkfs() failed for index=%d: %s\n", + idx, + platform_status_to_string(status)); goto cleanup; } } // Validate each tree against an empty shadow. for (uint8 idx = 0; idx < num_tables; idx++) { - core_handle *spl = spl_tables[idx]; + core_handle *spl = &spl_tables[idx]; test_splinter_shadow_tree *shadow = shadows[idx]; status = validate_tree_against_shadow( spl, &prg, shadow, hid, TRUE, async_lookup); @@ -770,7 +776,7 @@ test_functionality(allocator *al, // Run the main test loop for each table. for (uint8 idx = 0; idx < num_tables; idx++) { // cache *cache_to_use = num_caches > 1 ? cc[idx] : *cc; - core_handle *spl = spl_tables[idx]; + core_handle *spl = &spl_tables[idx]; test_splinter_shadow_tree *shadow = shadows[idx]; // allocator_root_id spl_id = splinters[idx]; @@ -832,7 +838,7 @@ test_functionality(allocator *al, // Validate each tree against the shadow one last time. for (uint8 idx = 0; idx < num_tables; idx++) { - core_handle *spl = spl_tables[idx]; + core_handle *spl = &spl_tables[idx]; test_splinter_shadow_tree *shadow = shadows[idx]; status = validate_tree_against_shadow( @@ -853,8 +859,8 @@ test_functionality(allocator *al, cleanup: for (uint8 idx = 0; idx < num_tables; idx++) { - if (spl_tables[idx] != NULL) { - core_destroy(spl_tables[idx]); + if (spl_tables[idx].cc != NULL) { + core_destroy(&spl_tables[idx]); } if (shadows[idx] != NULL) { test_splinter_shadow_destroy(hid, shadows[idx]); diff --git a/tests/functional/ycsb_test.c b/tests/functional/ycsb_test.c index dce0db51..73d22569 100644 --- a/tests/functional/ycsb_test.c +++ b/tests/functional/ycsb_test.c @@ -280,7 +280,7 @@ typedef struct ycsb_log_params { // State uint64 next_op; - core_handle *spl; + core_handle *spl; // pointer to spl in main function // Coordination uint64 *threads_complete; @@ -1272,7 +1272,7 @@ ycsb_test(int argc, char *argv[]) rc_allocator al; clockcache *cc = TYPED_MALLOC(hid, cc); - core_handle *spl; + core_handle spl; if (use_existing) { rc_allocator_mount( @@ -1285,13 +1285,14 @@ ycsb_test(int argc, char *argv[]) hid, platform_get_module_id()); platform_assert_status_ok(rc); - spl = core_mount(&system_cfg->splinter_cfg, - (allocator *)&al, - (cache *)cc, - &ts, - test_generate_allocator_root_id(), - hid); - platform_assert(spl); + rc = core_mount(&spl, + &system_cfg->splinter_cfg, + (allocator *)&al, + (cache *)cc, + &ts, + test_generate_allocator_root_id(), + hid); + platform_assert_status_ok(rc); } else { rc_allocator_init( &al, &system_cfg->allocator_cfg, io, hid, platform_get_module_id()); @@ -1303,16 +1304,17 @@ ycsb_test(int argc, char *argv[]) hid, platform_get_module_id()); platform_assert_status_ok(rc); - spl = core_create(&system_cfg->splinter_cfg, - (allocator *)&al, - (cache *)cc, - &ts, - test_generate_allocator_root_id(), - hid); - platform_assert(spl); + rc = core_mkfs(&spl, + &system_cfg->splinter_cfg, + (allocator *)&al, + (cache *)cc, + &ts, + test_generate_allocator_root_id(), + hid); + platform_assert_status_ok(rc); } - run_all_ycsb_phases(spl, phases, nphases, &ts, hid); + run_all_ycsb_phases(&spl, phases, nphases, &ts, hid); core_unmount(&spl); clockcache_deinit(cc); diff --git a/tests/unit/splinter_test.c b/tests/unit/splinter_test.c index 5ade6ae8..3e7139f6 100644 --- a/tests/unit/splinter_test.c +++ b/tests/unit/splinter_test.c @@ -224,22 +224,24 @@ CTEST2(splinter, test_inserts) { allocator *alp = (allocator *)&data->al; - core_handle *spl = core_create(&data->system_cfg->splinter_cfg, + core_handle spl; + platform_status rc = core_mkfs(&spl, + &data->system_cfg->splinter_cfg, alp, (cache *)data->clock_cache, &data->tasks, test_generate_allocator_root_id(), data->hid); - ASSERT_TRUE(spl != NULL); + ASSERT_TRUE(SUCCESS(rc)); // TRUE : Also do verification-after-inserts - uint64 num_inserts = splinter_do_inserts(data, spl, TRUE, NULL); + uint64 num_inserts = splinter_do_inserts(data, &spl, TRUE, NULL); ASSERT_NOT_EQUAL(0, num_inserts, "Expected to have inserted non-zero rows, num_inserts=%lu.", num_inserts); - core_destroy(spl); + core_destroy(&spl); } static void @@ -395,29 +397,31 @@ CTEST2(splinter, test_lookups) { allocator *alp = (allocator *)&data->al; - core_handle *spl = core_create(&data->system_cfg->splinter_cfg, - alp, - (cache *)data->clock_cache, - &data->tasks, - test_generate_allocator_root_id(), - data->hid); - ASSERT_TRUE(spl != NULL); + core_handle spl; + platform_status rc_init = core_mkfs(&spl, + &data->system_cfg->splinter_cfg, + alp, + (cache *)data->clock_cache, + &data->tasks, + test_generate_allocator_root_id(), + data->hid); + ASSERT_TRUE(SUCCESS(rc_init)); trunk_shadow shadow; trunk_shadow_init(&shadow, data->system_cfg->data_cfg, data->hid); // FALSE : No need to do verification-after-inserts, as that functionality // has been tested earlier in test_inserts() case. - uint64 num_inserts = splinter_do_inserts(data, spl, FALSE, &shadow); + uint64 num_inserts = splinter_do_inserts(data, &spl, FALSE, &shadow); ASSERT_NOT_EQUAL(0, num_inserts, "Expected to have inserted non-zero rows, num_inserts=%lu.", num_inserts); merge_accumulator qdata; - merge_accumulator_init(&qdata, spl->heap_id); + merge_accumulator_init(&qdata, spl.heap_id); DECLARE_AUTO_KEY_BUFFER(keybuf, data->hid); - const size_t key_size = core_max_key_size(spl); + const size_t key_size = core_max_key_size(&spl); platform_status rc; @@ -437,13 +441,13 @@ CTEST2(splinter, test_lookups) test_key(&keybuf, TEST_RANDOM, insert_num, 0, 0, key_size, 0); merge_accumulator_set_to_null(&qdata); - rc = core_lookup(spl, key_buffer_key(&keybuf), &qdata); + rc = core_lookup(&spl, key_buffer_key(&keybuf), &qdata); ASSERT_TRUE(SUCCESS(rc), "trunk_lookup() FAILURE, insert_num=%lu: %s\n", insert_num, platform_status_to_string(rc)); - verify_tuple(spl, + verify_tuple(&spl, &data->gen, insert_num, key_buffer_key(&keybuf), @@ -474,13 +478,13 @@ CTEST2(splinter, test_lookups) test_key(&keybuf, TEST_RANDOM, insert_num, 0, 0, key_size, 0); - rc = core_lookup(spl, key_buffer_key(&keybuf), &qdata); + rc = core_lookup(&spl, key_buffer_key(&keybuf), &qdata); ASSERT_TRUE(SUCCESS(rc), "trunk_lookup() FAILURE, insert_num=%lu: %s\n", insert_num, platform_status_to_string(rc)); - verify_tuple(spl, + verify_tuple(&spl, &data->gen, insert_num, key_buffer_key(&keybuf), @@ -513,7 +517,7 @@ CTEST2(splinter, test_lookups) // Range search uses the shadow-copy of the rows previously inserted while // doing a binary-search. rc = test_lookup_by_range( - (void *)data, spl, num_inserts, &shadow, num_ranges); + (void *)data, &spl, num_inserts, &shadow, num_ranges); ASSERT_TRUE(SUCCESS(rc), "test_lookup_by_range() FAILURE, num_ranges=%d: %s\n", num_ranges, @@ -547,14 +551,14 @@ CTEST2(splinter, test_lookups) num_inserts, "Verify async positive lookups %3lu%% complete"); - ctxt = test_async_ctxt_get(spl, async_lookup, &vtarg_true); + ctxt = test_async_ctxt_get(&spl, async_lookup, &vtarg_true); test_key(&ctxt->key, TEST_RANDOM, insert_num, 0, 0, key_size, 0); ctxt->lookup_num = insert_num; async_ctxt_submit( - spl, async_lookup, ctxt, NULL, verify_tuple_callback, &vtarg_true); + &spl, async_lookup, ctxt, NULL, verify_tuple_callback, &vtarg_true); } - test_wait_for_inflight(spl, async_lookup, &vtarg_true); + test_wait_for_inflight(&spl, async_lookup, &vtarg_true); elapsed_ns = platform_timestamp_elapsed(start_time); CTEST_LOG_INFO( @@ -579,13 +583,13 @@ CTEST2(splinter, test_lookups) num_inserts, "Verify async negative lookups %3lu%% complete"); - ctxt = test_async_ctxt_get(spl, async_lookup, &vtarg_false); + ctxt = test_async_ctxt_get(&spl, async_lookup, &vtarg_false); test_key(&ctxt->key, TEST_RANDOM, insert_num, 0, 0, key_size, 0); ctxt->lookup_num = insert_num; async_ctxt_submit( - spl, async_lookup, ctxt, NULL, verify_tuple_callback, &vtarg_false); + &spl, async_lookup, ctxt, NULL, verify_tuple_callback, &vtarg_false); } - test_wait_for_inflight(spl, async_lookup, &vtarg_false); + test_wait_for_inflight(&spl, async_lookup, &vtarg_false); elapsed_ns = platform_timestamp_elapsed(start_time); CTEST_LOG_INFO( @@ -598,7 +602,7 @@ CTEST2(splinter, test_lookups) async_ctxt_deinit(data->hid, async_lookup); } - core_destroy(spl); + core_destroy(&spl); trunk_shadow_deinit(&shadow); } @@ -617,15 +621,17 @@ CTEST2(splinter, test_splinter_print_diags) allocator *alp = (allocator *)&data->al; - core_handle *spl = core_create(&data->system_cfg->splinter_cfg, + core_handle spl; + platform_status rc = core_mkfs(&spl, + &data->system_cfg->splinter_cfg, alp, (cache *)data->clock_cache, &data->tasks, test_generate_allocator_root_id(), data->hid); - ASSERT_TRUE(spl != NULL); + ASSERT_TRUE(SUCCESS(rc)); - uint64 num_inserts = splinter_do_inserts(data, spl, FALSE, NULL); + uint64 num_inserts = splinter_do_inserts(data, &spl, FALSE, NULL); ASSERT_NOT_EQUAL(0, num_inserts, "Expected to have inserted non-zero rows, num_inserts=%lu", @@ -637,16 +643,16 @@ CTEST2(splinter, test_splinter_print_diags) __LINE__, __func__); - core_print_super_block(Platform_default_log_handle, spl); + core_print_super_block(Platform_default_log_handle, &spl); - core_print_space_use(Platform_default_log_handle, spl); + core_print_space_use(Platform_default_log_handle, &spl); CTEST_LOG_INFO("\n** Allocator stats **\n"); allocator_print_stats(alp); allocator_print_allocated(alp); set_log_streams_for_tests(MSG_LEVEL_INFO); - core_destroy(spl); + core_destroy(&spl); } /* diff --git a/tests/unit/splinterdb_stress_test.c b/tests/unit/splinterdb_stress_test.c index bb20f0c1..73d6580a 100644 --- a/tests/unit/splinterdb_stress_test.c +++ b/tests/unit/splinterdb_stress_test.c @@ -25,7 +25,7 @@ static void exec_worker_thread(void *w); static void -naive_range_delete(const splinterdb *kvsb, slice start_key, uint32 count); +naive_range_delete(splinterdb *kvsb, slice start_key, uint32 count); // Configuration for each worker thread typedef struct { @@ -309,7 +309,7 @@ exec_worker_thread(void *w) // Do a "range delete" by collecting keys and then deleting them one at a time static void -naive_range_delete(const splinterdb *kvsb, slice start_key, uint32 count) +naive_range_delete(splinterdb *kvsb, slice start_key, uint32 count) { CTEST_LOG_INFO("\tcollecting keys to delete...\n"); char *keys_to_delete = calloc(count, TEST_KEY_SIZE);