Commit 3c25dc75 authored by Barry Perlman's avatar Barry Perlman Committed by Yoni Fogel

[t:4050] #4050 Merge tokudb.4050 to merge, done with command svn merge...

[t:4050] #4050 Merge tokudb.4050 to merge, done with command svn merge -r36213:HEAD tokudb.4050 tokudb

git-svn-id: file:///svn/toku/tokudb@36808 c7de825b-a66e-492c-adef-691d508d4ae1
parent f977b6e7
......@@ -66,6 +66,7 @@ BRT_SOURCES = \
log_print \
logcursor \
memarena \
mempool \
minicron \
omt \
pqueue \
......
......@@ -26,6 +26,7 @@
#include "leafentry.h"
#include "block_table.h"
#include "c_dialects.h"
#include "mempool.h"
// Uncomment the following to use quicklz
......@@ -132,10 +133,12 @@ int toku_bnc_flush_to_child(
// data of an available partition of a leaf brtnode
struct brtnode_leaf_basement_node {
OMT buffer;
unsigned int n_bytes_in_buffer; /* How many bytes to represent the OMT (including the per-key overheads, but not including the overheads for the node. */
unsigned int seqinsert; /* number of sequential inserts to this leaf */
MSN max_msn_applied; // max message sequence number applied
OMT buffer; // pointers to individual leaf entries
struct mempool buffer_mempool; // storage for all leaf entries
unsigned int n_bytes_in_buffer; // How many bytes to represent the OMT (including the per-key overheads, ...
// ... but not including the overheads for the node.
unsigned int seqinsert; // number of sequential inserts to this leaf
MSN max_msn_applied; // max message sequence number applied
bool stale_ancestor_messages_applied;
};
......@@ -302,6 +305,7 @@ static inline void set_BSB(BRTNODE node, int i, SUB_BLOCK sb) {
#define BLB_MAX_MSN_APPLIED(node,i) (BLB(node,i)->max_msn_applied)
#define BLB_MAX_DSN_APPLIED(node,i) (BLB(node,i)->max_dsn_applied)
#define BLB_BUFFER(node,i) (BLB(node,i)->buffer)
#define BLB_BUFFER_MEMPOOL(node,i) (BLB(node,i)->buffer_mempool)
#define BLB_NBYTESINBUF(node,i) (BLB(node,i)->n_bytes_in_buffer)
#define BLB_SEQINSERT(node,i) (BLB(node,i)->seqinsert)
......@@ -683,6 +687,12 @@ int toku_cmd_leafval_heaviside (OMTVALUE leafentry, void *extra)
int toku_brt_root_put_cmd(BRT brt, BRT_MSG_S * cmd)
__attribute__((__warn_unused_result__));
void *mempool_malloc_from_omt(OMT omt, struct mempool *mp, size_t size, void **maybe_free);
// Effect: Allocate a new object of size SIZE in MP. If MP runs out of space, allocate new a new mempool space, and copy all the items
// from the OMT (which items refer to items in the old mempool) into the new mempool.
// If MAYBE_FREE is NULL then free the old mempool's space.
// Otherwise, store the old mempool's space in maybe_free.
int toku_verify_brtnode (BRT brt, MSN rootmsn, MSN parentmsn,
BLOCKNUM blocknum, int height, struct kv_pair *lesser_pivot, struct kv_pair *greatereq_pivot,
int (*progress_callback)(void *extra, float progress), void *extra,
......
......@@ -455,7 +455,6 @@ array_item (OMTVALUE lev, u_int32_t idx, void *vsi) {
struct sum_info {
unsigned int dsum;
unsigned int msum;
unsigned int count;
};
......@@ -464,8 +463,7 @@ sum_item (OMTVALUE lev, u_int32_t UU(idx), void *vsi) {
LEAFENTRY le=lev;
struct sum_info *si = vsi;
si->count++;
si->dsum += leafentry_disksize(le);
si->msum += leafentry_memsize(le);
si->dsum += leafentry_disksize(le); // TODO 4050 delete this redundant call and use le_sizes[]
return 0;
}
......@@ -479,34 +477,67 @@ rebalance_brtnode_leaf(BRTNODE node, unsigned int basementnodesize)
{
assert(node->height == 0);
assert(node->dirty);
// first create an array of OMTVALUE's that store all the data
u_int32_t num_le = 0;
for (int i = 0; i < node->n_children; i++) {
lazy_assert(BLB_BUFFER(node, i));
uint32_t num_orig_basements = node->n_children;
// Count number of leaf entries in this leaf (num_le).
u_int32_t num_le = 0;
for (uint32_t i = 0; i < num_orig_basements; i++) {
invariant(BLB_BUFFER(node, i));
num_le += toku_omt_size(BLB_BUFFER(node, i));
}
OMTVALUE *XMALLOC_N(num_le, array);
// creating array that will store id's of new pivots.
// allocating num_le of them is overkill, but num_le is an upper bound
u_int32_t *XMALLOC_N(num_le, new_pivots);
// now fill in the values into array
// TODO 4050 Maybe instead of using num_alloc and XCALLOC_N(),
// before every XMALLOC() test if num_le is zero,
// and if num_le is zero then XCALLOC_N() a single entry.
uint32_t num_alloc = num_le ? num_le : 1; // simplify logic below by always having at least one entry per array
// Create an array of OMTVALUE's that store all the pointers to all the data.
// Each element in leafpointers is a pointer to a leaf.
OMTVALUE *XCALLOC_N(num_alloc, leafpointers);
// Capture pointers to old mempools' buffers (so they can be destroyed)
void **XCALLOC_N(num_orig_basements, old_mempool_bases);
u_int32_t curr_le = 0;
for (int i = 0; i < node->n_children; i++) {
for (uint32_t i = 0; i < num_orig_basements; i++) {
OMT curr_omt = BLB_BUFFER(node, i);
struct array_info ai;
ai.offset = curr_le;
ai.array = array;
ai.offset = curr_le; // index of first le in basement
ai.array = leafpointers;
toku_omt_iterate(curr_omt, array_item, &ai);
curr_le += toku_omt_size(curr_omt);
BASEMENTNODE bn = BLB(node, i);
old_mempool_bases[i] = toku_mempool_get_base(&bn->buffer_mempool);
}
// figure out the new pivots
// Create an array that will store indexes of new pivots.
// Each element in new_pivots is the index of a pivot key.
// (Allocating num_le of them is overkill, but num_le is an upper bound.)
u_int32_t *XCALLOC_N(num_alloc, new_pivots);
// Each element in le_sizes is the size of the leafentry pointed to by leafpointers.
size_t *XCALLOC_N(num_alloc, le_sizes);
// Create an array that will store the size of each basement.
// This is the sum of the leaf sizes of all the leaves in that basement.
// We don't know how many basements there will be, so we use num_le as the upper bound.
size_t *XCALLOC_N(num_alloc, bn_sizes);
// TODO 4050 Maybe delete this as redundant, or maybe replace other accounting info.
// If we keep this, all these arrays should be combined into a single array of some bn_info struct (pivot, msize, num_les).
// Each entry is the number of leafentries in this basement. (Again, num_le is overkill upper bound.)
uint32_t *XCALLOC_N(num_alloc, num_les_this_bn);
// Figure out the new pivots.
// We need the index of each pivot, and for each basement we need
// the number of leaves and the sum of the sizes of the leaves (memory requirement for basement).
u_int32_t curr_pivot = 0;
u_int32_t num_le_in_curr_bn = 0;
u_int32_t bn_size_so_far = 0;
for (u_int32_t i = 0; i < num_le; i++) {
u_int32_t curr_size = leafentry_disksize(array[i]);
if ((bn_size_so_far + curr_size > basementnodesize) && (num_le_in_curr_bn != 0)) {
u_int32_t curr_le_size = leafentry_disksize(leafpointers[i]);
le_sizes[i] = curr_le_size;
if ((bn_size_so_far + curr_le_size > basementnodesize) && (num_le_in_curr_bn != 0)) {
// cap off the current basement node to end with the element before i
new_pivots[curr_pivot] = i-1;
curr_pivot++;
......@@ -514,8 +545,13 @@ rebalance_brtnode_leaf(BRTNODE node, unsigned int basementnodesize)
bn_size_so_far = 0;
}
num_le_in_curr_bn++;
bn_size_so_far += curr_size;
num_les_this_bn[curr_pivot] = num_le_in_curr_bn;
bn_size_so_far += curr_le_size;
bn_sizes[curr_pivot] = bn_size_so_far;
}
// curr_pivot is now the total number of pivot keys in the leaf node
int num_pivots = curr_pivot;
int num_children = num_pivots + 1;
// now we need to fill in the new basement nodes and pivots
......@@ -523,34 +559,34 @@ rebalance_brtnode_leaf(BRTNODE node, unsigned int basementnodesize)
// Need to figure out how to properly deal with seqinsert.
// I am not happy with how this is being
// handled with basement nodes
u_int32_t tmp_seqinsert = BLB_SEQINSERT(node, node->n_children-1);
u_int32_t tmp_seqinsert = BLB_SEQINSERT(node, num_orig_basements - 1);
// choose the max msn applied to any basement as the max msn applied to all new basements
MSN max_msn = MIN_MSN;
for (int i = 0; i < node->n_children; i++) {
for (uint32_t i = 0; i < num_orig_basements; i++) {
MSN curr_msn = BLB_MAX_MSN_APPLIED(node,i);
max_msn = (curr_msn.msn > max_msn.msn) ? curr_msn : max_msn;
}
// Now destroy the old stuff;
// Now destroy the old basements, but do not destroy leaves
toku_destroy_brtnode_internals(node);
// now reallocate pieces and start filling them in
int num_children = curr_pivot + 1;
assert(num_children > 0);
invariant(num_children > 0);
node->totalchildkeylens = 0;
XMALLOC_N(num_children-1, node->childkeys);
XCALLOC_N(num_pivots, node->childkeys); // allocate pointers to pivot structs
node->n_children = num_children;
XMALLOC_N(num_children, node->bp);
XCALLOC_N(num_children, node->bp); // allocate pointers to basements (bp)
for (int i = 0; i < num_children; i++) {
set_BLB(node, i, toku_create_empty_bn());
set_BLB(node, i, toku_create_empty_bn()); // allocate empty basements and set bp pointers
}
// now we start to fill in the data
// first the pivots
for (int i = 0; i < num_children-1; i++) {
LEAFENTRY curr_le_pivot = array[new_pivots[i]];
for (int i = 0; i < num_pivots; i++) {
LEAFENTRY curr_le_pivot = leafpointers[new_pivots[i]];
node->childkeys[i] = kv_pair_malloc(
le_key(curr_le_pivot),
le_keylen(curr_le_pivot),
......@@ -560,19 +596,39 @@ rebalance_brtnode_leaf(BRTNODE node, unsigned int basementnodesize)
assert(node->childkeys[i]);
node->totalchildkeylens += toku_brt_pivot_key_len(node->childkeys[i]);
}
uint32_t baseindex_this_bn = 0;
// now the basement nodes
for (int i = 0; i < num_children; i++) {
// put back seqinsert
BLB_SEQINSERT(node, i) = tmp_seqinsert;
// create start (inclusive) and end (exclusive) boundaries for data of basement node
u_int32_t curr_start = (i==0) ? 0 : new_pivots[i-1]+1;
u_int32_t curr_end = (i==num_children-1) ? num_le : new_pivots[i]+1;
u_int32_t num_in_bn = curr_end - curr_start;
u_int32_t curr_start = (i==0) ? 0 : new_pivots[i-1]+1; // index of first leaf in basement
u_int32_t curr_end = (i==num_pivots) ? num_le : new_pivots[i]+1; // index of first leaf in next basement
u_int32_t num_in_bn = curr_end - curr_start; // number of leaves in this basement
// create indexes for new basement
invariant(baseindex_this_bn == curr_start);
uint32_t num_les_to_copy = num_les_this_bn[i];
invariant(num_les_to_copy == num_in_bn);
// construct mempool for this basement
size_t size_this_bn = bn_sizes[i];
BASEMENTNODE bn = BLB(node, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_construct(mp, size_this_bn);
OMTVALUE *XCALLOC_N(num_in_bn, bn_array);
for (uint32_t le_index = 0; le_index < num_les_to_copy; le_index++) {
uint32_t le_within_node = baseindex_this_bn + le_index;
size_t le_size = le_sizes[le_within_node];
void * new_le = toku_mempool_malloc(mp, le_size, 1); // point to new location
void * old_le = leafpointers[le_within_node];
memcpy(new_le, old_le, le_size); // put le data at new location
bn_array[le_index] = new_le; // point to new location (in new mempool)
}
OMTVALUE *XMALLOC_N(num_in_bn, bn_array);
assert(bn_array);
memcpy(bn_array, &array[curr_start], num_in_bn*(sizeof(array[0])));
toku_omt_destroy(&BLB_BUFFER(node, i));
int r = toku_omt_create_steal_sorted_array(
&BLB_BUFFER(node, i),
......@@ -580,20 +636,31 @@ rebalance_brtnode_leaf(BRTNODE node, unsigned int basementnodesize)
num_in_bn,
num_in_bn
);
lazy_assert_zero(r);
struct sum_info sum_info = {0,0,0};
invariant_zero(r);
struct sum_info sum_info = {0,0};
toku_omt_iterate(BLB_BUFFER(node, i), sum_item, &sum_info);
BLB_NBYTESINBUF(node, i) = sum_info.dsum;
invariant(sum_info.dsum == size_this_bn);
BP_STATE(node,i) = PT_AVAIL;
BP_TOUCH_CLOCK(node,i);
BLB_MAX_MSN_APPLIED(node,i) = max_msn;
baseindex_this_bn += num_les_to_copy; // set to index of next bn
}
node->max_msn_applied_to_node_on_disk = max_msn;
toku_free(array);
// destroy buffers of old mempools
for (uint32_t i = 0; i < num_orig_basements; i++) {
toku_free(old_mempool_bases[i]);
}
toku_free(leafpointers);
toku_free(old_mempool_bases);
toku_free(new_pivots);
}
toku_free(le_sizes);
toku_free(bn_sizes);
toku_free(num_les_this_bn);
} // end of rebalance_brtnode_leaf()
static void
serialize_and_compress(BRTNODE node, int npartitions, struct sub_block sb[]);
......@@ -715,7 +782,7 @@ toku_serialize_brtnode_to_memory (BRTNODE node,
toku_assert_entire_node_in_memory(node);
if (node->height == 0) {
rebalance_brtnode_leaf(node, basementnodesize);
rebalance_brtnode_leaf(node, basementnodesize);
}
const int npartitions = node->n_children;
......@@ -961,6 +1028,7 @@ BASEMENTNODE toku_create_empty_bn_no_buffer(void) {
bn->n_bytes_in_buffer = 0;
bn->seqinsert = 0;
bn->stale_ancestor_messages_applied = false;
toku_mempool_zero(&bn->buffer_mempool);
return bn;
}
......@@ -1227,11 +1295,14 @@ static void setup_brtnode_partitions(BRTNODE node, struct brtnode_fetch_extra* b
}
}
/* deserialize the partition from the sub-block's uncompressed buffer
* and destroy the uncompressed buffer
*/
static void
deserialize_brtnode_partition(
struct sub_block *sb,
BRTNODE node,
int index,
int childnum, // which partition to deserialize
DESCRIPTOR desc,
brt_compare_func cmp
)
......@@ -1242,38 +1313,46 @@ deserialize_brtnode_partition(
// now with the data verified, we can read the information into the node
struct rbuf rb = {.buf = NULL, .size = 0, .ndone = 0};
rbuf_init(&rb, sb->uncompressed_ptr, data_size);
u_int32_t start_of_data;
unsigned char ch = rbuf_char(&rb);
if (node->height > 0) {
unsigned char ch = rbuf_char(&rb);
assert(ch == BRTNODE_PARTITION_FIFO_MSG);
deserialize_child_buffer(BNC(node, index), &rb, desc, cmp);
BP_WORKDONE(node, index) = 0;
deserialize_child_buffer(BNC(node, childnum), &rb, desc, cmp);
BP_WORKDONE(node, childnum) = 0;
}
else {
unsigned char ch = rbuf_char(&rb);
assert(ch == BRTNODE_PARTITION_OMT_LEAVES);
BLB_SEQINSERT(node, index) = 0;
u_int32_t num_entries = rbuf_int(&rb);
OMTVALUE *XMALLOC_N(num_entries, array);
start_of_data = rb.ndone;
for (u_int32_t i = 0; i < num_entries; i++) {
LEAFENTRY le = (LEAFENTRY)(&rb.buf[rb.ndone]);
BLB_SEQINSERT(node, childnum) = 0;
uint32_t num_entries = rbuf_int(&rb);
uint32_t start_of_data = rb.ndone; // index of first byte of first leafentry
data_size -= start_of_data; // remaining bytes of leafentry data
// TODO 3988 Count empty basements (data_size == 0)
if (data_size == 0) {
// printf("#### Deserialize empty basement, childnum = %d\n", childnum);
invariant_zero(num_entries);
}
OMTVALUE *XMALLOC_N(num_entries, array); // create array of pointers to leafentries
BASEMENTNODE bn = BLB(node, childnum);
toku_mempool_copy_construct(&bn->buffer_mempool, &rb.buf[rb.ndone], data_size);
uint8_t * le_base = toku_mempool_get_base(&bn->buffer_mempool); // point to first le in mempool
for (u_int32_t i = 0; i < num_entries; i++) { // now set up the pointers in the omt
LEAFENTRY le = (LEAFENTRY)&le_base[rb.ndone - start_of_data]; // point to durable mempool, not to transient rbuf
u_int32_t disksize = leafentry_disksize(le);
rb.ndone += disksize;
invariant(rb.ndone<=rb.size);
array[i]=toku_xmalloc(disksize);
assert(array[i]);
memcpy(array[i], le, disksize);
array[i]=(OMTVALUE)le;
}
u_int32_t end_of_data = rb.ndone;
BLB_NBYTESINBUF(node, index) += end_of_data-start_of_data;
// destroy old buffer that was created by toku_setup_basementnode, so we can create a new one
toku_omt_destroy(&BLB_BUFFER(node, index));
int r = toku_omt_create_steal_sorted_array(&BLB_BUFFER(node, index), &array, num_entries, num_entries);
assert(r == 0);
BLB_NBYTESINBUF(node, childnum) += end_of_data-start_of_data;
// destroy old omt (bn.buffer) that was created by toku_create_empty_bn(), so we can create a new one
toku_omt_destroy(&BLB_BUFFER(node, childnum));
int r = toku_omt_create_steal_sorted_array(&BLB_BUFFER(node, childnum), &array, num_entries, num_entries);
invariant_zero(r);
}
assert(rb.ndone == rb.size);
toku_free(sb->uncompressed_ptr);
}
static void
......@@ -1282,7 +1361,6 @@ decompress_and_deserialize_worker(struct rbuf curr_rbuf, struct sub_block curr_s
read_and_decompress_sub_block(&curr_rbuf, &curr_sb);
// at this point, sb->uncompressed_ptr stores the serialized node partition
deserialize_brtnode_partition(&curr_sb, node, child, desc, cmp);
toku_free(curr_sb.uncompressed_ptr);
}
static void
......@@ -1473,11 +1551,8 @@ deserialize_brtnode_from_rbuf(
}
node->layout_version_read_from_disk = rbuf_int(rb);
// TODO: (Zardosht), worry about upgrade
if (node->layout_version_read_from_disk != BRT_LAYOUT_VERSION) {
r = toku_db_badformat();
goto cleanup;
}
// TODO 4053
invariant(node->layout_version_read_from_disk == BRT_LAYOUT_VERSION);
node->layout_version = node->layout_version_read_from_disk;
node->layout_version_original = rbuf_int(rb);
node->build_id = rbuf_int(rb);
......@@ -1611,7 +1686,6 @@ toku_deserialize_bp_from_disk(BRTNODE node, int childnum, int fd, struct brtnode
read_and_decompress_sub_block(&rb, &curr_sb);
// at this point, sb->uncompressed_ptr stores the serialized node partition
deserialize_brtnode_partition(&curr_sb, node, childnum, &bfe->h->descriptor, bfe->h->compare_fun);
toku_free(curr_sb.uncompressed_ptr);
toku_free(raw_block);
}
......@@ -1635,7 +1709,6 @@ toku_deserialize_bp_from_compressed(BRTNODE node, int childnum,
curr_sb->compressed_size
);
deserialize_brtnode_partition(curr_sb, node, childnum, desc, cmp);
toku_free(curr_sb->uncompressed_ptr);
toku_free(curr_sb->compressed_ptr);
toku_free(curr_sb);
}
......@@ -1734,7 +1807,7 @@ toku_verify_or_set_counts (BRTNODE node) {
if (node->height==0) {
for (int i=0; i<node->n_children; i++) {
lazy_assert(BLB_BUFFER(node, i));
struct sum_info sum_info = {0,0,0};
struct sum_info sum_info = {0,0};
toku_omt_iterate(BLB_BUFFER(node, i), sum_item, &sum_info);
lazy_assert(sum_info.count==toku_omt_size(BLB_BUFFER(node, i)));
lazy_assert(sum_info.dsum==BLB_NBYTESINBUF(node, i));
......
......@@ -129,7 +129,7 @@ int toku_testsetup_insert_to_leaf (BRT brt, BLOCKNUM blocknum, char *key, int ke
toku_verify_or_set_counts(node);
assert(node->height==0);
size_t lesize, disksize;
size_t newlesize;
LEAFENTRY leafentry;
OMTVALUE storeddatav;
u_int32_t idx;
......@@ -139,8 +139,17 @@ int toku_testsetup_insert_to_leaf (BRT brt, BLOCKNUM blocknum, char *key, int ke
.u.id={toku_fill_dbt(&keydbt, key, keylen),
toku_fill_dbt(&valdbt, val, vallen)}};
//Generate a leafentry (committed insert key,val)
uint childnum = toku_brtnode_which_child(node,
&keydbt,
&brt->h->descriptor, brt->compare_fun);
BASEMENTNODE bn = BLB(node, childnum);
void * maybe_free = 0;
r = apply_msg_to_leafentry(&cmd, NULL, //No old leafentry
&lesize, &disksize, &leafentry,
&newlesize, &leafentry,
bn->buffer, &bn->buffer_mempool, &maybe_free,
NULL, NULL);
assert(r==0);
......@@ -163,7 +172,7 @@ int toku_testsetup_insert_to_leaf (BRT brt, BLOCKNUM blocknum, char *key, int ke
// hack to get tests passing. These tests should not be directly inserting into buffers
BLB(node, 0)->max_msn_applied = msn;
BLB_NBYTESINBUF(node, 0) += disksize;
BLB_NBYTESINBUF(node, 0) += newlesize;
node->dirty=1;
......
......@@ -103,6 +103,7 @@ Split_or_merge (node, childnum) {
#include "includes.h"
#include "checkpoint.h"
#include "mempool.h"
// Access to nested transaction logic
#include "ule.h"
#include "xids.h"
......@@ -569,7 +570,11 @@ brtnode_memory_size (BRTNODE node)
else {
BASEMENTNODE bn = BLB(node, i);
retval += sizeof(*bn);
retval += BLB_NBYTESINBUF(node,i);
{
size_t poolsize = toku_mempool_get_size(&bn->buffer_mempool); // include fragmentation overhead
invariant (poolsize >= BLB_NBYTESINBUF(node,i));
retval += poolsize;
}
OMT curr_omt = BLB_BUFFER(node, i);
retval += (toku_omt_memory_size(curr_omt));
}
......@@ -825,8 +830,8 @@ int toku_brtnode_pe_callback (void *brtnode_pv, PAIR_ATTR UU(old_attr), PAIR_ATT
if (BP_SHOULD_EVICT(node,i)) {
// free the basement node
BASEMENTNODE bn = BLB(node, i);
OMT curr_omt = BLB_BUFFER(node, i);
toku_omt_free_items(curr_omt);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(bn);
set_BNULL(node,i);
BP_STATE(node,i) = PT_ON_DISK;
......@@ -1046,6 +1051,28 @@ brt_compare_pivot(DESCRIPTOR desc, brt_compare_func cmp, const DBT *key, bytevec
return r;
}
static int
verify_in_mempool (OMTVALUE lev, u_int32_t UU(idx), void *vmp)
{
LEAFENTRY le=lev;
struct mempool *mp=vmp;
lazy_assert(toku_mempool_inrange(mp, le, leafentry_memsize(le)));
return 0;
}
static void
verify_all_in_mempool (BRTNODE node)
{
if (node->height==0) {
for (int i = 0; i < node->n_children; i++) {
invariant(BP_STATE(node,i) == PT_AVAIL);
BASEMENTNODE bn = BLB(node, i);
toku_omt_iterate(bn->buffer, verify_in_mempool, &bn->buffer_mempool);
}
}
}
// destroys the internals of the brtnode, but it does not free the values
// that are stored
// this is common functionality for toku_brtnode_free and rebalance_brtnode_leaf
......@@ -1089,8 +1116,8 @@ void toku_brtnode_free (BRTNODE *nodep) {
if (node->height == 0) {
for (int i = 0; i < node->n_children; i++) {
if (BP_STATE(node,i) == PT_AVAIL) {
OMT curr_omt = BLB_BUFFER(node, i);
toku_omt_free_items(curr_omt);
struct mempool * mp = &(BLB_BUFFER_MEMPOOL(node, i));
toku_mempool_destroy(mp);
}
}
}
......@@ -1399,8 +1426,8 @@ static void
brtleaf_get_split_loc(
BRTNODE node,
u_int64_t sumlesizes,
int* bn_index,
int* le_index
int* bn_index, // which basement within leaf
int* le_index // which key within basement
)
// Effect: Find the location within a leaf node where we want to perform a split
// bn_index is which basement node (which OMT) should be split.
......@@ -1450,8 +1477,8 @@ brtleaf_get_split_loc(
// brtleaf_split
static void
move_leafentries(
OMT* dest_omt,
OMT src_omt,
BASEMENTNODE dest_bn,
BASEMENTNODE src_bn,
u_int32_t lbi, //lower bound inclusive
u_int32_t ube, //upper bound exclusive
u_int32_t* num_bytes_moved
......@@ -1459,34 +1486,50 @@ move_leafentries(
//Effect: move leafentries in the range [lbi, upe) from src_omt to newly created dest_omt
{
assert(lbi < ube);
OMTVALUE *XMALLOC_N(ube-lbi, new_le);
OMTVALUE *XMALLOC_N(ube-lbi, newleafpointers); // create new omt
size_t mpsize = toku_mempool_get_used_space(&src_bn->buffer_mempool); // overkill, but safe
struct mempool * dest_mp = &dest_bn->buffer_mempool;
struct mempool * src_mp = &src_bn->buffer_mempool;
toku_mempool_construct(dest_mp, mpsize);
u_int32_t i = 0;
*num_bytes_moved = 0;
for (i = lbi; i < ube; i++) {
LEAFENTRY curr_le = NULL;
curr_le = fetch_from_buf(src_omt, i);
curr_le = fetch_from_buf(src_bn->buffer, i);
size_t le_size = leafentry_memsize(curr_le);
*num_bytes_moved += leafentry_disksize(curr_le);
new_le[i-lbi] = curr_le;
LEAFENTRY new_le = toku_mempool_malloc(dest_mp, le_size, 1);
memcpy(new_le, curr_le, le_size);
newleafpointers[i-lbi] = new_le;
toku_mempool_mfree(src_mp, curr_le, le_size);
}
int r = toku_omt_create_steal_sorted_array(
dest_omt,
&new_le,
&dest_bn->buffer,
&newleafpointers,
ube-lbi,
ube-lbi
);
assert_zero(r);
// now remove the elements from src_omt
for (i=ube-1; i >= lbi; i--) {
toku_omt_delete_at(src_omt,i);
toku_omt_delete_at(src_bn->buffer,i);
}
}
void
brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *nodeb, DBT *splitk, BOOL create_new_node, u_int32_t num_dependent_nodes, BRTNODE* dependent_nodes)
// Effect: Split a leaf node.
// Argument "node" is node to be split.
// Upon return:
// nodea and nodeb point to new nodes that result from split of "node"
// nodea is the left node that results from the split
// splitk is the right-most key of nodea
{
// printf("###### brtleaf_split(): create_new_node = %d, num_dependent_nodes = %d\n", create_new_node, num_dependent_nodes);
BRTNODE B;
u_int32_t fullhash;
......@@ -1495,12 +1538,12 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
// put value in cachetable and do checkpointing
// of dependent nodes
//
// We do this here, before evaluating the split_node
// and split_at_in_node because this operation
// We do this here, before evaluating the last_bn_on_left
// and last_le_on_left_within_bn because this operation
// may write to disk the dependent nodes.
// While doing so, we may rebalance the leaf node
// we are splitting, thereby invalidating the
// values of split_node and split_at_in_node.
// values of last_bn_on_left and last_le_on_left_within_bn.
// So, we must call this before evaluating
// those two values
cachetable_put_empty_node_with_dep_nodes(
......@@ -1518,6 +1561,7 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
assert(node->height==0);
assert(node->nodesize>0);
toku_assert_entire_node_in_memory(node);
verify_all_in_mempool(node);
MSN max_msn_applied_to_node = node->max_msn_applied_to_node_on_disk;
//printf("%s:%d A is at %lld\n", __FILE__, __LINE__, A->thisnodename);
......@@ -1525,9 +1569,9 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
// variables that say where we will do the split. We do it in the basement node indexed at
// at split_node, and at the index split_at_in_node within that basement node.
int split_node = 0;
int split_at_in_node = 0;
// at last_bn_on_left and at the index last_le_on_left_within_bn within that basement node.
int last_bn_on_left = 0; // last_bn_on_left may or may not be fully included
int last_le_on_left_within_bn = 0;
{
{
// TODO: (Zardosht) see if we can/should make this faster, we iterate over the rows twice
......@@ -1538,23 +1582,26 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
brtleaf_get_split_loc(
node,
sumlesizes,
&split_node,
&split_at_in_node
&last_bn_on_left,
&last_le_on_left_within_bn
);
}
// did we split right on the boundary between basement nodes?
BOOL split_on_boundary = (split_at_in_node == ((int) toku_omt_size(BLB_BUFFER(node, split_node)) - 1));
BOOL split_on_boundary = (last_le_on_left_within_bn == ((int) toku_omt_size(BLB_BUFFER(node, last_bn_on_left)) - 1));
// Now we know where we are going to break it
// the two nodes will have a total of n_children+1 basement nodes
// and n_children-1 pivots
// the left node, node, will have split_node+1 basement nodes
// the right node, B, will have n_children-split_node basement nodes
// the pivots of node will be the first split_node pivots that originally exist
// the pivots of B will be the last (n_children - 1 - split_node) pivots that originally exist
// the left node, node, will have last_bn_on_left+1 basement nodes
// the right node, B, will have n_children-last_bn_on_left basement nodes
// the pivots of node will be the first last_bn_on_left pivots that originally exist
// the pivots of B will be the last (n_children - 1 - last_bn_on_left) pivots that originally exist
// Note: The basements will not be rebalanced. Only the mempool of the basement that is split
// (if split_on_boundary is false) will be affected. All other mempools will remain intact. ???
//set up the basement nodes in the new node
int num_children_in_node = split_node + 1;
int num_children_in_b = node->n_children - split_node - (split_on_boundary ? 1 : 0);
int num_children_in_node = last_bn_on_left + 1;
int num_children_in_b = node->n_children - last_bn_on_left - (split_on_boundary ? 1 : 0);
if (create_new_node) {
toku_initialize_empty_brtnode(
B,
......@@ -1586,23 +1633,22 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
// first move all the data
//
int curr_src_bn_index = split_node;
int curr_src_bn_index = last_bn_on_left;
int curr_dest_bn_index = 0;
// handle the move of a subset of data in split_node from node to B
// handle the move of a subset of data in last_bn_on_left from node to B
if (!split_on_boundary) {
BP_STATE(B,curr_dest_bn_index) = PT_AVAIL;
u_int32_t diff_size = 0;
destroy_basement_node (BLB(B, curr_dest_bn_index)); // Destroy B's empty OMT, so I can rebuild it from an array
set_BNULL(B, curr_dest_bn_index);
set_BLB(B, curr_dest_bn_index, toku_create_empty_bn_no_buffer());
move_leafentries(
&BLB_BUFFER(B, curr_dest_bn_index),
BLB_BUFFER(node, curr_src_bn_index),
split_at_in_node+1,
toku_omt_size(BLB_BUFFER(node, curr_src_bn_index)),
&diff_size
);
move_leafentries(BLB(B, curr_dest_bn_index),
BLB(node, curr_src_bn_index),
last_le_on_left_within_bn+1, // first row to be moved to B
toku_omt_size(BLB_BUFFER(node, curr_src_bn_index)), // number of rows in basement to be split
&diff_size
);
BLB_NBYTESINBUF(node, curr_src_bn_index) -= diff_size;
BLB_NBYTESINBUF(B, curr_dest_bn_index) += diff_size;
curr_dest_bn_index++;
......@@ -1624,7 +1670,7 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
// the child index in the original node that corresponds to the
// first node in the right node of the split
int base_index = (split_on_boundary ? split_node + 1 : split_node);
int base_index = (split_on_boundary ? last_bn_on_left + 1 : last_bn_on_left);
// make pivots in B
for (int i=0; i < num_children_in_b-1; i++) {
B->childkeys[i] = node->childkeys[i+base_index];
......@@ -1635,7 +1681,7 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
if (split_on_boundary) {
// destroy the extra childkey between the nodes, we'll
// recreate it in splitk below
toku_free(node->childkeys[split_node]);
toku_free(node->childkeys[last_bn_on_left]);
}
REALLOC_N(num_children_in_node, node->bp);
REALLOC_N(num_children_in_node-1, node->childkeys);
......@@ -1644,7 +1690,7 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
if (splitk) {
memset(splitk, 0, sizeof *splitk);
OMTVALUE lev = 0;
int r=toku_omt_fetch(BLB_BUFFER(node, split_node), toku_omt_size(BLB_BUFFER(node, split_node))-1, &lev);
int r=toku_omt_fetch(BLB_BUFFER(node, last_bn_on_left), toku_omt_size(BLB_BUFFER(node, last_bn_on_left))-1, &lev);
assert_zero(r); // that fetch should have worked.
LEAFENTRY le=lev;
splitk->size = le_keylen(le);
......@@ -1658,6 +1704,9 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
node->dirty = 1;
B->dirty = 1;
verify_all_in_mempool(node);
verify_all_in_mempool(B);
*nodea = node;
*nodeb = B;
......@@ -1666,7 +1715,10 @@ brtleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *node
// B ->thisnodename.b, toku_serialize_brtnode_size(B ), B ->height==0 ? (int)(toku_omt_size(B ->u.l.buffer)) : -1, B->dirty);
//toku_dump_brtnode(t, node->thisnodename, 0, NULL, 0, NULL, 0);
//toku_dump_brtnode(t, B ->thisnodename, 0, NULL, 0, NULL, 0);
}
} // end of brtleaf_split()
static void
brt_nonleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *nodeb, DBT *splitk, u_int32_t num_dependent_nodes, BRTNODE* dependent_nodes)
......@@ -1743,6 +1795,9 @@ brt_nonleaf_split (struct brt_header* h, BRTNODE node, BRTNODE *nodea, BRTNODE *
*nodeb = B;
}
/* NODE is a node with a child.
* childnum was split into two nodes childa, and childb. childa is the same as the original child. childb is a new child.
* We must slide things around, & move things from the old table to the new tables.
......@@ -1931,6 +1986,7 @@ brt_leaf_delete_leafentry (
bn->n_bytes_in_buffer -= leafentry_disksize(le);
toku_mempool_mfree(&bn->buffer_mempool, 0, leafentry_memsize(le)); // Must pass 0, since le is no good any more.
}
void
......@@ -1950,49 +2006,50 @@ brt_leaf_apply_cmd_once (
{
// brt_leaf_check_leaf_stats(node);
size_t newlen=0, newdisksize=0, oldsize=0, workdone_this_le=0;
size_t newsize=0, oldsize=0, workdone_this_le=0;
LEAFENTRY new_le=0;
void *maybe_free = 0;
if (le)
oldsize = leafentry_memsize(le);
// This function may call mempool_malloc_dont_release() to allocate more space.
// That means the old pointers are guaranteed to still be good, but the data may have been copied into a new mempool.
// We'll have to release the old mempool later.
// This function may call mempool_malloc_from_omt() to allocate more space.
// That means le is guaranteed to not cause a sigsegv but it may point to a mempool that is
// no longer in use. We'll have to release the old mempool later.
{
int r = apply_msg_to_leafentry(cmd, le, &newlen, &newdisksize, &new_le, snapshot_txnids, live_list_reverse);
assert(r==0);
int r = apply_msg_to_leafentry(cmd, le, &newsize, &new_le, bn->buffer, &bn->buffer_mempool, &maybe_free, snapshot_txnids, live_list_reverse);
invariant(r==0);
}
if (new_le) assert(newdisksize == leafentry_disksize(new_le));
if (new_le) assert(newsize == leafentry_disksize(new_le));
if (le && new_le) {
// If we are replacing a leafentry, then the counts on the estimates remain unchanged, but the size might change
bn->n_bytes_in_buffer -= leafentry_disksize(le);
//printf("%s:%d Added %u-%u got %lu\n", __FILE__, __LINE__, le_keylen(new_le), le_latest_vallen(le), node->u.l.leaf_stats.dsize);
// the ndata and nkeys remains unchanged
bn->n_bytes_in_buffer += newdisksize;
{ int r = toku_omt_set_at(bn->buffer, new_le, idx); assert(r==0); }
toku_free(le);
bn->n_bytes_in_buffer -= oldsize;
bn->n_bytes_in_buffer += newsize;
// This mfree must occur after the mempool_malloc so that when the mempool is compressed everything is accounted for.
// But we must compute the size before doing the mempool mfree because otherwise the le pointer is no good.
toku_mempool_mfree(&bn->buffer_mempool, 0, oldsize); // Must pass 0, since le may be no good any more.
{
int r = toku_omt_set_at(bn->buffer, new_le, idx);
invariant(r==0);
}
workdone_this_le = (oldsize > newlen ? oldsize : newlen); // work done is max of le size before and after message application
workdone_this_le = (oldsize > newsize ? oldsize : newsize); // work done is max of le size before and after message application
} else {
if (le) {
} else { // we did not just replace a row, so ...
if (le) {
// ... we just deleted a row ...
// It was there, note that it's gone and remove it from the mempool
brt_leaf_delete_leafentry (bn, idx, le);
toku_free(le);
workdone_this_le = oldsize;
}
if (new_le) {
if (new_le) {
// ... or we just added a row
int r = toku_omt_insert_at(bn->buffer, new_le, idx);
assert(r==0);
bn->n_bytes_in_buffer += newdisksize;
workdone_this_le = newlen;
invariant(r==0);
bn->n_bytes_in_buffer += newsize;
workdone_this_le = newsize;
}
}
if (workdone) { // test programs may call with NULL
......@@ -2001,6 +2058,9 @@ brt_leaf_apply_cmd_once (
brt_status.max_workdone = *workdone;
}
// if we created a new mempool buffer, free the old one
if (maybe_free) toku_free(maybe_free);
// brt_leaf_check_leaf_stats(node);
}
......@@ -2422,7 +2482,10 @@ static void brt_nonleaf_cmd_once_to_child (brt_compare_func compare_fun, DESCRIP
toku_brt_append_to_child_buffer(compare_fun, desc, node, childnum, cmd->type, cmd->msn, cmd->xids, is_fresh, cmd->u.id.key, cmd->u.id.val);
}
/* find the leftmost child that may contain the key */
/* Find the leftmost child that may contain the key.
* If the key exists it will be in the child whose number
* is the return value of this function.
*/
unsigned int toku_brtnode_which_child(BRTNODE node, const DBT *k,
DESCRIPTOR desc, brt_compare_func cmp) {
#define DO_PIVOT_SEARCH_LR 0
......@@ -2576,7 +2639,16 @@ merge_leaf_nodes (BRTNODE a, BRTNODE b) {
// move the estimates
int num_children = a->n_children + b->n_children;
if (!a_has_tail) {
destroy_basement_node(BLB(a, a->n_children-1));
uint lastchild = a->n_children-1;
BASEMENTNODE bn = BLB(a, lastchild);
{
// verify that last basement in a is empty, then destroy mempool
struct mempool * mp = &bn->buffer_mempool;
size_t used_space = toku_mempool_get_used_space(mp);
invariant_zero(used_space);
toku_mempool_destroy(mp);
}
destroy_basement_node(bn);
set_BNULL(a, a->n_children-1);
num_children--;
}
......@@ -3047,8 +3119,8 @@ maybe_destroy_child_blbs(BRTNODE node, BRTNODE child)
if (BP_STATE(child, i) == PT_AVAIL &&
node->max_msn_applied_to_node_on_disk.msn < BLB_MAX_MSN_APPLIED(child, i).msn) {
BASEMENTNODE bn = BLB(child, i);
OMT curr_omt = BLB_BUFFER(child, i);
toku_omt_free_items(curr_omt);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(bn);
set_BNULL(child,i);
BP_STATE(child,i) = PT_ON_DISK;
......@@ -4047,6 +4119,58 @@ toku_brt_send_delete(BRT brt, DBT *key, XIDS xids) {
return result;
}
/* mempool support */
struct omt_compressor_state {
struct mempool *new_kvspace;
OMT omt;
};
static int move_it (OMTVALUE lev, u_int32_t idx, void *v) {
LEAFENTRY le=lev;
struct omt_compressor_state *oc = v;
u_int32_t size = leafentry_memsize(le);
LEAFENTRY newdata = toku_mempool_malloc(oc->new_kvspace, size, 1);
lazy_assert(newdata); // we do this on a fresh mempool, so nothing bad shouldhapepn
memcpy(newdata, le, size);
toku_omt_set_at(oc->omt, newdata, idx);
return 0;
}
// Compress things, and grow the mempool if needed.
// TODO 4092 should copy data to new memory, then call toku_mempool_destory() followed by toku_mempool_init()
static int omt_compress_kvspace (OMT omt, struct mempool *memp, size_t added_size, void **maybe_free) {
u_int32_t total_size_needed = memp->free_offset-memp->frag_size + added_size;
if (total_size_needed+total_size_needed/4 >= memp->size) {
memp->size = total_size_needed+total_size_needed/4;
}
void *newmem = toku_xmalloc(memp->size);
struct mempool new_kvspace;
toku_mempool_init(&new_kvspace, newmem, memp->size);
struct omt_compressor_state oc = { &new_kvspace, omt };
toku_omt_iterate(omt, move_it, &oc);
if (maybe_free) {
*maybe_free = memp->base;
} else {
toku_free(memp->base);
}
*memp = new_kvspace;
return 0;
}
void *
mempool_malloc_from_omt(OMT omt, struct mempool *mp, size_t size, void **maybe_free) {
void *v = toku_mempool_malloc(mp, size, 1);
if (v==0) {
if (0 == omt_compress_kvspace(omt, mp, size, maybe_free)) {
v = toku_mempool_malloc(mp, size, 1);
lazy_assert(v);
}
}
return v;
}
/* ******************** open,close and create ********************** */
// Test only function (not used in running system). This one has no env
......
......@@ -168,7 +168,6 @@ leafentry_disksize_13(LEAFENTRY_13 le);
int
toku_le_upgrade_13_14(LEAFENTRY_13 old_leafentry, // NULL if there was no stored data.
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY *new_leafentry_p);
......
/* -*- mode: C; c-basic-offset: 4 -*- */
#ident "$Id: mempool.c 19902 2010-05-06 20:41:32Z bkuszmaul $"
#ident "Copyright (c) 2007-2010 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
#include "includes.h"
/* Contract:
* Caller allocates mempool struct as convenient for caller, but memory used for data storage
* must be dynamically allocated via toku_malloc().
* Caller dynamically allocates memory for mempool and initializes mempool by calling toku_mempool_init().
* Once a buffer is assigned to a mempool (via toku_mempool_init()), the mempool owns it and
* is responsible for destroying it when the mempool is destroyed.
* Caller destroys mempool by calling toku_mempool_destroy().
*
* Note, toku_mempool_init() does not allocate the memory because sometimes the caller will already have
* the memory allocated and will assign the pre-allocated memory to the mempool.
*/
/* This is a constructor to be used when the memory for the mempool struct has been
* allocated by the caller, but no memory has yet been allocatd for the data.
*/
void toku_mempool_zero(struct mempool *mp) {
// printf("mempool_zero %p\n", mp);
memset(mp, 0, sizeof(*mp));
}
/* Copy constructor. Any time a new mempool is needed, allocate 1/4 more space
* than is currently needed.
*/
void toku_mempool_copy_construct(struct mempool *mp, const void * const data_source, const size_t data_size) {
// printf("mempool_copy %p %p %lu\n", mp, data_source, data_size);
if (data_size) {
invariant(data_source);
toku_mempool_construct(mp, data_size);
memcpy(mp->base, data_source, data_size);
mp->free_offset = data_size; // address of first available memory for new data
}
else {
toku_mempool_zero(mp);
// fprintf(stderr, "Empty mempool created (copy constructor)\n");
}
}
// TODO 4050 this is dirty, try to replace all uses of this
void toku_mempool_init(struct mempool *mp, void *base, size_t size) {
// printf("mempool_init %p %p %lu\n", mp, base, size);
invariant(base != 0);
invariant(size < (1U<<31)); // used to be assert(size >= 0), but changed to size_t so now let's make sure it's not more than 2GB...
mp->base = base;
mp->size = size;
mp->free_offset = 0; // address of first available memory
mp->frag_size = 0; // byte count of wasted space (formerly used, no longer used or available)
}
/* allocate memory and construct mempool
*/
void toku_mempool_construct(struct mempool *mp, size_t data_size) {
if (data_size) {
size_t mpsize = data_size + (data_size/4); // allow 1/4 room for expansion (would be wasted if read-only)
mp->base = toku_xmalloc(mpsize); // allocate buffer for mempool
mp->size = mpsize;
mp->free_offset = 0; // address of first available memory for new data
mp->frag_size = 0; // all allocated space is now in use
}
else {
toku_mempool_zero(mp);
// fprintf(stderr, "Empty mempool created (base constructor)\n");
}
}
void toku_mempool_destroy(struct mempool *mp) {
// printf("mempool_destroy %p %p %lu %lu\n", mp, mp->base, mp->size, mp->frag_size);
if (mp->base)
toku_free(mp->base);
toku_mempool_zero(mp);
}
void *toku_mempool_get_base(struct mempool *mp) {
return mp->base;
}
size_t toku_mempool_get_size(struct mempool *mp) {
return mp->size;
}
size_t toku_mempool_get_frag_size(struct mempool *mp) {
return mp->frag_size;
}
size_t toku_mempool_get_used_space(struct mempool *mp) {
return mp->free_offset - mp->frag_size;
}
size_t toku_mempool_get_free_space(struct mempool *mp) {
return mp->size - mp->free_offset;
}
void *toku_mempool_malloc(struct mempool *mp, size_t size, int alignment) {
invariant(size < (1U<<31));
invariant(mp->size < (1U<<31));
invariant(mp->free_offset < (1U<<31));
assert(mp->free_offset <= mp->size);
void *vp;
size_t offset = (mp->free_offset + (alignment-1)) & ~(alignment-1);
//printf("mempool_malloc size=%ld base=%p free_offset=%ld mp->size=%ld offset=%ld\n", size, mp->base, mp->free_offset, mp->size, offset);
if (offset + size > mp->size) {
vp = 0;
} else {
vp = (char *)mp->base + offset;
mp->free_offset = offset + size;
}
assert(mp->free_offset <= mp->size);
assert(((long)vp & (alignment-1)) == 0);
assert(vp == 0 || toku_mempool_inrange(mp, vp, size));
//printf("mempool returning %p\n", vp);
return vp;
}
// if vp is null then we are freeing something, but not specifying what. The data won't be freed until compression is done.
void toku_mempool_mfree(struct mempool *mp, void *vp, size_t size) {
if (vp) assert(toku_mempool_inrange(mp, vp, size));
mp->frag_size += size;
assert(mp->frag_size <= mp->size);
}
#ifndef _TOKU_MEMPOOL_H
#define _TOKU_MEMPOOL_H
#ident "$Id: mempool.h 19902 2010-05-06 20:41:32Z bkuszmaul $"
#ident "Copyright (c) 2007-2010 Tokutek Inc. All rights reserved."
#ident "The technology is licensed by the Massachusetts Institute of Technology, Rutgers State University of New Jersey, and the Research Foundation of State University of New York at Stony Brook under United States of America Serial No. 11/760379 and to the patents and/or patent applications resulting from it."
/* a memory pool is a contiguous region of memory that supports single
allocations from the pool. these allocated regions are never recycled.
when the memory pool no longer has free space, the allocated chunks
must be relocated by the application to a new memory pool. */
#include <sys/types.h>
#if defined(__cplusplus) || defined(__cilkplusplus)
extern "C" {
#endif
struct mempool;
// TODO 4050 Hide mempool struct internals from callers
struct mempool {
void *base; /* the base address of the memory */
size_t free_offset; /* the offset of the memory pool free space */
size_t size; /* the size of the memory */
size_t frag_size; /* the size of the fragmented memory */
};
/* This is a constructor to be used when the memory for the mempool struct has been
* allocated by the caller, but no memory has yet been allocatd for the data.
*/
void toku_mempool_zero(struct mempool *mp);
/* Copy constructor. Fill in empty mempool struct with new values, allocating
* a new buffer and filling the buffer with data from from data_source.
* Any time a new mempool is needed, allocate 1/4 more space
* than is currently needed.
*/
void toku_mempool_copy_construct(struct mempool *mp, const void * const data_source, const size_t data_size);
/* initialize the memory pool with the base address and size of a
contiguous chunk of memory */
void toku_mempool_init(struct mempool *mp, void *base, size_t size);
/* allocate memory and construct mempool
*/
void toku_mempool_construct(struct mempool *mp, size_t data_size);
/* destroy the memory pool */
void toku_mempool_destroy(struct mempool *mp);
/* get the base address of the memory pool */
void *toku_mempool_get_base(struct mempool *mp);
/* get the size of the memory pool */
size_t toku_mempool_get_size(struct mempool *mp);
/* get the amount of fragmented (wasted) space in the memory pool */
size_t toku_mempool_get_frag_size(struct mempool *mp);
/* get the amount of space that is holding useful data */
size_t toku_mempool_get_used_space(struct mempool *mp);
/* get the amount of space that is available for new data */
size_t toku_mempool_get_free_space(struct mempool *mp);
/* allocate a chunk of memory from the memory pool suitably aligned */
void *toku_mempool_malloc(struct mempool *mp, size_t size, int alignment);
/* free a previously allocated chunk of memory. the free only updates
a count of the amount of free space in the memory pool. the memory
pool does not keep track of the locations of the free chunks */
void toku_mempool_mfree(struct mempool *mp, void *vp, size_t size);
/* verify that a memory range is contained within a mempool */
static inline int toku_mempool_inrange(struct mempool *mp, void *vp, size_t size) {
return (mp->base <= vp) && ((char *)vp + size <= (char *)mp->base + mp->size);
}
#if defined(__cplusplus) || defined(__cilkplusplus)
};
#endif
#endif
......@@ -24,6 +24,7 @@ static int omt_int_cmp(OMTVALUE p, void *q)
else { return 0; }
}
static int omt_cmp(OMTVALUE p, void *q)
{
LEAFENTRY a = p, b = q;
......@@ -41,26 +42,35 @@ static int omt_cmp(OMTVALUE p, void *q)
else { return 0; }
}
static size_t
calc_le_size(int keylen, int vallen) {
size_t rval;
LEAFENTRY le;
rval = sizeof(le->type) + sizeof(le->keylen) + sizeof(le->u.clean.vallen) + keylen + vallen;
return rval;
}
static LEAFENTRY
le_fastmalloc(char *key, int keylen, char *val, int vallen)
le_fastmalloc(struct mempool * mp, char *key, int keylen, char *val, int vallen)
{
LEAFENTRY r = toku_malloc(sizeof(r->type) + sizeof(r->keylen) + sizeof(r->u.clean.vallen) +
keylen + vallen);
resource_assert(r);
r->type = LE_CLEAN;
r->keylen = keylen;
r->u.clean.vallen = vallen;
memcpy(&r->u.clean.key_val[0], key, keylen);
memcpy(&r->u.clean.key_val[keylen], val, vallen);
return r;
LEAFENTRY le;
size_t le_size = calc_le_size(keylen, vallen);
le = toku_mempool_malloc(mp, le_size, 1);
resource_assert(le);
le->type = LE_CLEAN;
le->keylen = keylen;
le->u.clean.vallen = vallen;
memcpy(&le->u.clean.key_val[0], key, keylen);
memcpy(&le->u.clean.key_val[keylen], val, vallen);
return le;
}
static LEAFENTRY
le_malloc(char *key, char *val)
le_malloc(struct mempool * mp, char *key, char *val)
{
int keylen = strlen(key) + 1;
int vallen = strlen(val) + 1;
return le_fastmalloc(key, keylen, val, vallen);
return le_fastmalloc(mp, key, keylen, val, vallen);
}
struct check_leafentries_struct {
......@@ -193,10 +203,6 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft) {
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
LEAFENTRY elts[3];
elts[0] = le_malloc("a", "aval");
elts[1] = le_malloc("b", "bval");
elts[2] = le_malloc("x", "xval");
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(1, sn.childkeys);
sn.childkeys[0] = kv_pair_malloc("b", 2, 0, 0);
......@@ -205,9 +211,21 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft) {
BP_STATE(&sn,1) = PT_AVAIL;
set_BLB(&sn, 0, toku_create_empty_bn());
set_BLB(&sn, 1, toku_create_empty_bn());
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
LEAFENTRY elts[3];
{
BASEMENTNODE bn = BLB(&sn,0);
struct mempool * mp0 = &bn->buffer_mempool;
bn = BLB(&sn,1);
struct mempool * mp1 = &bn->buffer_mempool;
toku_mempool_construct(mp0, 1024);
toku_mempool_construct(mp1, 1024);
elts[0] = le_malloc(mp0, "a", "aval");
elts[1] = le_malloc(mp0, "b", "bval");
elts[2] = le_malloc(mp1, "x", "xval");
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
}
BLB_NBYTESINBUF(&sn, 0) = 2*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 0));
BLB_NBYTESINBUF(&sn, 1) = 1*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 1));
BLB_MAX_MSN_APPLIED(&sn, 0) = ((MSN) { MIN_MSN.msn + 73 });
......@@ -253,6 +271,13 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft) {
assert(dn->n_children>=1);
assert(dn->max_msn_applied_to_node_on_disk.msn == POSTSERIALIZE_MSN_ON_DISK.msn);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
toku_mempool_construct(&dummy_mp, 1024);
elts[0] = le_malloc(&dummy_mp, "a", "aval");
elts[1] = le_malloc(&dummy_mp, "b", "bval");
elts[2] = le_malloc(&dummy_mp, "x", "xval");
const u_int32_t npartitions = dn->n_children;
assert(dn->totalchildkeylens==(2*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = 3, .elts = elts, .i = 0, .cmp = omt_cmp };
......@@ -273,6 +298,7 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) == (extra.i-last_i)*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(dn, i)));
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == 3);
}
toku_brtnode_free(&dn);
......@@ -280,10 +306,10 @@ test_serialize_leaf_check_msn(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < 3; ++i) {
toku_free(elts[i]);
}
for (int i = 0; i < sn.n_children; i++) {
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -316,16 +342,6 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft) {
sn.n_children = nrows;
sn.dirty = 1;
LEAFENTRY les[nrows];
{
char key[keylens], val[vallens];
key[keylens-1] = '\0';
for (int i = 0; i < nrows; ++i) {
char c = 'a' + i;
memset(key, c, keylens-1);
les[i] = le_fastmalloc((char *) &key, sizeof(key), (char *) &val, sizeof(val));
}
}
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children-1, sn.childkeys);
sn.totalchildkeylens = (sn.n_children-1)*sizeof(int);
......@@ -333,13 +349,23 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft) {
BP_STATE(&sn,i) = PT_AVAIL;
set_BLB(&sn, i, toku_create_empty_bn());
}
for (int i = 0; i < nrows; ++i) {
r = toku_omt_insert(BLB_BUFFER(&sn, i), les[i], omt_cmp, les[i], NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, i) = leafentry_disksize(les[i]);
for (int i = 0; i < nrows; ++i) { // one basement per row
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
size_t le_size = calc_le_size(keylens, vallens);
size_t mpsize = le_size; // one basement per row implies one row per basement
toku_mempool_construct(mp, mpsize);
char key[keylens], val[vallens];
key[keylens-1] = '\0';
char c = 'a' + i;
memset(key, c, keylens-1);
LEAFENTRY le = le_fastmalloc(mp, (char *) &key, sizeof(key), (char *) &val, sizeof(val));
r = toku_omt_insert(BLB_BUFFER(&sn, i), le, omt_cmp, le, NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, i) = leafentry_disksize(le);
if (i < nrows-1) {
u_int32_t keylen;
char *key = le_key_and_len(les[i], &keylen);
sn.childkeys[i] = kv_pair_malloc(key, keylen, 0, 0);
char *keyp = le_key_and_len(le, &keylen);
sn.childkeys[i] = kv_pair_malloc(keyp, keylen, 0, 0);
}
}
......@@ -378,6 +404,22 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft) {
assert(dn->layout_version ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
size_t le_size = calc_le_size(keylens, vallens);
size_t mpsize = nrows * le_size;
toku_mempool_construct(&dummy_mp, mpsize);
LEAFENTRY les[nrows];
{
char key[keylens], val[vallens];
key[keylens-1] = '\0';
for (int i = 0; i < nrows; ++i) {
char c = 'a' + i;
memset(key, c, keylens-1);
les[i] = le_fastmalloc(&dummy_mp, (char *) &key, sizeof(key), (char *) &val, sizeof(val));
}
}
const u_int32_t npartitions = dn->n_children;
assert(dn->totalchildkeylens==(keylens*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = nrows, .elts = les, .i = 0, .cmp = omt_cmp };
......@@ -394,6 +436,7 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) == (extra.i-last_i)*(KEY_VALUE_OVERHEAD+keylens+vallens) + toku_omt_size(BLB_BUFFER(dn, i)));
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == nrows);
}
......@@ -401,11 +444,11 @@ test_serialize_leaf_with_large_pivots(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < nrows; ++i) {
toku_free(les[i]);
}
toku_free(sn.childkeys);
for (int i = 0; i < sn.n_children; i++) {
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -437,13 +480,6 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
sn.n_children = 1;
sn.dirty = 1;
LEAFENTRY les[nrows];
{
int key = 0, val = 0;
for (int i = 0; i < nrows; ++i, key++, val++) {
les[i] = le_fastmalloc((char *) &key, sizeof(key), (char *) &val, sizeof(val));
}
}
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children-1, sn.childkeys);
sn.totalchildkeylens = (sn.n_children-1)*sizeof(int);
......@@ -452,9 +488,19 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
set_BLB(&sn, i, toku_create_empty_bn());
}
BLB_NBYTESINBUF(&sn, 0) = 0;
BASEMENTNODE bn = BLB(&sn,0);
struct mempool * mp = &bn->buffer_mempool;
{
size_t le_size = calc_le_size(keylens, vallens);
size_t mpsize = nrows * le_size; // one basement, so all rows must fit in this one mempool
toku_mempool_construct(mp, mpsize);
}
for (int i = 0; i < nrows; ++i) {
r = toku_omt_insert(BLB_BUFFER(&sn, 0), les[i], omt_int_cmp, les[i], NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, 0) += leafentry_disksize(les[i]);
int key = i;
int val = i;
LEAFENTRY le = le_fastmalloc(mp, (char *) &key, sizeof(key), (char *) &val, sizeof(val));
r = toku_omt_insert(BLB_BUFFER(&sn, 0), le, omt_int_cmp, le, NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, 0) += leafentry_disksize(le);
}
struct brt *XMALLOC(brt);
......@@ -492,6 +538,19 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
assert(dn->layout_version ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
size_t le_size = calc_le_size(keylens, vallens);
size_t mpsize = nrows * le_size;
toku_mempool_construct(&dummy_mp, mpsize);
LEAFENTRY les[nrows];
{
int key = 0, val = 0;
for (int i = 0; i < nrows; ++i, key++, val++) {
les[i] = le_fastmalloc(&dummy_mp, (char *) &key, sizeof(key), (char *) &val, sizeof(val));
}
}
const u_int32_t npartitions = dn->n_children;
assert(dn->totalchildkeylens==(sizeof(int)*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = nrows, .elts = les, .i = 0, .cmp = omt_int_cmp };
......@@ -509,6 +568,7 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) < 128*1024); // BN_MAX_SIZE, apt to change
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == nrows);
}
......@@ -516,10 +576,10 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < nrows; ++i) {
toku_free(les[i]);
}
for (int i = 0; i < sn.n_children; i++) {
bn = BLB(&sn, i);
mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -533,10 +593,13 @@ test_serialize_leaf_with_many_rows(enum brtnode_verify_type bft) {
r = close(fd); assert(r != -1);
}
static void
test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
int r;
struct brtnode sn, *dn;
const uint32_t nrows = 7;
const size_t key_size = 8;
const size_t val_size = 512*1024;
// assert(val_size > BN_MAX_SIZE); // BN_MAX_SIZE isn't visible
int fd = open(__FILE__ ".brt", O_RDWR|O_CREAT|O_BINARY, S_IRWXU|S_IRWXG|S_IRWXO); assert(fd >= 0);
......@@ -552,18 +615,6 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
sn.n_children = 1;
sn.dirty = 1;
LEAFENTRY les[7];
{
char key[8], val[val_size];
key[7] = '\0';
val[val_size-1] = '\0';
for (int i = 0; i < 7; ++i) {
char c = 'a' + i;
memset(key, c, 7);
memset(val, c, val_size-1);
les[i] = le_fastmalloc(key, 8, val, val_size);
}
}
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children-1, sn.childkeys);
sn.totalchildkeylens = (sn.n_children-1)*8;
......@@ -571,10 +622,24 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
BP_STATE(&sn,i) = PT_AVAIL;
set_BLB(&sn, i, toku_create_empty_bn());
}
BASEMENTNODE bn = BLB(&sn,0);
struct mempool * mp = &bn->buffer_mempool;
{
size_t le_size = calc_le_size(key_size, val_size);
size_t mpsize = nrows * le_size; // one basement, so all rows must fit in this one mempool
toku_mempool_construct(mp, mpsize);
}
BLB_NBYTESINBUF(&sn, 0) = 0;
for (int i = 0; i < 7; ++i) {
r = toku_omt_insert(BLB_BUFFER(&sn, 0), les[i], omt_cmp, les[i], NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, 0) += leafentry_disksize(les[i]);
for (uint32_t i = 0; i < nrows; ++i) {
char key[key_size], val[val_size];
key[key_size-1] = '\0';
val[val_size-1] = '\0';
char c = 'a' + i;
memset(key, c, key_size-1);
memset(val, c, val_size-1);
LEAFENTRY le = le_fastmalloc(mp, key, 8, val, val_size);
r = toku_omt_insert(BLB_BUFFER(&sn, 0), le, omt_cmp, le, NULL); assert(r==0);
BLB_NBYTESINBUF(&sn, 0) += leafentry_disksize(le);
}
struct brt *XMALLOC(brt);
......@@ -612,10 +677,28 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
assert(dn->layout_version ==BRT_LAYOUT_VERSION);
assert(dn->layout_version_original ==BRT_LAYOUT_VERSION);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
size_t le_size = calc_le_size(key_size, val_size);
size_t mpsize = nrows * le_size;
toku_mempool_construct(&dummy_mp, mpsize);
LEAFENTRY les[nrows];
{
char key[key_size], val[val_size];
key[key_size-1] = '\0';
val[val_size-1] = '\0';
for (uint32_t i = 0; i < nrows; ++i) {
char c = 'a' + i;
memset(key, c, key_size-1);
memset(val, c, val_size-1);
les[i] = le_fastmalloc(&dummy_mp, key, key_size, val, val_size);
}
}
const u_int32_t npartitions = dn->n_children;
assert(npartitions == 7);
assert(dn->totalchildkeylens==(8*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = 7, .elts = les, .i = 0, .cmp = omt_cmp };
assert(npartitions == nrows);
assert(dn->totalchildkeylens==(key_size*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = nrows, .elts = les, .i = 0, .cmp = omt_cmp };
u_int32_t last_i = 0;
for (u_int32_t i = 0; i < npartitions; ++i) {
assert(dn->bp[i].start > 0);
......@@ -629,6 +712,7 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) == (extra.i-last_i)*(KEY_VALUE_OVERHEAD+8+val_size) + toku_omt_size(BLB_BUFFER(dn, i)));
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == 7);
}
......@@ -636,10 +720,10 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < 7; ++i) {
toku_free(les[i]);
}
for (int i = 0; i < sn.n_children; i++) {
bn = BLB(&sn, i);
mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -653,6 +737,7 @@ test_serialize_leaf_with_large_rows(enum brtnode_verify_type bft) {
r = close(fd); assert(r != -1);
}
static void
test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
const int nodesize = 1024;
......@@ -672,10 +757,6 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
sn.optimized_for_upgrade = 1234;
sn.n_children = 7;
sn.dirty = 1;
LEAFENTRY elts[3];
elts[0] = le_malloc("a", "aval");
elts[1] = le_malloc("b", "bval");
elts[2] = le_malloc("x", "xval");
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children-1, sn.childkeys);
sn.childkeys[0] = kv_pair_malloc("A", 2, 0, 0);
......@@ -690,9 +771,24 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
set_BLB(&sn, i, toku_create_empty_bn());
BLB_SEQINSERT(&sn, i) = 0;
}
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 3), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 5), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
LEAFENTRY elts[3];
{
BASEMENTNODE bn = BLB(&sn,1);
struct mempool * mp1 = &bn->buffer_mempool;
bn = BLB(&sn,3);
struct mempool * mp3 = &bn->buffer_mempool;
bn = BLB(&sn,5);
struct mempool * mp5 = &bn->buffer_mempool;
toku_mempool_construct(mp1, 1024);
toku_mempool_construct(mp3, 1024);
toku_mempool_construct(mp5, 1024);
elts[0] = le_malloc(mp1, "a", "aval");
elts[1] = le_malloc(mp3, "b", "bval");
elts[2] = le_malloc(mp5, "x", "xval");
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 3), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 5), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
}
BLB_NBYTESINBUF(&sn, 0) = 0*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 0));
BLB_NBYTESINBUF(&sn, 1) = 1*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 1));
BLB_NBYTESINBUF(&sn, 2) = 0*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 2));
......@@ -740,6 +836,13 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children>0);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
toku_mempool_construct(&dummy_mp, 1024);
elts[0] = le_malloc(&dummy_mp, "a", "aval");
elts[1] = le_malloc(&dummy_mp, "b", "bval");
elts[2] = le_malloc(&dummy_mp, "x", "xval");
const u_int32_t npartitions = dn->n_children;
assert(dn->totalchildkeylens==(2*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = 3, .elts = elts, .i = 0, .cmp = omt_cmp };
......@@ -756,6 +859,7 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) == (extra.i-last_i)*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(dn, i)));
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == 3);
}
toku_brtnode_free(&dn);
......@@ -763,10 +867,10 @@ test_serialize_leaf_with_empty_basement_nodes(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < 3; ++i) {
toku_free(elts[i]);
}
for (int i = 0; i < sn.n_children; i++) {
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -890,6 +994,7 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum brtnode_verify_type
r = close(fd); assert(r != -1);
}
static void
test_serialize_leaf(enum brtnode_verify_type bft) {
// struct brt source_brt;
......@@ -910,10 +1015,6 @@ test_serialize_leaf(enum brtnode_verify_type bft) {
sn.optimized_for_upgrade = 1234;
sn.n_children = 2;
sn.dirty = 1;
LEAFENTRY elts[3];
elts[0] = le_malloc("a", "aval");
elts[1] = le_malloc("b", "bval");
elts[2] = le_malloc("x", "xval");
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(1, sn.childkeys);
sn.childkeys[0] = kv_pair_malloc("b", 2, 0, 0);
......@@ -922,9 +1023,21 @@ test_serialize_leaf(enum brtnode_verify_type bft) {
BP_STATE(&sn,1) = PT_AVAIL;
set_BLB(&sn, 0, toku_create_empty_bn());
set_BLB(&sn, 1, toku_create_empty_bn());
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
LEAFENTRY elts[3];
{
BASEMENTNODE bn = BLB(&sn,0);
struct mempool * mp0 = &bn->buffer_mempool;
bn = BLB(&sn,1);
struct mempool * mp1 = &bn->buffer_mempool;
toku_mempool_construct(mp0, 1024);
toku_mempool_construct(mp1, 1024);
elts[0] = le_malloc(mp0, "a", "aval");
elts[1] = le_malloc(mp0, "b", "bval");
elts[2] = le_malloc(mp1, "x", "xval");
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[0], omt_cmp, elts[0], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 0), elts[1], omt_cmp, elts[1], NULL); assert(r==0);
r = toku_omt_insert(BLB_BUFFER(&sn, 1), elts[2], omt_cmp, elts[2], NULL); assert(r==0);
}
BLB_NBYTESINBUF(&sn, 0) = 2*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 0));
BLB_NBYTESINBUF(&sn, 1) = 1*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(&sn, 1));
......@@ -967,6 +1080,13 @@ test_serialize_leaf(enum brtnode_verify_type bft) {
assert(dn->optimized_for_upgrade == 1234);
assert(dn->n_children>=1);
{
// Man, this is way too ugly. This entire test suite needs to be refactored.
// Create a dummy mempool and put the leaves there. Ugh.
struct mempool dummy_mp;
toku_mempool_construct(&dummy_mp, 1024);
elts[0] = le_malloc(&dummy_mp, "a", "aval");
elts[1] = le_malloc(&dummy_mp, "b", "bval");
elts[2] = le_malloc(&dummy_mp, "x", "xval");
const u_int32_t npartitions = dn->n_children;
assert(dn->totalchildkeylens==(2*(npartitions-1)));
struct check_leafentries_struct extra = { .nelts = 3, .elts = elts, .i = 0, .cmp = omt_cmp };
......@@ -986,6 +1106,7 @@ test_serialize_leaf(enum brtnode_verify_type bft) {
assert(BLB_NBYTESINBUF(dn, i) == (extra.i-last_i)*(KEY_VALUE_OVERHEAD+2+5) + toku_omt_size(BLB_BUFFER(dn, i)));
last_i = extra.i;
}
toku_mempool_destroy(&dummy_mp);
assert(extra.i == 3);
}
toku_brtnode_free(&dn);
......@@ -993,10 +1114,10 @@ test_serialize_leaf(enum brtnode_verify_type bft) {
for (int i = 0; i < sn.n_children-1; ++i) {
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < 3; ++i) {
toku_free(elts[i]);
}
for (int i = 0; i < sn.n_children; i++) {
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......
......@@ -106,17 +106,16 @@ insert_random_message_to_leaf(BRT t, BASEMENTNODE blb, LEAFENTRY *save, XIDS xid
toku_fill_dbt(keydbt, key, keylen + (sizeof pfx));
toku_fill_dbt(valdbt, val, vallen);
BRT_MSG_S msg;
BRT_MSG_S *result = &msg;
result->type = BRT_INSERT;
result->msn = msn;
result->xids = xids;
result->u.id.key = keydbt;
result->u.id.val = valdbt;
size_t memsize, disksize;
int r = apply_msg_to_leafentry(result, NULL, &memsize, &disksize, save, NULL, NULL);
msg.type = BRT_INSERT;
msg.msn = msn;
msg.xids = xids;
msg.u.id.key = keydbt;
msg.u.id.val = valdbt;
size_t memsize;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL);
assert_zero(r);
bool made_change;
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb, result, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb, &msg, &made_change, NULL, NULL, NULL);
if (msn.msn > blb->max_msn_applied.msn) {
blb->max_msn_applied = msn;
}
......@@ -140,21 +139,20 @@ insert_same_message_to_leaves(BRT t, BASEMENTNODE blb1, BASEMENTNODE blb2, LEAFE
toku_fill_dbt(keydbt, key, keylen + (sizeof pfx));
toku_fill_dbt(valdbt, val, vallen);
BRT_MSG_S msg;
BRT_MSG_S *result = &msg;
result->type = BRT_INSERT;
result->msn = msn;
result->xids = xids;
result->u.id.key = keydbt;
result->u.id.val = valdbt;
size_t memsize, disksize;
int r = apply_msg_to_leafentry(result, NULL, &memsize, &disksize, save, NULL, NULL);
msg.type = BRT_INSERT;
msg.msn = msn;
msg.xids = xids;
msg.u.id.key = keydbt;
msg.u.id.val = valdbt;
size_t memsize;
int r = apply_msg_to_leafentry(&msg, NULL, &memsize, save, NULL, NULL, NULL, NULL, NULL);
assert_zero(r);
bool made_change;
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb1, result, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb1, &msg, &made_change, NULL, NULL, NULL);
if (msn.msn > blb1->max_msn_applied.msn) {
blb1->max_msn_applied = msn;
}
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb2, result, &made_change, NULL, NULL, NULL);
brt_leaf_put_cmd(t->compare_fun, t->update_fun, NULL, blb2, &msg, &made_change, NULL, NULL, NULL);
if (msn.msn > blb2->max_msn_applied.msn) {
blb2->max_msn_applied = msn;
}
......
......@@ -130,11 +130,8 @@ test_le_offsets (void) {
static void
test_ule_packs_to_nothing (ULE ule) {
size_t memsize;
size_t disksize;
LEAFENTRY le;
int r = le_pack(ule,
&memsize, &disksize,
&le);
int r = le_pack(ule, &memsize, &le, NULL, NULL, NULL);
assert(r==0);
assert(le==NULL);
}
......@@ -177,16 +174,13 @@ test_le_empty_packs_to_nothing (void) {
}
static void
le_verify_accessors(LEAFENTRY le, ULE ule,
size_t pre_calculated_memsize,
size_t pre_calculated_disksize) {
le_verify_accessors(LEAFENTRY le, ULE ule, size_t pre_calculated_memsize) {
assert(le);
assert(ule->num_cuxrs > 0);
assert(ule->num_puxrs <= MAX_TRANSACTION_RECORDS);
assert(ule->uxrs[ule->num_cuxrs + ule->num_puxrs-1].type != XR_PLACEHOLDER);
//Extract expected values from ULE
size_t memsize = le_memsize_from_ule(ule);
size_t disksize = le_memsize_from_ule(ule);
size_t num_uxrs = ule->num_cuxrs + ule->num_puxrs;
void *key = ule->keyp;
......@@ -209,10 +203,7 @@ found_insert:;
assert(le!=NULL);
//Verify all accessors
assert(memsize == pre_calculated_memsize);
assert(disksize == pre_calculated_disksize);
assert(memsize == disksize);
assert(memsize == leafentry_memsize(le));
assert(disksize == leafentry_disksize(le));
{
u_int32_t test_keylen;
void* test_keyp = le_key_and_len(le, &test_keylen);
......@@ -265,26 +256,19 @@ test_le_pack_committed (void) {
ule.uxrs[0].vallen = valsize;
size_t memsize;
size_t disksize;
LEAFENTRY le;
int r = le_pack(&ule,
&memsize, &disksize,
&le);
int r = le_pack(&ule, &memsize, &le, NULL, NULL, NULL);
assert(r==0);
assert(le!=NULL);
le_verify_accessors(le, &ule, memsize, disksize);
le_verify_accessors(le, &ule, memsize);
ULE_S tmp_ule;
le_unpack(&tmp_ule, le);
verify_ule_equal(&ule, &tmp_ule);
LEAFENTRY tmp_le;
size_t tmp_memsize;
size_t tmp_disksize;
r = le_pack(&tmp_ule,
&tmp_memsize, &tmp_disksize,
&tmp_le);
r = le_pack(&tmp_ule, &tmp_memsize, &tmp_le, NULL, NULL, NULL);
assert(r==0);
assert(tmp_memsize == memsize);
assert(tmp_disksize == disksize);
assert(memcmp(le, tmp_le, memsize) == 0);
toku_free(tmp_le);
......@@ -334,26 +318,19 @@ test_le_pack_uncommitted (u_int8_t committed_type, u_int8_t prov_type, int num_p
ule.uxrs[idx].valp = pval;
size_t memsize;
size_t disksize;
LEAFENTRY le;
int r = le_pack(&ule,
&memsize, &disksize,
&le);
int r = le_pack(&ule, &memsize, &le, NULL, NULL, NULL);
assert(r==0);
assert(le!=NULL);
le_verify_accessors(le, &ule, memsize, disksize);
le_verify_accessors(le, &ule, memsize);
ULE_S tmp_ule;
le_unpack(&tmp_ule, le);
verify_ule_equal(&ule, &tmp_ule);
LEAFENTRY tmp_le;
size_t tmp_memsize;
size_t tmp_disksize;
r = le_pack(&tmp_ule,
&tmp_memsize, &tmp_disksize,
&tmp_le);
r = le_pack(&tmp_ule, &tmp_memsize, &tmp_le, NULL, NULL, NULL);
assert(r==0);
assert(tmp_memsize == memsize);
assert(tmp_disksize == disksize);
assert(memcmp(le, tmp_le, memsize) == 0);
toku_free(tmp_le);
......@@ -412,34 +389,29 @@ test_le_apply(ULE ule_initial, BRT_MSG msg, ULE ule_expected) {
LEAFENTRY le_result;
size_t initial_memsize;
size_t initial_disksize;
r = le_pack(ule_initial, &initial_memsize, &initial_disksize,
&le_initial);
r = le_pack(ule_initial, &initial_memsize, &le_initial, NULL, NULL, NULL);
CKERR(r);
size_t result_memsize;
size_t result_disksize;
r = apply_msg_to_leafentry(msg,
le_initial,
&result_memsize, &result_disksize,
&result_memsize,
&le_result,
NULL, NULL, NULL,
NULL, NULL);
CKERR(r);
if (le_result)
le_verify_accessors(le_result, ule_expected, result_memsize, result_disksize);
le_verify_accessors(le_result, ule_expected, result_memsize);
size_t expected_memsize;
size_t expected_disksize;
r = le_pack(ule_expected, &expected_memsize, &expected_disksize,
&le_expected);
r = le_pack(ule_expected, &expected_memsize, &le_expected, NULL, NULL, NULL);
CKERR(r);
verify_le_equal(le_result, le_expected);
if (le_result && le_expected) {
assert(result_memsize == expected_memsize);
assert(result_disksize == expected_disksize);
}
if (le_initial) toku_free(le_initial);
if (le_result) toku_free(le_result);
......
......@@ -27,24 +27,41 @@ static int omt_long_cmp(OMTVALUE p, void *q)
return (*ai > *bi) - (*ai < *bi);
}
static size_t
calc_le_size(int keylen, int vallen) {
size_t rval;
LEAFENTRY le;
rval = sizeof(le->type) + sizeof(le->keylen) + sizeof(le->u.clean.vallen) + keylen + vallen;
return rval;
}
static LEAFENTRY
le_fastmalloc(char *key, int keylen, char *val, int vallen)
le_fastmalloc(struct mempool * mp, char *key, int keylen, char *val, int vallen)
{
LEAFENTRY r = toku_malloc(sizeof(r->type) + sizeof(r->keylen) + sizeof(r->u.clean.vallen) +
keylen + vallen);
resource_assert(r);
r->type = LE_CLEAN;
r->keylen = keylen;
r->u.clean.vallen = vallen;
memcpy(&r->u.clean.key_val[0], key, keylen);
memcpy(&r->u.clean.key_val[keylen], val, vallen);
return r;
LEAFENTRY le;
size_t le_size = calc_le_size(keylen, vallen);
le = toku_mempool_malloc(mp, le_size, 1);
resource_assert(le);
le->type = LE_CLEAN;
le->keylen = keylen;
le->u.clean.vallen = vallen;
memcpy(&le->u.clean.key_val[0], key, keylen);
memcpy(&le->u.clean.key_val[keylen], val, vallen);
return le;
}
//
// Maximum node size according to the BRT: 1024 (expected node size after split)
// Maximum basement node size: 256
// Actual node size before split: 2048
// Actual basement node size before split: 256
// Start by creating 8 basements, then split node, expected result of two nodes with 4 basements each.
static void
test_split_on_boundary(void)
{
const int nodesize = 1024, eltsize = 64, bnsize = 256;
const size_t maxbnsize = bnsize;
const int keylen = sizeof(long), vallen = eltsize - keylen - (sizeof(((LEAFENTRY)NULL)->type) // overhead from LE_CLEAN_MEMSIZE
+sizeof(((LEAFENTRY)NULL)->keylen)
+sizeof(((LEAFENTRY)NULL)->u.clean.vallen));
......@@ -66,22 +83,24 @@ test_split_on_boundary(void)
const int nelts = 2 * nodesize / eltsize;
sn.n_children = nelts * eltsize / bnsize;
sn.dirty = 1;
LEAFENTRY elts[nelts];
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children - 1, sn.childkeys);
sn.totalchildkeylens = 0;
for (int bn = 0; bn < sn.n_children; ++bn) {
BP_STATE(&sn,bn) = PT_AVAIL;
set_BLB(&sn, bn, toku_create_empty_bn());
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
toku_mempool_construct(mp, maxbnsize);
BLB_NBYTESINBUF(&sn,bn) = 0;
long k;
for (int i = 0; i < eltsperbn; ++i) {
k = bn * eltsperbn + i;
char val[vallen];
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
LEAFENTRY le = le_fastmalloc(mp, (char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
}
if (bn < sn.n_children - 1) {
sn.childkeys[bn] = kv_pair_malloc(&k, sizeof k, 0, 0);
......@@ -112,17 +131,30 @@ test_split_on_boundary(void)
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < sn.n_children; ++i) {
toku_omt_free_items(BLB_BUFFER(&sn, i));
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
toku_free(sn.childkeys);
}
//
// Maximum node size according to the BRT: 1024 (expected node size after split)
// Maximum basement node size: 256 (except the last)
// Actual node size before split: 4095
// Actual basement node size before split: 256 (except the last, of size 2K)
//
// Start by creating 9 basements, the first 8 being of 256 bytes each,
// and the last with one row of size 2047 bytes. Then split node,
// expected result is two nodes, one with 8 basement nodes and one
// with 1 basement node.
static void
test_split_with_everything_on_the_left(void)
{
const int nodesize = 1024, eltsize = 64, bnsize = 256;
const size_t maxbnsize = 1024 * 2;
const int keylen = sizeof(long), vallen = eltsize - keylen - (sizeof(((LEAFENTRY)NULL)->type) // overhead from LE_CLEAN_MEMSIZE
+sizeof(((LEAFENTRY)NULL)->keylen)
+sizeof(((LEAFENTRY)NULL)->u.clean.vallen));
......@@ -144,15 +176,15 @@ test_split_with_everything_on_the_left(void)
const int nelts = 2 * nodesize / eltsize;
sn.n_children = nelts * eltsize / bnsize + 1;
sn.dirty = 1;
LEAFENTRY elts[nelts];
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children - 1, sn.childkeys);
sn.totalchildkeylens = 0;
LEAFENTRY big_element;
char *big_val = NULL;
for (int bn = 0; bn < sn.n_children; ++bn) {
BP_STATE(&sn,bn) = PT_AVAIL;
set_BLB(&sn, bn, toku_create_empty_bn());
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
toku_mempool_construct(mp, maxbnsize);
BLB_NBYTESINBUF(&sn,bn) = 0;
long k;
if (bn < sn.n_children - 1) {
......@@ -160,17 +192,19 @@ test_split_with_everything_on_the_left(void)
k = bn * eltsperbn + i;
char val[vallen];
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
LEAFENTRY le = le_fastmalloc(mp, (char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
}
sn.childkeys[bn] = kv_pair_malloc(&k, sizeof k, 0, 0);
sn.totalchildkeylens += (sizeof k);
} else {
k = bn * eltsperbn;
big_val = toku_xmalloc(nelts * eltsize - 1);
memset(big_val, k, nelts * eltsize - 1);
big_element = le_fastmalloc((char *) &k, keylen, big_val, nelts * eltsize - 1);
size_t big_val_size = (nelts * eltsize - 1); // TODO: Explain this
char * big_val = toku_xmalloc(big_val_size);
memset(big_val, k, big_val_size);
LEAFENTRY big_element = le_fastmalloc(mp, (char *) &k, keylen, big_val, big_val_size);
toku_free(big_val);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), big_element, omt_long_cmp, big_element, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(big_element);
}
......@@ -199,20 +233,31 @@ test_split_with_everything_on_the_left(void)
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < sn.n_children; ++i) {
toku_omt_free_items(BLB_BUFFER(&sn, i));
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
toku_free(sn.childkeys);
if (big_val) {
toku_free(big_val);
}
}
//
// Maximum node size according to the BRT: 1024 (expected node size after split)
// Maximum basement node size: 256 (except the last)
// Actual node size before split: 4095
// Actual basement node size before split: 256 (except the last, of size 2K)
//
// Start by creating 9 basements, the first 8 being of 256 bytes each,
// and the last with one row of size 2047 bytes. Then split node,
// expected result is two nodes, one with 8 basement nodes and one
// with 1 basement node.
static void
test_split_on_boundary_of_last_node(void)
{
const int nodesize = 1024, eltsize = 64, bnsize = 256;
const size_t maxbnsize = 1024 * 2;
const int keylen = sizeof(long), vallen = eltsize - keylen - (sizeof(((LEAFENTRY)NULL)->type) // overhead from LE_CLEAN_MEMSIZE
+sizeof(((LEAFENTRY)NULL)->keylen)
+sizeof(((LEAFENTRY)NULL)->u.clean.vallen));
......@@ -234,15 +279,15 @@ test_split_on_boundary_of_last_node(void)
const int nelts = 2 * nodesize / eltsize;
sn.n_children = nelts * eltsize / bnsize + 1;
sn.dirty = 1;
LEAFENTRY elts[nelts];
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children - 1, sn.childkeys);
sn.totalchildkeylens = 0;
LEAFENTRY big_element;
char *big_val = NULL;
for (int bn = 0; bn < sn.n_children; ++bn) {
BP_STATE(&sn,bn) = PT_AVAIL;
set_BLB(&sn, bn, toku_create_empty_bn());
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
toku_mempool_construct(mp, maxbnsize);
BLB_NBYTESINBUF(&sn,bn) = 0;
long k;
if (bn < sn.n_children - 1) {
......@@ -250,17 +295,20 @@ test_split_on_boundary_of_last_node(void)
k = bn * eltsperbn + i;
char val[vallen];
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
LEAFENTRY le = le_fastmalloc(mp, (char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
}
sn.childkeys[bn] = kv_pair_malloc(&k, sizeof k, 0, 0);
sn.totalchildkeylens += (sizeof k);
} else {
k = bn * eltsperbn;
big_val = toku_xmalloc(nelts * eltsize - 100);
memset(big_val, k, nelts * eltsize - 100);
big_element = le_fastmalloc((char *) &k, keylen, big_val, nelts * eltsize - 100);
size_t big_val_size = (nelts * eltsize - 100); // TODO: This looks wrong, should perhaps be +100?
invariant(big_val_size <= maxbnsize);
char * big_val = toku_xmalloc(big_val_size);
memset(big_val, k, big_val_size);
LEAFENTRY big_element = le_fastmalloc(mp, (char *) &k, keylen, big_val, big_val_size);
toku_free(big_val);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), big_element, omt_long_cmp, big_element, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(big_element);
}
......@@ -289,20 +337,20 @@ test_split_on_boundary_of_last_node(void)
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < sn.n_children; ++i) {
toku_omt_free_items(BLB_BUFFER(&sn, i));
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
toku_free(sn.childkeys);
if (big_val) {
toku_free(big_val);
}
}
static void
test_split_at_begin(void)
{
const int nodesize = 1024, eltsize = 64, bnsize = 256;
const size_t maxbnsize = 1024 * 2;
const int keylen = sizeof(long), vallen = eltsize - keylen - (sizeof(((LEAFENTRY)NULL)->type) // overhead from LE_CLEAN_MEMSIZE
+sizeof(((LEAFENTRY)NULL)->keylen)
+sizeof(((LEAFENTRY)NULL)->u.clean.vallen));
......@@ -324,14 +372,16 @@ test_split_at_begin(void)
const int nelts = 2 * nodesize / eltsize;
sn.n_children = nelts * eltsize / bnsize;
sn.dirty = 1;
LEAFENTRY elts[nelts];
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children - 1, sn.childkeys);
sn.totalchildkeylens = 0;
long totalbytes = 0;
size_t totalbytes = 0;
for (int bn = 0; bn < sn.n_children; ++bn) {
BP_STATE(&sn,bn) = PT_AVAIL;
set_BLB(&sn, bn, toku_create_empty_bn());
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
toku_mempool_construct(mp, maxbnsize);
BLB_NBYTESINBUF(&sn,bn) = 0;
long k;
for (int i = 0; i < eltsperbn; ++i) {
......@@ -343,10 +393,10 @@ test_split_at_begin(void)
}
char val[vallen];
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
totalbytes += leafentry_disksize(elts[k]);
LEAFENTRY le = le_fastmalloc(mp, (char *) &k, keylen, val, vallen);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
totalbytes += leafentry_disksize(le);
}
if (bn < sn.n_children - 1) {
sn.childkeys[bn] = kv_pair_malloc(&k, sizeof k, 0, 0);
......@@ -355,12 +405,15 @@ test_split_at_begin(void)
}
{ // now add the first element
int bn = 0; long k = 0;
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
char val[totalbytes + 3];
invariant(totalbytes + 3 <= maxbnsize);
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, totalbytes + 3);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
totalbytes += leafentry_disksize(elts[k]);
LEAFENTRY le = le_fastmalloc(mp, (char *) &k, keylen, val, totalbytes + 3);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
totalbytes += leafentry_disksize(le);
}
unlink(fname);
......@@ -386,7 +439,9 @@ test_split_at_begin(void)
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < sn.n_children; ++i) {
toku_omt_free_items(BLB_BUFFER(&sn, i));
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......@@ -397,6 +452,7 @@ static void
test_split_at_end(void)
{
const int nodesize = 1024, eltsize = 64, bnsize = 256;
const size_t maxbnsize = 1024 * 2;
const int keylen = sizeof(long), vallen = eltsize - keylen - (sizeof(((LEAFENTRY)NULL)->type) // overhead from LE_CLEAN_MEMSIZE
+sizeof(((LEAFENTRY)NULL)->keylen)
+sizeof(((LEAFENTRY)NULL)->u.clean.vallen));
......@@ -418,7 +474,6 @@ test_split_at_end(void)
const int nelts = 2 * nodesize / eltsize;
sn.n_children = nelts * eltsize / bnsize;
sn.dirty = 1;
LEAFENTRY elts[nelts];
MALLOC_N(sn.n_children, sn.bp);
MALLOC_N(sn.n_children - 1, sn.childkeys);
sn.totalchildkeylens = 0;
......@@ -426,22 +481,26 @@ test_split_at_end(void)
for (int bn = 0; bn < sn.n_children; ++bn) {
BP_STATE(&sn,bn) = PT_AVAIL;
set_BLB(&sn, bn, toku_create_empty_bn());
BASEMENTNODE basement = BLB(&sn, bn);
struct mempool * mp = &basement->buffer_mempool;
toku_mempool_construct(mp, maxbnsize);
BLB_NBYTESINBUF(&sn,bn) = 0;
long k;
for (int i = 0; i < eltsperbn; ++i) {
LEAFENTRY le;
k = bn * eltsperbn + i;
if (bn < sn.n_children - 1 || i < eltsperbn - 1) {
char val[vallen];
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, vallen);
le = le_fastmalloc(mp, (char *) &k, keylen, val, vallen);
} else { // the last element
char val[totalbytes + 3]; // just to be sure
memset(val, k, sizeof val);
elts[k] = le_fastmalloc((char *) &k, keylen, val, totalbytes + 3);
le = le_fastmalloc(mp, (char *) &k, keylen, val, totalbytes + 3);
}
r = toku_omt_insert(BLB_BUFFER(&sn, bn), elts[k], omt_long_cmp, elts[k], NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(elts[k]);
totalbytes += leafentry_disksize(elts[k]);
r = toku_omt_insert(BLB_BUFFER(&sn, bn), le, omt_long_cmp, le, NULL); assert(r == 0);
BLB_NBYTESINBUF(&sn, bn) += leafentry_disksize(le);
totalbytes += leafentry_disksize(le);
}
if (bn < sn.n_children - 1) {
sn.childkeys[bn] = kv_pair_malloc(&k, sizeof k, 0, 0);
......@@ -472,7 +531,9 @@ test_split_at_end(void)
kv_pair_free(sn.childkeys[i]);
}
for (int i = 0; i < sn.n_children; ++i) {
toku_omt_free_items(BLB_BUFFER(&sn, i));
BASEMENTNODE bn = BLB(&sn, i);
struct mempool * mp = &bn->buffer_mempool;
toku_mempool_destroy(mp);
destroy_basement_node(BLB(&sn, i));
}
toku_free(sn.bp);
......
......@@ -62,10 +62,11 @@ void test_msg_modify_ule(ULE ule, BRT_MSG msg);
//Functions exported for test purposes only (used internally for non-test purposes).
void le_unpack(ULE ule, LEAFENTRY le);
int le_pack(ULE ule, // data to be packed into new leafentry
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY * const new_leafentry_p // this is what this function creates
);
size_t *new_leafentry_memorysize,
LEAFENTRY * const new_leafentry_p, // this is what this function creates
OMT omt,
struct mempool *mp,
void **maybe_free);
size_t le_memsize_from_ule (ULE ule);
......
......@@ -120,12 +120,19 @@ static inline size_t uxr_unpack_length_and_bit(UXR uxr, uint8_t *p);
static inline size_t uxr_unpack_data(UXR uxr, uint8_t *p);
static void *
le_malloc(size_t size)
le_malloc(OMT omt, struct mempool *mp, size_t size, void **maybe_free)
{
return toku_xmalloc(size);
void * rval;
if (omt)
rval = mempool_malloc_from_omt(omt, mp, size, maybe_free);
else
rval = toku_xmalloc(size);
resource_assert(rval);
return rval;
}
/////////////////////////////////////////////////////////////////////
// Garbage collection related functions
//
......@@ -288,13 +295,15 @@ done:;
// Return 0 on success.
// If the leafentry is destroyed it sets *new_leafentry_p to NULL.
// Otehrwise the new_leafentry_p points at the new leaf entry.
// As of September 2010, the only possible error returned is ENOMEM.
// As of October 2011, this function always returns 0.
int
apply_msg_to_leafentry(BRT_MSG msg, // message to apply to leafentry
LEAFENTRY old_leafentry, // NULL if there was no stored data.
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY *new_leafentry_p,
OMT omt,
struct mempool *mp,
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse) {
ULE_S ule;
......@@ -309,9 +318,11 @@ apply_msg_to_leafentry(BRT_MSG msg, // message to apply to leafentry
garbage_collection(&ule, snapshot_xids, live_list_reverse);
}
rval = le_pack(&ule, // create packed leafentry
new_leafentry_memorysize,
new_leafentry_disksize,
new_leafentry_p
new_leafentry_memorysize,
new_leafentry_p,
omt,
mp,
maybe_free
);
ule_cleanup(&ule);
return rval;
......@@ -625,9 +636,10 @@ update_le_status(ULE ule, size_t memsize, LE_STATUS s) {
int
le_pack(ULE ule, // data to be packed into new leafentry
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY * const new_leafentry_p // this is what this function creates
)
LEAFENTRY * const new_leafentry_p, // this is what this function creates
OMT omt,
struct mempool *mp,
void **maybe_free)
{
invariant(ule->num_cuxrs > 0);
invariant(ule->uxrs[0].xid == TXNID_NONE);
......@@ -651,11 +663,8 @@ le_pack(ULE ule, // data to be packed into new leafen
}
found_insert:;
memsize = le_memsize_from_ule(ule);
LEAFENTRY new_leafentry = le_malloc(memsize);
if (new_leafentry==NULL) {
rval = ENOMEM;
goto cleanup;
}
LEAFENTRY new_leafentry = le_malloc(omt, mp, memsize, maybe_free);
//Universal data
new_leafentry->keylen = toku_htod32(ule->keylen);
......@@ -781,7 +790,6 @@ found_insert:;
*new_leafentry_p = (LEAFENTRY)new_leafentry;
*new_leafentry_memorysize = memsize;
*new_leafentry_disksize = memsize;
rval = 0;
cleanup:
update_le_status(ule, memsize, &status);
......@@ -2210,7 +2218,6 @@ leafentry_disksize_13(LEAFENTRY_13 le) {
int
toku_le_upgrade_13_14(LEAFENTRY_13 old_leafentry,
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY *new_leafentry_p) {
ULE_S ule;
int rval;
......@@ -2218,8 +2225,8 @@ toku_le_upgrade_13_14(LEAFENTRY_13 old_leafentry,
le_unpack_13(&ule, old_leafentry);
rval = le_pack(&ule, // create packed leafentry
new_leafentry_memorysize,
new_leafentry_disksize,
new_leafentry_p);
new_leafentry_p,
NULL, NULL, NULL); // NULL for omt means that we use malloc instead of mempool
ule_cleanup(&ule);
return rval;
}
......
......@@ -16,6 +16,8 @@
extern "C" {
#endif
#include "mempool.h"
// opaque handles used by outside world (i.e. indexer)
typedef struct ule *ULEHANDLE;
typedef struct uxr *UXRHANDLE;
......@@ -53,8 +55,10 @@ void fast_msg_to_leafentry(
int apply_msg_to_leafentry(BRT_MSG msg,
LEAFENTRY old_leafentry, // NULL if there was no stored data.
size_t *new_leafentry_memorysize,
size_t *new_leafentry_disksize,
LEAFENTRY *new_leafentry_p,
OMT omt,
struct mempool *mp,
void **maybe_free,
OMT snapshot_xids,
OMT live_list_reverse);
......
/* -*- mode: C; c-basic-offset: 4 -*- */
#ident "Copyright (c) 2009 Tokutek Inc. All rights reserved."
#ident "$Id: env_startup.c 20778 2010-05-28 20:38:42Z yfogel $"
/* Purpose of this test is to verify that a failed assert will
* cause a panic, which should be visible via engine status.
* This is a manual test, should not be checked in to repository.
* The panic must be manually induced in the debugger.
*/
#include "test.h"
#include <db.h>
static DB_ENV *env;
#define FLAGS_NOLOG DB_INIT_LOCK|DB_INIT_MPOOL|DB_CREATE|DB_PRIVATE
#define FLAGS_LOG FLAGS_NOLOG|DB_INIT_TXN|DB_INIT_LOG
static int mode = S_IRWXU+S_IRWXG+S_IRWXO;
static void test_shutdown(void);
static void
test_shutdown(void) {
int r;
r=env->close(env, 0); CKERR(r);
env = NULL;
}
static void
setup (u_int32_t flags) {
int r;
if (env)
test_shutdown();
r = system("rm -rf " ENVDIR);
CKERR(r);
r=toku_os_mkdir(ENVDIR, S_IRWXU+S_IRWXG+S_IRWXO);
CKERR(r);
r=db_env_create(&env, 0);
CKERR(r);
env->set_errfile(env, stderr);
r=env->open(env, ENVDIR, flags, mode);
CKERR(r);
}
int
test_main (int argc, char * const argv[]) {
parse_args(argc, argv);
setup(FLAGS_LOG);
env->txn_checkpoint(env, 0, 0, 0);
print_engine_status(env);
test_shutdown();
return 0;
}
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment