Commit 40573588 authored by John Esmet's avatar John Esmet

FT-304 Move the blocktable into its own class, fix tests.

parent 36a5eadb
......@@ -105,10 +105,10 @@ ftnode_get_key_and_fullhash(
void* extra)
{
FT ft = (FT) extra;
BLOCKNUM name;
toku_allocate_blocknum(ft->blocktable, &name, ft);
*cachekey = name;
*fullhash = toku_cachetable_hash(ft->cf, name);
BLOCKNUM blocknum;
ft->blocktable.allocate_blocknum(&blocknum, ft);
*cachekey = blocknum;
*fullhash = toku_cachetable_hash(ft->cf, blocknum);
}
void
......@@ -116,7 +116,7 @@ cachetable_put_empty_node_with_dep_nodes(
FT ft,
uint32_t num_dependent_nodes,
FTNODE* dependent_nodes,
BLOCKNUM* name, //output
BLOCKNUM* blocknum, //output
uint32_t* fullhash, //output
FTNODE* result)
{
......@@ -138,7 +138,7 @@ cachetable_put_empty_node_with_dep_nodes(
num_dependent_nodes,
dependent_pairs,
dependent_dirty_bits,
name,
blocknum,
fullhash,
toku_ftnode_save_ct_pair);
*result = new_node;
......@@ -154,13 +154,13 @@ create_new_ftnode_with_dep_nodes(
FTNODE* dependent_nodes)
{
uint32_t fullhash = 0;
BLOCKNUM name;
BLOCKNUM blocknum;
cachetable_put_empty_node_with_dep_nodes(
ft,
num_dependent_nodes,
dependent_nodes,
&name,
&blocknum,
&fullhash,
result);
......@@ -171,7 +171,7 @@ create_new_ftnode_with_dep_nodes(
toku_initialize_empty_ftnode(
*result,
name,
blocknum,
height,
n_children,
ft->h->layout_version,
......
......@@ -1338,7 +1338,7 @@ maybe_merge_pinned_nodes(
static void merge_remove_key_callback(BLOCKNUM *bp, bool for_checkpoint, void *extra) {
FT ft = (FT) extra;
toku_free_blocknum(ft->blocktable, bp, ft, for_checkpoint);
ft->blocktable.free_blocknum(bp, ft, for_checkpoint);
}
//
......@@ -1517,7 +1517,7 @@ void toku_ft_flush_some_child(FT ft, FTNODE parent, struct flusher_advice *fa)
// get the child into memory
BLOCKNUM targetchild = BP_BLOCKNUM(parent, childnum);
toku_verify_blocknum_allocated(ft->blocktable, targetchild);
ft->blocktable.verify_blocknum_allocated(targetchild);
uint32_t childfullhash = compute_child_fullhash(ft->cf, parent, childnum);
FTNODE child;
struct ftnode_fetch_extra bfe;
......
......@@ -101,6 +101,7 @@ PATENT RIGHTS GRANT:
#include "ft/ft.h"
#include "ft/ft-ops.h"
#include "ft/node.h"
#include "ft/serialize/block_table.h"
#include "ft/txn/rollback.h"
// Symbol TOKUDB_REVISION is not defined by fractal-tree makefiles, so
......@@ -111,7 +112,6 @@ PATENT RIGHTS GRANT:
#error
#endif
struct block_table;
struct ft_search;
enum { FT_DEFAULT_FANOUT = 16 };
......@@ -225,7 +225,7 @@ struct ft {
// These are not read-only:
// protected by blocktable lock
struct block_table *blocktable;
struct block_table blocktable;
// protected by atomic builtins
STAT64INFO_S in_memory_stats;
......
......@@ -1927,7 +1927,7 @@ static void push_something_in_subtree(
{
const BLOCKNUM child_blocknum = BP_BLOCKNUM(subtree_root, childnum);
toku_verify_blocknum_allocated(ft->blocktable, child_blocknum);
ft->blocktable.verify_blocknum_allocated(child_blocknum);
const uint32_t child_fullhash = toku_cachetable_hash(ft->cf, child_blocknum);
FTNODE child;
......@@ -3088,10 +3088,11 @@ ft_handle_open(FT_HANDLE ft_h, const char *fname_in_env, int is_create, int only
toku_txn_maybe_note_ft(txn, ft);
}
//Opening an ft may restore to previous checkpoint. Truncate if necessary.
// Opening an ft may restore to previous checkpoint.
// Truncate if necessary.
{
int fd = toku_cachefile_get_fd (ft->cf);
toku_maybe_truncate_file_on_open(ft->blocktable, fd);
ft->blocktable.maybe_truncate_file_on_open(fd);
}
r = 0;
......@@ -4528,17 +4529,15 @@ toku_dump_ftnode (FILE *file, FT_HANDLE ft_handle, BLOCKNUM blocknum, int depth,
return result;
}
int toku_dump_ft (FILE *f, FT_HANDLE ft_handle) {
int r;
assert(ft_handle->ft);
toku_dump_translation_table(f, ft_handle->ft->blocktable);
{
uint32_t fullhash = 0;
CACHEKEY root_key;
toku_calculate_root_offset_pointer(ft_handle->ft, &root_key, &fullhash);
r = toku_dump_ftnode(f, ft_handle, root_key, 0, 0, 0);
}
return r;
int toku_dump_ft(FILE *f, FT_HANDLE ft_handle) {
FT ft = ft_handle->ft;
invariant_notnull(ft);
ft->blocktable.dump_translation_table(f);
uint32_t fullhash = 0;
CACHEKEY root_key;
toku_calculate_root_offset_pointer(ft_handle->ft, &root_key, &fullhash);
return toku_dump_ftnode(f, ft_handle, root_key, 0, 0, 0);
}
int toku_ft_layer_init(void) {
......@@ -4630,18 +4629,15 @@ void toku_ft_unlink(FT_HANDLE handle) {
toku_cachefile_unlink_on_close(cf);
}
int
toku_ft_get_fragmentation(FT_HANDLE ft_handle, TOKU_DB_FRAGMENTATION report) {
int r;
int toku_ft_get_fragmentation(FT_HANDLE ft_handle, TOKU_DB_FRAGMENTATION report) {
int fd = toku_cachefile_get_fd(ft_handle->ft->cf);
toku_ft_lock(ft_handle->ft);
int64_t file_size;
r = toku_os_get_file_size(fd, &file_size);
if (r==0) {
int r = toku_os_get_file_size(fd, &file_size);
if (r == 0) {
report->file_size_bytes = file_size;
toku_block_table_get_fragmentation_unlocked(ft_handle->ft->blocktable, report);
ft_handle->ft->blocktable.get_fragmentation_unlocked(report);
}
toku_ft_unlock(ft_handle->ft);
return r;
......
......@@ -121,7 +121,7 @@ ft_destroy(FT ft) {
//header and checkpoint_header have same Blocktable pointer
//cannot destroy since it is still in use by CURRENT
assert(ft->h->type == FT_CURRENT);
toku_blocktable_destroy(&ft->blocktable);
ft->blocktable.destroy();
ft->cmp.destroy();
toku_destroy_dbt(&ft->descriptor.dbt);
toku_destroy_dbt(&ft->cmp_descriptor.dbt);
......@@ -203,7 +203,7 @@ static void ft_begin_checkpoint (LSN checkpoint_lsn, void *header_v) {
assert(ft->checkpoint_header == NULL);
ft_copy_for_checkpoint_unlocked(ft, checkpoint_lsn);
ft->h->dirty = 0; // this is only place this bit is cleared (in currentheader)
toku_block_translation_note_start_checkpoint_unlocked(ft->blocktable);
ft->blocktable.note_start_checkpoint_unlocked();
toku_ft_unlock (ft);
}
......@@ -239,8 +239,6 @@ ft_hack_highest_unused_msn_for_upgrade_for_checkpoint(FT ft) {
static void ft_checkpoint (CACHEFILE cf, int fd, void *header_v) {
FT ft = (FT) header_v;
FT_HEADER ch = ft->checkpoint_header;
//printf("%s:%d allocated_limit=%lu writing queue to %lu\n", __FILE__, __LINE__,
// block_allocator_allocated_limit(h->block_allocator), h->unused_blocks.b*h->nodesize);
assert(ch);
assert(ch->type == FT_CHECKPOINT_INPROGRESS);
if (ch->dirty) { // this is only place this bit is tested (in checkpoint_header)
......@@ -255,16 +253,15 @@ static void ft_checkpoint (CACHEFILE cf, int fd, void *header_v) {
ft_hack_highest_unused_msn_for_upgrade_for_checkpoint(ft);
// write translation and header to disk (or at least to OS internal buffer)
toku_serialize_ft_to(fd, ch, ft->blocktable, ft->cf);
toku_serialize_ft_to(fd, ch, &ft->blocktable, ft->cf);
ch->dirty = 0; // this is only place this bit is cleared (in checkpoint_header)
// fsync the cachefile
toku_cachefile_fsync(cf);
ft->h->checkpoint_count++; // checkpoint succeeded, next checkpoint will save to alternate header location
ft->h->checkpoint_lsn = ch->checkpoint_lsn; //Header updated.
}
else {
toku_block_translation_note_skipped_checkpoint(ft->blocktable);
} else {
ft->blocktable.note_skipped_checkpoint();
}
}
......@@ -272,14 +269,12 @@ static void ft_checkpoint (CACHEFILE cf, int fd, void *header_v) {
// free unused disk space
// (i.e. tell BlockAllocator to liberate blocks used by previous checkpoint).
// Must have access to fd (protected)
static void ft_end_checkpoint (CACHEFILE UU(cachefile), int fd, void *header_v) {
static void ft_end_checkpoint(CACHEFILE UU(cf), int fd, void *header_v) {
FT ft = (FT) header_v;
assert(ft->h->type == FT_CURRENT);
toku_block_translation_note_end_checkpoint(ft->blocktable, fd);
if (ft->checkpoint_header) {
toku_free(ft->checkpoint_header);
ft->checkpoint_header = NULL;
}
ft->blocktable.note_end_checkpoint(fd);
toku_free(ft->checkpoint_header);
ft->checkpoint_header = nullptr;
}
// maps to cf->close_userdata
......@@ -407,7 +402,7 @@ static void ft_init(FT ft, FT_OPTIONS options, CACHEFILE cf) {
ft_note_pin_by_checkpoint,
ft_note_unpin_by_checkpoint);
toku_block_verify_no_free_blocknums(ft->blocktable);
ft->blocktable.verify_no_free_blocknums();
}
......@@ -456,8 +451,8 @@ void toku_ft_create(FT *ftp, FT_OPTIONS options, CACHEFILE cf, TOKUTXN txn) {
toku_ft_init_reflock(ft);
// Assign blocknum for root block, also dirty the header
toku_blocktable_create_new(&ft->blocktable);
toku_allocate_blocknum(ft->blocktable, &ft->h->root_blocknum, ft);
ft->blocktable.create();
ft->blocktable.allocate_blocknum(&ft->h->root_blocknum, ft);
ft_init(ft, options, cf);
......@@ -875,14 +870,13 @@ toku_ft_stat64 (FT ft, struct ftstat64_s *s) {
s->verify_time_sec = ft->h->time_of_last_verification;
}
void
toku_ft_get_fractal_tree_info64(FT ft, struct ftinfo64 *s) {
toku_blocktable_get_info64(ft->blocktable, s);
void toku_ft_get_fractal_tree_info64(FT ft, struct ftinfo64 *info) {
ft->blocktable.get_info64(info);
}
int toku_ft_iterate_fractal_tree_block_map(FT ft, int (*iter)(uint64_t,int64_t,int64_t,int64_t,int64_t,void*), void *iter_extra) {
uint64_t this_checkpoint_count = ft->h->checkpoint_count;
return toku_blocktable_iterate_translation_tables(ft->blocktable, this_checkpoint_count, iter, iter_extra);
return ft->blocktable.iterate_translation_tables(this_checkpoint_count, iter, iter_extra);
}
void
......@@ -908,7 +902,7 @@ toku_ft_update_descriptor_with_fd(FT ft, DESCRIPTOR desc, int fd) {
// make space for the new descriptor and write it out to disk
DISKOFF offset, size;
size = toku_serialize_descriptor_size(desc) + 4;
toku_realloc_descriptor_on_disk(ft->blocktable, size, &offset, ft, fd);
ft->blocktable.realloc_descriptor_on_disk(size, &offset, ft, fd);
toku_serialize_descriptor_contents_to_fd(fd, desc, offset);
// cleanup the old descriptor and set the in-memory descriptor to the new one
......@@ -1086,7 +1080,7 @@ void toku_ft_get_garbage(FT ft, uint64_t *total_space, uint64_t *used_space) {
.total_space = 0,
.used_space = 0
};
toku_blocktable_iterate(ft->blocktable, TRANSLATION_CHECKPOINTED, garbage_helper, &info, true, true);
ft->blocktable.iterate(block_table::TRANSLATION_CHECKPOINTED, garbage_helper, &info, true, true);
*total_space = info.total_space;
*used_space = info.used_space;
}
......
......@@ -111,6 +111,9 @@ void toku_ft_destroy_reflock(FT ft);
void toku_ft_grab_reflock(FT ft);
void toku_ft_release_reflock(FT ft);
void toku_ft_lock(struct ft *ft);
void toku_ft_unlock(struct ft *ft);
void toku_ft_create(FT *ftp, FT_OPTIONS options, CACHEFILE cf, TOKUTXN txn);
void toku_ft_free (FT ft);
......
......@@ -269,32 +269,30 @@ bool toku_logger_rollback_is_open (TOKULOGGER logger) {
#define MAX_CACHED_ROLLBACK_NODES 4096
void
toku_logger_initialize_rollback_cache(TOKULOGGER logger, FT ft) {
toku_free_unused_blocknums(ft->blocktable, ft->h->root_blocknum);
void toku_logger_initialize_rollback_cache(TOKULOGGER logger, FT ft) {
ft->blocktable.free_unused_blocknums(ft->h->root_blocknum);
logger->rollback_cache.init(MAX_CACHED_ROLLBACK_NODES);
}
int
toku_logger_open_rollback(TOKULOGGER logger, CACHETABLE cachetable, bool create) {
int toku_logger_open_rollback(TOKULOGGER logger, CACHETABLE cachetable, bool create) {
assert(logger->is_open);
assert(!logger->rollback_cachefile);
FT_HANDLE t = NULL; // Note, there is no DB associated with this FT.
toku_ft_handle_create(&t);
int r = toku_ft_handle_open(t, toku_product_name_strings.rollback_cachefile, create, create, cachetable, nullptr);
FT_HANDLE ft_handle = nullptr; // Note, there is no DB associated with this FT.
toku_ft_handle_create(&ft_handle);
int r = toku_ft_handle_open(ft_handle, toku_product_name_strings.rollback_cachefile, create, create, cachetable, nullptr);
if (r == 0) {
logger->rollback_cachefile = t->ft->cf;
toku_logger_initialize_rollback_cache(logger, t->ft);
//Verify it is empty
//Must have no data blocks (rollback logs or otherwise).
toku_block_verify_no_data_blocks_except_root(t->ft->blocktable, t->ft->h->root_blocknum);
bool is_empty;
is_empty = toku_ft_is_empty_fast(t);
FT ft = ft_handle->ft;
logger->rollback_cachefile = ft->cf;
toku_logger_initialize_rollback_cache(logger, ft_handle->ft);
// Verify it is empty
// Must have no data blocks (rollback logs or otherwise).
ft->blocktable.verify_no_data_blocks_except_root(ft->h->root_blocknum);
bool is_empty = toku_ft_is_empty_fast(ft_handle);
assert(is_empty);
} else {
toku_ft_handle_close(t);
toku_ft_handle_close(ft_handle);
}
return r;
}
......@@ -314,9 +312,9 @@ void toku_logger_close_rollback_check_empty(TOKULOGGER logger, bool clean_shutdo
if (clean_shutdown) {
//Verify it is safe to close it.
assert(!ft->h->dirty); //Must not be dirty.
toku_free_unused_blocknums(ft->blocktable, ft->h->root_blocknum);
//Must have no data blocks (rollback logs or otherwise).
toku_block_verify_no_data_blocks_except_root(ft->blocktable, ft->h->root_blocknum);
ft->blocktable.free_unused_blocknums(ft->h->root_blocknum);
// Must have no data blocks (rollback logs or otherwise).
ft->blocktable.verify_no_data_blocks_except_root(ft->h->root_blocknum);
assert(!ft->h->dirty);
} else {
ft->h->dirty = 0;
......
......@@ -114,7 +114,7 @@ static inline bool ba_trace_enabled() {
#endif
}
void block_allocator::create(uint64_t reserve_at_beginning, uint64_t alignment) {
void block_allocator::_create_internal(uint64_t reserve_at_beginning, uint64_t alignment) {
// the alignment must be at least 512 and aligned with 512 to work with direct I/O
assert(alignment >= 512 && (alignment % 512) == 0);
......@@ -127,7 +127,10 @@ void block_allocator::create(uint64_t reserve_at_beginning, uint64_t alignment)
_strategy = BA_STRATEGY_FIRST_FIT;
VALIDATE();
}
void block_allocator::create(uint64_t reserve_at_beginning, uint64_t alignment) {
_create_internal(reserve_at_beginning, alignment);
if (ba_trace_enabled()) {
fprintf(stderr, "ba_trace_create %p\n", this);
}
......@@ -161,41 +164,6 @@ void block_allocator::grow_blocks_array() {
grow_blocks_array_by(1);
}
void block_allocator::merge_blockpairs_into(uint64_t d, struct blockpair dst[],
uint64_t s, const struct blockpair src[])
{
uint64_t tail = d+s;
while (d > 0 && s > 0) {
struct blockpair *dp = &dst[d - 1];
struct blockpair const *sp = &src[s - 1];
struct blockpair *tp = &dst[tail - 1];
assert(tail > 0);
if (dp->offset > sp->offset) {
*tp = *dp;
d--;
tail--;
} else {
*tp = *sp;
s--;
tail--;
}
}
while (d > 0) {
struct blockpair *dp = &dst[d - 1];
struct blockpair *tp = &dst[tail - 1];
*tp = *dp;
d--;
tail--;
}
while (s > 0) {
struct blockpair const *sp = &src[s - 1];
struct blockpair *tp = &dst[tail - 1];
*tp = *sp;
s--;
tail--;
}
}
int block_allocator::compare_blockpairs(const void *av, const void *bv) {
const struct blockpair *a = (const struct blockpair *) av;
const struct blockpair *b = (const struct blockpair *) bv;
......@@ -208,30 +176,25 @@ int block_allocator::compare_blockpairs(const void *av, const void *bv) {
}
}
// See the documentation in block_allocator.h
void block_allocator::alloc_blocks_at(uint64_t n_blocks, struct blockpair pairs[]) {
VALIDATE();
qsort(pairs, n_blocks, sizeof(*pairs), compare_blockpairs);
for (uint64_t i = 0; i < n_blocks; i++) {
assert(pairs[i].offset >= _reserve_at_beginning);
assert(pairs[i].offset % _alignment == 0);
_n_bytes_in_use += pairs[i].size;
void block_allocator::create_from_blockpairs(uint64_t reserve_at_beginning, uint64_t alignment,
struct blockpair *pairs, uint64_t n_blocks) {
_create_internal(reserve_at_beginning, alignment);
for (uint64_t i = 0; i < _n_blocks; i++) {
// Allocator does not support size 0 blocks. See block_allocator_free_block.
invariant(pairs[i].size > 0);
invariant(pairs[i].offset >= _reserve_at_beginning);
invariant(pairs[i].offset % _alignment == 0);
_n_bytes_in_use += pairs[i].size;
}
grow_blocks_array_by(n_blocks);
merge_blockpairs_into(_n_blocks, _blocks_array, n_blocks, pairs);
_n_blocks += n_blocks;
VALIDATE();
}
_n_blocks = n_blocks;
void block_allocator::alloc_block_at(uint64_t size, uint64_t offset) {
struct blockpair p(offset, size);
grow_blocks_array_by(_n_blocks);
memcpy(_blocks_array, pairs, _n_blocks * sizeof(struct blockpair));
qsort(_blocks_array, _n_blocks, sizeof(struct blockpair), compare_blockpairs);
// Just do a linear search for the block.
// This data structure is a sorted array (no gaps or anything), so the search isn't really making this any slower than the insertion.
// To speed up the insertion when opening a file, we provide the block_allocator_alloc_blocks_at function.
alloc_blocks_at(1, &p);
VALIDATE();
}
// Effect: align a value by rounding up.
......
......@@ -128,6 +128,14 @@ public:
BA_STRATEGY_FIRST_FIT = 1
};
struct blockpair {
uint64_t offset;
uint64_t size;
blockpair(uint64_t o, uint64_t s) :
offset(o), size(s) {
}
};
// Effect: Create a block allocator, in which the first RESERVE_AT_BEGINNING bytes are not put into a block.
// The default allocation strategy is first fit (BA_STRATEGY_FIRST_FIT)
// All blocks be start on a multiple of ALIGNMENT.
......@@ -137,6 +145,19 @@ public:
// alignment (IN) Block alignment.
void create(uint64_t reserve_at_beginning, uint64_t alignment);
// Effect: Create a block allocator, in which the first RESERVE_AT_BEGINNING bytes are not put into a block.
// The default allocation strategy is first fit (BA_STRATEGY_FIRST_FIT)
// The allocator is initialized to contain `n_blocks' of blockpairs, taken from `pairs'
// All blocks be start on a multiple of ALIGNMENT.
// Aborts if we run out of memory.
// Parameters
// pairs, unowned array of pairs to copy
// n_blocks, Size of pairs array
// reserve_at_beginning (IN) Size of reserved block at beginning. This size does not have to be aligned.
// alignment (IN) Block alignment.
void create_from_blockpairs(uint64_t reserve_at_beginning, uint64_t alignment,
struct blockpair *pairs, uint64_t n_blocks);
// Effect: Destroy this block allocator
void destroy();
......@@ -144,35 +165,10 @@ public:
// Requires: No other threads are operating on this block allocator
void set_strategy(enum allocation_strategy strategy);
// Effect: Allocate a block of the specified size at a particular offset.
// Aborts if anything goes wrong.
// The performance of this function may be as bad as Theta(N), where N is the number of blocks currently in use.
// Usage note: To allocate several blocks (e.g., when opening a FT), use block_allocator_alloc_blocks_at().
// Requires: The resulting block may not overlap any other allocated block.
// And the offset must be a multiple of the block alignment.
// Parameters:
// size (IN): The size of the block.
// offset (IN): The location of the block.
void alloc_block_at(uint64_t size, uint64_t offset);
struct blockpair {
uint64_t offset;
uint64_t size;
blockpair(uint64_t o, uint64_t s) :
offset(o), size(s) {
}
};
// Effect: Take pairs in any order, and add them all, as if we did block_allocator_alloc_block() on each pair.
// This should run in time O(N + M log M) where N is the number of blocks in ba, and M is the number of new blocks.
// Modifies: pairs (sorts them).
void alloc_blocks_at(uint64_t n_blocks, blockpair *pairs);
// Effect: Allocate a block of the specified size at an address chosen by the allocator.
// Aborts if anything goes wrong.
// The block address will be a multiple of the alignment.
// Parameters:
// ba (IN/OUT): The block allocator. (Modifies ba.)
// size (IN): The size of the block. (The size does not have to be aligned.)
// offset (OUT): The location of the block.
void alloc_block(uint64_t size, uint64_t *offset);
......@@ -180,14 +176,12 @@ public:
// Effect: Free the block at offset.
// Requires: There must be a block currently allocated at that offset.
// Parameters:
// ba (IN/OUT): The block allocator. (Modifies ba.)
// offset (IN): The offset of the block.
void free_block(uint64_t offset);
// Effect: Return the size of the block that starts at offset.
// Requires: There must be a block currently allocated at that offset.
// Parameters:
// ba (IN/OUT): The block allocator. (Modifies ba.)
// offset (IN): The offset of the block.
uint64_t block_size(uint64_t offset);
......@@ -221,18 +215,8 @@ public:
// report->checkpoint_bytes_additional is ignored on return
void get_statistics(TOKU_DB_FRAGMENTATION report);
// Effect: Merge dst[d] and src[s] into dst[d+s], merging in place.
// Initially dst and src hold sorted arrays (sorted by increasing offset).
// Finally dst contains all d+s elements sorted in order.
// Requires:
// dst and src are sorted.
// dst must be large enough (sizeof(dst) >= d && sizeof(src) >= s)
// No blocks may overlap.
// Rationale: This is exposed so it can be tested by a glass box tester.
static void merge_blockpairs_into(uint64_t d, struct blockpair dst[],
uint64_t s, const struct blockpair src[]);
private:
void _create_internal(uint64_t reserve_at_beginning, uint64_t alignment);
void grow_blocks_array_by(uint64_t n_to_add);
void grow_blocks_array();
int64_t find_block(uint64_t offset);
......
......@@ -89,13 +89,16 @@ PATENT RIGHTS GRANT:
#ident "Copyright (c) 2007-2013 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 "portability/toku_portability.h"
#include "portability/memory.h"
#include "portability/toku_assert.h"
#include "portability/toku_portability.h"
#include "portability/toku_pthread.h"
#include "ft/ft-internal.h" // ugly but pragmatic, need access to dirty bits while holding translation lock
// TODO: reorganize this dependency
// ugly but pragmatic, need access to dirty bits while holding translation lock
// TODO: Refactor this (possibly with FT-301)
#include "ft/ft-internal.h"
// TODO: reorganize this dependency (FT-303)
#include "ft/ft-ops.h" // for toku_maybe_truncate_file
#include "ft/serialize/block_table.h"
#include "ft/serialize/rbuf.h"
......@@ -103,124 +106,164 @@ PATENT RIGHTS GRANT:
#include "ft/serialize/block_allocator.h"
#include "util/nb_mutex.h"
#include "util/scoped_malloc.h"
// indicates the end of a freelist
static const BLOCKNUM freelist_null = { -1 };
// value of block_translation_pair.size if blocknum is unused
static const DISKOFF size_is_free = (DISKOFF) -1;
// value of block_translation_pair.u.diskoff if blocknum is used but does not yet have a diskblock
static const DISKOFF diskoff_unused = (DISKOFF) -2;
void block_table::_mutex_lock() {
toku_mutex_lock(&_mutex);
}
void block_table::_mutex_unlock() {
toku_mutex_unlock(&_mutex);
}
// TODO: Move lock to FT
void toku_ft_lock(FT ft) {
block_table *bt = &ft->blocktable;
bt->_mutex_lock();
}
// TODO: Move lock to FT
void toku_ft_unlock(FT ft) {
block_table *bt = &ft->blocktable;
toku_mutex_assert_locked(&bt->_mutex);
bt->_mutex_unlock();
}
// There are two headers: the reserve must fit them both and be suitably aligned.
static_assert(block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE %
block_allocator::BLOCK_ALLOCATOR_ALIGNMENT == 0,
"Block allocator's header reserve must be suitibly aligned");
static_assert(block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE * 2 ==
block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE,
"Block allocator's total header reserve must exactly fit two headers");
// does NOT initialize the block allocator: the caller is responsible
void block_table::_create_internal() {
memset(&_mutex, 0, sizeof(_mutex));
toku_mutex_init(&_mutex, nullptr);
nb_mutex_init(&_safe_file_size_lock);
_checkpointed.type = TRANSLATION_CHECKPOINTED;
_checkpointed.smallest_never_used_blocknum = make_blocknum(RESERVED_BLOCKNUMS);
_checkpointed.length_of_array = _checkpointed.smallest_never_used_blocknum.b;
_checkpointed.blocknum_freelist_head = freelist_null;
XMALLOC_N(_checkpointed.length_of_array, _checkpointed.block_translation);
for (int64_t i = 0; i < _checkpointed.length_of_array; i++) {
_checkpointed.block_translation[i].size = 0;
_checkpointed.block_translation[i].u.diskoff = diskoff_unused;
}
// we just created a default checkpointed, now copy it to current.
_copy_translation(&_current, &_checkpointed, TRANSLATION_CURRENT);
}
// Fill in the checkpointed translation from buffer, and copy checkpointed to current.
// The one read from disk is the last known checkpointed one, so we are keeping it in
// place and then setting current (which is never stored on disk) for current use.
// The translation_buffer has translation only, we create the rest of the block_table.
int block_table::create_from_buffer(int fd,
DISKOFF location_on_disk, //Location of translation_buffer
DISKOFF size_on_disk,
unsigned char *translation_buffer) {
// Does not initialize the block allocator
_create_internal();
// Deserialize the translation and copy it to current
int r = _translation_deserialize_from_buffer(&_checkpointed,
location_on_disk, size_on_disk,
translation_buffer);
if (r != 0) {
return r;
}
_copy_translation(&_current, &_checkpointed, TRANSLATION_CURRENT);
// Determine the file size
int64_t file_size;
r = toku_os_get_file_size(fd, &file_size);
lazy_assert_zero(r);
invariant(file_size >= 0);
_safe_file_size = file_size;
// Gather the non-empty translations and use them to create the block allocator
toku::scoped_malloc pairs_buf(_checkpointed.smallest_never_used_blocknum.b *
sizeof(struct block_allocator::blockpair));
struct block_allocator::blockpair *CAST_FROM_VOIDP(pairs, pairs_buf.get());
uint64_t n_pairs = 0;
for (int64_t i = 0; i < _checkpointed.smallest_never_used_blocknum.b; i++) {
struct block_translation_pair pair = _checkpointed.block_translation[i];
if (pair.size > 0) {
invariant(pair.u.diskoff != diskoff_unused);
pairs[n_pairs++] = block_allocator::blockpair(pair.u.diskoff, pair.size);
}
}
_bt_block_allocator.create_from_blockpairs(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE,
block_allocator::BLOCK_ALLOCATOR_ALIGNMENT,
pairs, n_pairs);
return 0;
}
void block_table::create() {
// Does not initialize the block allocator
_create_internal();
// Create an empty block allocator.
_bt_block_allocator.create(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE,
block_allocator::BLOCK_ALLOCATOR_ALIGNMENT);
}
//When the translation (btt) is stored on disk:
// In Header:
// size_on_disk
// location_on_disk
// In block translation table (in order):
// smallest_never_used_blocknum
// blocknum_freelist_head
// array
// a checksum
struct translation { //This is the BTT (block translation table)
enum translation_type type;
int64_t length_of_array; //Number of elements in array (block_translation). always >= smallest_never_used_blocknum
BLOCKNUM smallest_never_used_blocknum;
BLOCKNUM blocknum_freelist_head; // next (previously used) unused blocknum (free list)
struct block_translation_pair *block_translation;
// Where and how big is the block translation vector stored on disk.
// size_on_disk is stored in block_translation[RESERVED_BLOCKNUM_TRANSLATION].size
// location_on is stored in block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff
};
static const BLOCKNUM freelist_null = {-1}; // in a freelist, this indicates end of list
static const DISKOFF size_is_free = (DISKOFF)-1; // value of block_translation_pair.size if blocknum is unused
static const DISKOFF diskoff_unused = (DISKOFF)-2; // value of block_translation_pair.u.diskoff if blocknum is used but does not yet have a diskblock
/********
* There are three copies of the translation table (btt) in the block table:
*
* checkpointed Is initialized by deserializing from disk,
* and is the only version ever read from disk.
* When read from disk it is copied to current.
* It is immutable. It can be replaced by an inprogress btt.
*
* inprogress Is only filled by copying from current,
* and is the only version ever serialized to disk.
* (It is serialized to disk on checkpoint and clean shutdown.)
* At end of checkpoint it replaces 'checkpointed'.
* During a checkpoint, any 'pending' dirty writes will update
* inprogress.
*
* current Is initialized by copying from checkpointed,
* is the only version ever modified while the database is in use,
* and is the only version ever copied to inprogress.
* It is never stored on disk.
********/
struct block_table {
struct translation current; // The current translation is the one used by client threads. It is not represented on disk.
struct translation inprogress; // the translation used by the checkpoint currently in progress. If the checkpoint thread allocates a block, it must also update the current translation.
struct translation checkpointed; // the translation for the data that shall remain inviolate on disk until the next checkpoint finishes, after which any blocks used only in this translation can be freed.
// The in-memory data structure for block allocation. There is no on-disk data structure for block allocation.
// Note: This is *allocation* not *translation*. The bt_block_allocator is unaware of which blocks are used for which translation, but simply allocates and deallocates blocks.
block_allocator bt_block_allocator;
toku_mutex_t mutex;
struct nb_mutex safe_file_size_lock;
bool checkpoint_skipped;
uint64_t safe_file_size;
};
//forward decls
static int64_t calculate_size_on_disk (struct translation *t);
static inline bool translation_prevents_freeing (struct translation *t, BLOCKNUM b, struct block_translation_pair *old_pair);
static inline void lock_for_blocktable (BLOCK_TABLE bt);
static inline void unlock_for_blocktable (BLOCK_TABLE bt);
static void
ft_set_dirty(FT ft, bool for_checkpoint){
toku_mutex_assert_locked(&ft->blocktable->mutex);
paranoid_invariant(ft->h->type == FT_CURRENT);
// TODO: Refactor with FT-303
static void ft_set_dirty(FT ft, bool for_checkpoint) {
invariant(ft->h->type == FT_CURRENT);
if (for_checkpoint) {
paranoid_invariant(ft->checkpoint_header->type == FT_CHECKPOINT_INPROGRESS);
invariant(ft->checkpoint_header->type == FT_CHECKPOINT_INPROGRESS);
ft->checkpoint_header->dirty = 1;
}
else {
} else {
ft->h->dirty = 1;
}
}
static void
maybe_truncate_file(BLOCK_TABLE bt, int fd, uint64_t size_needed_before) {
toku_mutex_assert_locked(&bt->mutex);
uint64_t new_size_needed = bt->bt_block_allocator.allocated_limit();
void block_table::_maybe_truncate_file(int fd, uint64_t size_needed_before) {
toku_mutex_assert_locked(&_mutex);
uint64_t new_size_needed = _bt_block_allocator.allocated_limit();
//Save a call to toku_os_get_file_size (kernel call) if unlikely to be useful.
if (new_size_needed < size_needed_before && new_size_needed < bt->safe_file_size) {
nb_mutex_lock(&bt->safe_file_size_lock, &bt->mutex);
if (new_size_needed < size_needed_before && new_size_needed < _safe_file_size) {
nb_mutex_lock(&_safe_file_size_lock, &_mutex);
// Must hold safe_file_size_lock to change safe_file_size.
if (new_size_needed < bt->safe_file_size) {
int64_t safe_file_size_before = bt->safe_file_size;
// Must hold _safe_file_size_lock to change _safe_file_size.
if (new_size_needed < _safe_file_size) {
int64_t safe_file_size_before = _safe_file_size;
// Not safe to use the 'to-be-truncated' portion until truncate is done.
bt->safe_file_size = new_size_needed;
unlock_for_blocktable(bt);
_safe_file_size = new_size_needed;
_mutex_unlock();
uint64_t size_after;
toku_maybe_truncate_file(fd, new_size_needed, safe_file_size_before, &size_after);
lock_for_blocktable(bt);
_mutex_lock();
bt->safe_file_size = size_after;
_safe_file_size = size_after;
}
nb_mutex_unlock(&bt->safe_file_size_lock);
nb_mutex_unlock(&_safe_file_size_lock);
}
}
void
toku_maybe_truncate_file_on_open(BLOCK_TABLE bt, int fd) {
lock_for_blocktable(bt);
maybe_truncate_file(bt, fd, bt->safe_file_size);
unlock_for_blocktable(bt);
void block_table::maybe_truncate_file_on_open(int fd) {
_mutex_lock();
_maybe_truncate_file(fd, _safe_file_size);
_mutex_unlock();
}
static void
copy_translation(struct translation * dst, struct translation * src, enum translation_type newtype) {
void block_table::_copy_translation(struct translation * dst, struct translation * src, enum translation_type newtype) {
paranoid_invariant(src->length_of_array >= src->smallest_never_used_blocknum.b); //verify invariant
paranoid_invariant(newtype==TRANSLATION_DEBUG ||
(src->type == TRANSLATION_CURRENT && newtype == TRANSLATION_INPROGRESS) ||
......@@ -239,10 +282,9 @@ copy_translation(struct translation * dst, struct translation * src, enum transl
dst->block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff = diskoff_unused;
}
int64_t
toku_block_get_blocks_in_use_unlocked(BLOCK_TABLE bt) {
int64_t block_table::get_blocks_in_use_unlocked() {
BLOCKNUM b;
struct translation *t = &bt->current;
struct translation *t = &_current;
int64_t num_blocks = 0;
{
//Reserved blocknums do not get upgraded; They are part of the header.
......@@ -255,8 +297,7 @@ toku_block_get_blocks_in_use_unlocked(BLOCK_TABLE bt) {
return num_blocks;
}
static void
maybe_optimize_translation(struct translation *t) {
void block_table::_maybe_optimize_translation(struct translation *t) {
//Reduce 'smallest_never_used_blocknum.b' (completely free blocknums instead of just
//on a free list. Doing so requires us to regenerate the free list.
//This is O(n) work, so do it only if you're already doing that.
......@@ -295,25 +336,24 @@ maybe_optimize_translation(struct translation *t) {
}
// block table must be locked by caller of this function
void
toku_block_translation_note_start_checkpoint_unlocked (BLOCK_TABLE bt) {
toku_mutex_assert_locked(&bt->mutex);
void block_table::note_start_checkpoint_unlocked() {
toku_mutex_assert_locked(&_mutex);
// Copy current translation to inprogress translation.
paranoid_invariant(bt->inprogress.block_translation == NULL);
paranoid_invariant(_inprogress.block_translation == NULL);
//We're going to do O(n) work to copy the translation, so we
//can afford to do O(n) work by optimizing the translation
maybe_optimize_translation(&bt->current);
copy_translation(&bt->inprogress, &bt->current, TRANSLATION_INPROGRESS);
_maybe_optimize_translation(&_current);
_copy_translation(&_inprogress, &_current, TRANSLATION_INPROGRESS);
bt->checkpoint_skipped = false;
_checkpoint_skipped = false;
}
void toku_block_translation_note_skipped_checkpoint (BLOCK_TABLE bt) {
void block_table::note_skipped_checkpoint() {
//Purpose, alert block translation that the checkpoint was skipped, e.x. for a non-dirty header
lock_for_blocktable(bt);
paranoid_invariant_notnull(bt->inprogress.block_translation);
bt->checkpoint_skipped = true;
unlock_for_blocktable(bt);
_mutex_lock();
paranoid_invariant_notnull(_inprogress.block_translation);
_checkpoint_skipped = true;
_mutex_unlock();
}
// Purpose: free any disk space used by previous checkpoint that isn't in use by either
......@@ -326,143 +366,92 @@ void toku_block_translation_note_skipped_checkpoint (BLOCK_TABLE bt) {
// free (offset,len) from checkpoint
// move inprogress to checkpoint (resetting type)
// inprogress = NULL
void
toku_block_translation_note_end_checkpoint (BLOCK_TABLE bt, int fd) {
void block_table::note_end_checkpoint(int fd) {
// Free unused blocks
lock_for_blocktable(bt);
uint64_t allocated_limit_at_start = bt->bt_block_allocator.allocated_limit();
paranoid_invariant_notnull(bt->inprogress.block_translation);
if (bt->checkpoint_skipped) {
toku_free(bt->inprogress.block_translation);
memset(&bt->inprogress, 0, sizeof(bt->inprogress));
_mutex_lock();
uint64_t allocated_limit_at_start = _bt_block_allocator.allocated_limit();
paranoid_invariant_notnull(_inprogress.block_translation);
if (_checkpoint_skipped) {
toku_free(_inprogress.block_translation);
memset(&_inprogress, 0, sizeof(_inprogress));
goto end;
}
//Make certain inprogress was allocated space on disk
assert(bt->inprogress.block_translation[RESERVED_BLOCKNUM_TRANSLATION].size > 0);
assert(bt->inprogress.block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff > 0);
assert(_inprogress.block_translation[RESERVED_BLOCKNUM_TRANSLATION].size > 0);
assert(_inprogress.block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff > 0);
{
int64_t i;
struct translation *t = &bt->checkpointed;
for (i = 0; i < t->length_of_array; i++) {
struct translation *t = &_checkpointed;
for (int64_t i = 0; i < t->length_of_array; i++) {
struct block_translation_pair *pair = &t->block_translation[i];
if (pair->size > 0 && !translation_prevents_freeing(&bt->inprogress, make_blocknum(i), pair)) {
assert(!translation_prevents_freeing(&bt->current, make_blocknum(i), pair));
bt->bt_block_allocator.free_block(pair->u.diskoff);
if (pair->size > 0 && !_translation_prevents_freeing(&_inprogress, make_blocknum(i), pair)) {
assert(!_translation_prevents_freeing(&_current, make_blocknum(i), pair));
_bt_block_allocator.free_block(pair->u.diskoff);
}
}
toku_free(bt->checkpointed.block_translation);
bt->checkpointed = bt->inprogress;
bt->checkpointed.type = TRANSLATION_CHECKPOINTED;
memset(&bt->inprogress, 0, sizeof(bt->inprogress));
maybe_truncate_file(bt, fd, allocated_limit_at_start);
toku_free(_checkpointed.block_translation);
_checkpointed = _inprogress;
_checkpointed.type = TRANSLATION_CHECKPOINTED;
memset(&_inprogress, 0, sizeof(_inprogress));
_maybe_truncate_file(fd, allocated_limit_at_start);
}
end:
unlock_for_blocktable(bt);
_mutex_unlock();
}
__attribute__((nonnull,const))
static inline bool
is_valid_blocknum(struct translation *t, BLOCKNUM b) {
//Sanity check: Verify invariant
paranoid_invariant(t->length_of_array >= t->smallest_never_used_blocknum.b);
bool block_table::_is_valid_blocknum(struct translation *t, BLOCKNUM b) {
invariant(t->length_of_array >= t->smallest_never_used_blocknum.b);
return b.b >= 0 && b.b < t->smallest_never_used_blocknum.b;
}
static inline void
verify_valid_blocknum (struct translation *UU(t), BLOCKNUM UU(b)) {
paranoid_invariant(is_valid_blocknum(t, b));
void block_table::_verify_valid_blocknum(struct translation *UU(t), BLOCKNUM UU(b)) {
invariant(_is_valid_blocknum(t, b));
}
__attribute__((nonnull,const))
static inline bool
is_valid_freeable_blocknum(struct translation *t, BLOCKNUM b) {
//Sanity check: Verify invariant
paranoid_invariant(t->length_of_array >= t->smallest_never_used_blocknum.b);
bool block_table::_is_valid_freeable_blocknum(struct translation *t, BLOCKNUM b) {
invariant(t->length_of_array >= t->smallest_never_used_blocknum.b);
return b.b >= RESERVED_BLOCKNUMS && b.b < t->smallest_never_used_blocknum.b;
}
//Can be freed
static inline void
verify_valid_freeable_blocknum (struct translation *UU(t), BLOCKNUM UU(b)) {
paranoid_invariant(is_valid_freeable_blocknum(t, b));
}
static void
blocktable_lock_init (BLOCK_TABLE bt) {
memset(&bt->mutex, 0, sizeof(bt->mutex));
toku_mutex_init(&bt->mutex, NULL);
}
static void
blocktable_lock_destroy (BLOCK_TABLE bt) {
toku_mutex_destroy(&bt->mutex);
}
static inline void
lock_for_blocktable (BLOCK_TABLE bt) {
// Locks the blocktable_mutex.
toku_mutex_lock(&bt->mutex);
}
static inline void
unlock_for_blocktable (BLOCK_TABLE bt) {
toku_mutex_unlock(&bt->mutex);
}
void
toku_ft_lock (FT ft) {
BLOCK_TABLE bt = ft->blocktable;
lock_for_blocktable(bt);
}
void
toku_ft_unlock (FT ft) {
BLOCK_TABLE bt = ft->blocktable;
toku_mutex_assert_locked(&bt->mutex);
unlock_for_blocktable(bt);
// should be freeable
void block_table::_verify_valid_freeable_blocknum(struct translation *UU(t), BLOCKNUM UU(b)) {
invariant(_is_valid_freeable_blocknum(t, b));
}
// Also used only in ft-serialize-test.
void
toku_block_free(BLOCK_TABLE bt, uint64_t offset) {
lock_for_blocktable(bt);
bt->bt_block_allocator.free_block(offset);
unlock_for_blocktable(bt);
void block_table::block_free(uint64_t offset) {
_mutex_lock();
_bt_block_allocator.free_block(offset);
_mutex_unlock();
}
static int64_t
calculate_size_on_disk (struct translation *t) {
int64_t r = (8 + // smallest_never_used_blocknum
8 + // blocknum_freelist_head
t->smallest_never_used_blocknum.b * 16 + // Array
4); // 4 for checksum
return r;
int64_t block_table::_calculate_size_on_disk(struct translation *t) {
return 8 + // smallest_never_used_blocknum
8 + // blocknum_freelist_head
t->smallest_never_used_blocknum.b * 16 + // Array
4; // 4 for checksum
}
// We cannot free the disk space allocated to this blocknum if it is still in use by the given translation table.
static inline bool
translation_prevents_freeing(struct translation *t, BLOCKNUM b, struct block_translation_pair *old_pair) {
return (t->block_translation &&
b.b < t->smallest_never_used_blocknum.b &&
old_pair->u.diskoff == t->block_translation[b.b].u.diskoff);
bool block_table::_translation_prevents_freeing(struct translation *t, BLOCKNUM b, struct block_translation_pair *old_pair) {
return t->block_translation &&
b.b < t->smallest_never_used_blocknum.b &&
old_pair->u.diskoff == t->block_translation[b.b].u.diskoff;
}
static void
blocknum_realloc_on_disk_internal (BLOCK_TABLE bt, BLOCKNUM b, DISKOFF size, DISKOFF *offset, FT ft, bool for_checkpoint) {
toku_mutex_assert_locked(&bt->mutex);
void block_table::_realloc_on_disk_internal(BLOCKNUM b, DISKOFF size, DISKOFF *offset, FT ft, bool for_checkpoint) {
toku_mutex_assert_locked(&_mutex);
ft_set_dirty(ft, for_checkpoint);
struct translation *t = &bt->current;
struct translation *t = &_current;
struct block_translation_pair old_pair = t->block_translation[b.b];
//Free the old block if it is not still in use by the checkpoint in progress or the previous checkpoint
bool cannot_free = (bool)
((!for_checkpoint && translation_prevents_freeing(&bt->inprogress, b, &old_pair)) ||
translation_prevents_freeing(&bt->checkpointed, b, &old_pair));
((!for_checkpoint && _translation_prevents_freeing(&_inprogress, b, &old_pair)) ||
_translation_prevents_freeing(&_checkpointed, b, &old_pair));
if (!cannot_free && old_pair.u.diskoff!=diskoff_unused) {
bt->bt_block_allocator.free_block(old_pair.u.diskoff);
_bt_block_allocator.free_block(old_pair.u.diskoff);
}
uint64_t allocator_offset = diskoff_unused;
......@@ -470,90 +459,84 @@ blocknum_realloc_on_disk_internal (BLOCK_TABLE bt, BLOCKNUM b, DISKOFF size, DIS
if (size > 0) {
// Allocate a new block if the size is greater than 0,
// if the size is just 0, offset will be set to diskoff_unused
bt->bt_block_allocator.alloc_block(size, &allocator_offset);
_bt_block_allocator.alloc_block(size, &allocator_offset);
}
t->block_translation[b.b].u.diskoff = allocator_offset;
*offset = allocator_offset;
//Update inprogress btt if appropriate (if called because Pending bit is set).
if (for_checkpoint) {
paranoid_invariant(b.b < bt->inprogress.length_of_array);
bt->inprogress.block_translation[b.b] = t->block_translation[b.b];
paranoid_invariant(b.b < _inprogress.length_of_array);
_inprogress.block_translation[b.b] = t->block_translation[b.b];
}
}
static void
ensure_safe_write_unlocked(BLOCK_TABLE bt, int fd, DISKOFF block_size, DISKOFF block_offset) {
// Requires: holding bt->mutex
void block_table::_ensure_safe_write_unlocked(int fd, DISKOFF block_size, DISKOFF block_offset) {
// Requires: holding _mutex
uint64_t size_needed = block_size + block_offset;
if (size_needed > bt->safe_file_size) {
// Must hold safe_file_size_lock to change safe_file_size.
nb_mutex_lock(&bt->safe_file_size_lock, &bt->mutex);
if (size_needed > bt->safe_file_size) {
unlock_for_blocktable(bt);
if (size_needed > _safe_file_size) {
// Must hold _safe_file_size_lock to change _safe_file_size.
nb_mutex_lock(&_safe_file_size_lock, &_mutex);
if (size_needed > _safe_file_size) {
_mutex_unlock();
int64_t size_after;
toku_maybe_preallocate_in_file(fd, size_needed, bt->safe_file_size, &size_after);
toku_maybe_preallocate_in_file(fd, size_needed, _safe_file_size, &size_after);
lock_for_blocktable(bt);
bt->safe_file_size = size_after;
_mutex_lock();
_safe_file_size = size_after;
}
nb_mutex_unlock(&bt->safe_file_size_lock);
nb_mutex_unlock(&_safe_file_size_lock);
}
}
void
toku_blocknum_realloc_on_disk (BLOCK_TABLE bt, BLOCKNUM b, DISKOFF size, DISKOFF *offset, FT ft, int fd, bool for_checkpoint) {
lock_for_blocktable(bt);
struct translation *t = &bt->current;
verify_valid_freeable_blocknum(t, b);
blocknum_realloc_on_disk_internal(bt, b, size, offset, ft, for_checkpoint);
void block_table::realloc_on_disk(BLOCKNUM b, DISKOFF size, DISKOFF *offset, FT ft, int fd, bool for_checkpoint) {
_mutex_lock();
struct translation *t = &_current;
_verify_valid_freeable_blocknum(t, b);
_realloc_on_disk_internal(b, size, offset, ft, for_checkpoint);
ensure_safe_write_unlocked(bt, fd, size, *offset);
unlock_for_blocktable(bt);
_ensure_safe_write_unlocked(fd, size, *offset);
_mutex_unlock();
}
__attribute__((nonnull,const))
static inline bool
pair_is_unallocated(struct block_translation_pair *pair) {
bool block_table::_pair_is_unallocated(struct block_translation_pair *pair) {
return pair->size == 0 && pair->u.diskoff == diskoff_unused;
}
static void blocknum_alloc_translation_on_disk_unlocked(BLOCK_TABLE bt)
// Effect: figure out where to put the inprogress btt on disk, allocate space for it there.
// The space must be 512-byte aligned (both the starting address and the size).
// As a result, the allcoated space may be a little bit bigger (up to the next 512-byte boundary) than the actual btt.
{
toku_mutex_assert_locked(&bt->mutex);
void block_table::_alloc_inprogress_translation_on_disk_unlocked() {
toku_mutex_assert_locked(&_mutex);
struct translation *t = &bt->inprogress;
struct translation *t = &_inprogress;
paranoid_invariant_notnull(t->block_translation);
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_TRANSLATION);
//Each inprogress is allocated only once
paranoid_invariant(pair_is_unallocated(&t->block_translation[b.b]));
paranoid_invariant(_pair_is_unallocated(&t->block_translation[b.b]));
//Allocate a new block
int64_t size = calculate_size_on_disk(t);
int64_t size = _calculate_size_on_disk(t);
uint64_t offset;
bt->bt_block_allocator.alloc_block(size, &offset);
_bt_block_allocator.alloc_block(size, &offset);
t->block_translation[b.b].u.diskoff = offset;
t->block_translation[b.b].size = size;
}
void toku_serialize_translation_to_wbuf(BLOCK_TABLE bt, int fd, struct wbuf *w,
int64_t *address, int64_t *size)
// Effect: Fills wbuf (which starts uninitialized) with bt
// Effect: Serializes the blocktable to a wbuf (which starts uninitialized)
// A clean shutdown runs checkpoint start so that current and inprogress are copies.
// The resulting wbuf buffer is guaranteed to be be 512-byte aligned and the total length is a multiple of 512 (so we pad with zeros at the end if needd)
// The address is guaranteed to be 512-byte aligned, but the size is not guaranteed.
// It *is* guaranteed that we can read up to the next 512-byte boundary, however
{
lock_for_blocktable(bt);
struct translation *t = &bt->inprogress;
void block_table::serialize_translation_to_wbuf(int fd, struct wbuf *w,
int64_t *address, int64_t *size) {
_mutex_lock();
struct translation *t = &_inprogress;
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_TRANSLATION);
blocknum_alloc_translation_on_disk_unlocked(bt); // The allocated block must be 512-byte aligned to make O_DIRECT happy.
uint64_t size_translation = calculate_size_on_disk(t);
_alloc_inprogress_translation_on_disk_unlocked(); // The allocated block must be 512-byte aligned to make O_DIRECT happy.
uint64_t size_translation = _calculate_size_on_disk(t);
uint64_t size_aligned = roundup_to_multiple(512, size_translation);
assert((int64_t)size_translation==t->block_translation[b.b].size);
{
......@@ -579,33 +562,33 @@ void toku_serialize_translation_to_wbuf(BLOCK_TABLE bt, int fd, struct wbuf *w,
*size = size_translation;
assert((*address)%512 == 0);
ensure_safe_write_unlocked(bt, fd, size_aligned, *address);
unlock_for_blocktable(bt);
_ensure_safe_write_unlocked(fd, size_aligned, *address);
_mutex_unlock();
}
// Perhaps rename: purpose is get disk address of a block, given its blocknum (blockid?)
static void
translate_blocknum_to_offset_size_unlocked(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF *offset, DISKOFF *size) {
struct translation *t = &bt->current;
verify_valid_blocknum(t, b);
if (offset) *offset = t->block_translation[b.b].u.diskoff;
if (size) *size = t->block_translation[b.b].size;
void block_table::_translate_blocknum_to_offset_size_unlocked(BLOCKNUM b, DISKOFF *offset, DISKOFF *size) {
struct translation *t = &_current;
_verify_valid_blocknum(t, b);
if (offset) {
*offset = t->block_translation[b.b].u.diskoff;
}
if (size) {
*size = t->block_translation[b.b].size;
}
}
// Perhaps rename: purpose is get disk address of a block, given its blocknum (blockid?)
void
toku_translate_blocknum_to_offset_size(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF *offset, DISKOFF *size) {
lock_for_blocktable(bt);
translate_blocknum_to_offset_size_unlocked(bt, b, offset, size);
unlock_for_blocktable(bt);
void block_table::translate_blocknum_to_offset_size(BLOCKNUM b, DISKOFF *offset, DISKOFF *size) {
_mutex_lock();
_translate_blocknum_to_offset_size_unlocked(b, offset, size);
_mutex_unlock();
}
//Only called by toku_allocate_blocknum
static void
maybe_expand_translation (struct translation *t) {
// Only called by toku_allocate_blocknum
// Effect: expand the array to maintain size invariant
// given that one more never-used blocknum will soon be used.
void block_table::_maybe_expand_translation(struct translation *t) {
if (t->length_of_array <= t->smallest_never_used_blocknum.b) {
//expansion is necessary
uint64_t new_length = t->smallest_never_used_blocknum.b * 2;
......@@ -619,15 +602,14 @@ maybe_expand_translation (struct translation *t) {
}
}
void
toku_allocate_blocknum_unlocked(BLOCK_TABLE bt, BLOCKNUM *res, FT ft) {
toku_mutex_assert_locked(&bt->mutex);
void block_table::_allocate_blocknum_unlocked(BLOCKNUM *res, FT ft) {
toku_mutex_assert_locked(&_mutex);
BLOCKNUM result;
struct translation * t = &bt->current;
struct translation *t = &_current;
if (t->blocknum_freelist_head.b == freelist_null.b) {
// no previously used blocknums are available
// use a never used blocknum
maybe_expand_translation(t); //Ensure a never used blocknums is available
_maybe_expand_translation(t); //Ensure a never used blocknums is available
result = t->smallest_never_used_blocknum;
t->smallest_never_used_blocknum.b++;
} else { // reuse a previously used blocknum
......@@ -640,22 +622,19 @@ toku_allocate_blocknum_unlocked(BLOCK_TABLE bt, BLOCKNUM *res, FT ft) {
//blocknum is not free anymore
t->block_translation[result.b].u.diskoff = diskoff_unused;
t->block_translation[result.b].size = 0;
verify_valid_freeable_blocknum(t, result);
_verify_valid_freeable_blocknum(t, result);
*res = result;
ft_set_dirty(ft, false);
}
void
toku_allocate_blocknum(BLOCK_TABLE bt, BLOCKNUM *res, FT ft) {
lock_for_blocktable(bt);
toku_allocate_blocknum_unlocked(bt, res, ft);
unlock_for_blocktable(bt);
void block_table::allocate_blocknum(BLOCKNUM *res, FT ft) {
_mutex_lock();
_allocate_blocknum_unlocked(res, ft);
_mutex_unlock();
}
static void
free_blocknum_in_translation(struct translation *t, BLOCKNUM b)
{
verify_valid_freeable_blocknum(t, b);
void block_table::_free_blocknum_in_translation(struct translation *t, BLOCKNUM b) {
_verify_valid_freeable_blocknum(t, b);
paranoid_invariant(t->block_translation[b.b].size != size_is_free);
t->block_translation[b.b].size = size_is_free;
......@@ -663,30 +642,29 @@ free_blocknum_in_translation(struct translation *t, BLOCKNUM b)
t->blocknum_freelist_head = b;
}
static void
free_blocknum_unlocked(BLOCK_TABLE bt, BLOCKNUM *bp, FT ft, bool for_checkpoint) {
// Effect: Free a blocknum.
// If the blocknum holds the only reference to a block on disk, free that block
toku_mutex_assert_locked(&bt->mutex);
void block_table::_free_blocknum_unlocked(BLOCKNUM *bp, FT ft, bool for_checkpoint) {
toku_mutex_assert_locked(&_mutex);
BLOCKNUM b = *bp;
bp->b = 0; //Remove caller's reference.
struct block_translation_pair old_pair = bt->current.block_translation[b.b];
struct block_translation_pair old_pair = _current.block_translation[b.b];
free_blocknum_in_translation(&bt->current, b);
_free_blocknum_in_translation(&_current, b);
if (for_checkpoint) {
paranoid_invariant(ft->checkpoint_header->type == FT_CHECKPOINT_INPROGRESS);
free_blocknum_in_translation(&bt->inprogress, b);
_free_blocknum_in_translation(&_inprogress, b);
}
//If the size is 0, no disk block has ever been assigned to this blocknum.
if (old_pair.size > 0) {
//Free the old block if it is not still in use by the checkpoint in progress or the previous checkpoint
bool cannot_free = (bool)
(translation_prevents_freeing(&bt->inprogress, b, &old_pair) ||
translation_prevents_freeing(&bt->checkpointed, b, &old_pair));
(_translation_prevents_freeing(&_inprogress, b, &old_pair) ||
_translation_prevents_freeing(&_checkpointed, b, &old_pair));
if (!cannot_free) {
bt->bt_block_allocator.free_block(old_pair.u.diskoff);
_bt_block_allocator.free_block(old_pair.u.diskoff);
}
}
else {
......@@ -696,91 +674,80 @@ free_blocknum_unlocked(BLOCK_TABLE bt, BLOCKNUM *bp, FT ft, bool for_checkpoint)
ft_set_dirty(ft, for_checkpoint);
}
void
toku_free_blocknum(BLOCK_TABLE bt, BLOCKNUM *bp, FT ft, bool for_checkpoint) {
lock_for_blocktable(bt);
free_blocknum_unlocked(bt, bp, ft, for_checkpoint);
unlock_for_blocktable(bt);
void block_table::free_blocknum(BLOCKNUM *bp, FT ft, bool for_checkpoint) {
_mutex_lock();
_free_blocknum_unlocked(bp, ft, for_checkpoint);
_mutex_unlock();
}
//Verify there are no free blocks.
void
toku_block_verify_no_free_blocknums(BLOCK_TABLE UU(bt)) {
paranoid_invariant(bt->current.blocknum_freelist_head.b == freelist_null.b);
// Verify there are no free blocks.
void block_table::verify_no_free_blocknums() {
invariant(_current.blocknum_freelist_head.b == freelist_null.b);
}
// Frees blocknums that have a size of 0 and unused diskoff
// Currently used for eliminating unused cached rollback log nodes
void
toku_free_unused_blocknums(BLOCK_TABLE bt, BLOCKNUM root) {
lock_for_blocktable(bt);
int64_t smallest = bt->current.smallest_never_used_blocknum.b;
void block_table::free_unused_blocknums(BLOCKNUM root) {
_mutex_lock();
int64_t smallest = _current.smallest_never_used_blocknum.b;
for (int64_t i=RESERVED_BLOCKNUMS; i < smallest; i++) {
if (i == root.b) {
continue;
}
BLOCKNUM b = make_blocknum(i);
if (bt->current.block_translation[b.b].size == 0) {
invariant(bt->current.block_translation[b.b].u.diskoff == diskoff_unused);
free_blocknum_in_translation(&bt->current, b);
if (_current.block_translation[b.b].size == 0) {
invariant(_current.block_translation[b.b].u.diskoff == diskoff_unused);
_free_blocknum_in_translation(&_current, b);
}
}
unlock_for_blocktable(bt);
_mutex_unlock();
}
__attribute__((nonnull,const,unused))
static inline bool
no_data_blocks_except_root(BLOCK_TABLE bt, BLOCKNUM root) {
bool block_table::_no_data_blocks_except_root(BLOCKNUM root) {
bool ok = true;
lock_for_blocktable(bt);
int64_t smallest = bt->current.smallest_never_used_blocknum.b;
_mutex_lock();
int64_t smallest = _current.smallest_never_used_blocknum.b;
if (root.b < RESERVED_BLOCKNUMS) {
ok = false;
goto cleanup;
}
int64_t i;
for (i=RESERVED_BLOCKNUMS; i < smallest; i++) {
for (int64_t i = RESERVED_BLOCKNUMS; i < smallest; i++) {
if (i == root.b) {
continue;
}
BLOCKNUM b = make_blocknum(i);
if (bt->current.block_translation[b.b].size != size_is_free) {
if (_current.block_translation[b.b].size != size_is_free) {
ok = false;
goto cleanup;
}
}
cleanup:
unlock_for_blocktable(bt);
_mutex_unlock();
return ok;
}
//Verify there are no data blocks except root.
// Verify there are no data blocks except root.
// TODO(leif): This actually takes a lock, but I don't want to fix all the callers right now.
void
toku_block_verify_no_data_blocks_except_root(BLOCK_TABLE UU(bt), BLOCKNUM UU(root)) {
paranoid_invariant(no_data_blocks_except_root(bt, root));
void block_table::verify_no_data_blocks_except_root(BLOCKNUM UU(root)) {
paranoid_invariant(_no_data_blocks_except_root(root));
}
__attribute__((nonnull,const,unused))
static inline bool
blocknum_allocated(BLOCK_TABLE bt, BLOCKNUM b) {
lock_for_blocktable(bt);
struct translation *t = &bt->current;
verify_valid_blocknum(t, b);
bool block_table::_blocknum_allocated(BLOCKNUM b) {
_mutex_lock();
struct translation *t = &_current;
_verify_valid_blocknum(t, b);
bool ok = t->block_translation[b.b].size != size_is_free;
unlock_for_blocktable(bt);
_mutex_unlock();
return ok;
}
//Verify a blocknum is currently allocated.
void
toku_verify_blocknum_allocated(BLOCK_TABLE UU(bt), BLOCKNUM UU(b)) {
paranoid_invariant(blocknum_allocated(bt, b));
// Verify a blocknum is currently allocated.
void block_table::verify_blocknum_allocated(BLOCKNUM UU(b)) {
paranoid_invariant(_blocknum_allocated(b));
}
//Only used by toku_dump_translation table (debug info)
static void
dump_translation(FILE *f, struct translation *t) {
// Only used by toku_dump_translation table (debug info)
void block_table::_dump_translation_internal(FILE *f, struct translation *t) {
if (t->block_translation) {
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_TRANSLATION);
fprintf(f, " length_of_array[%" PRId64 "]", t->length_of_array);
......@@ -793,230 +760,128 @@ dump_translation(FILE *f, struct translation *t) {
fprintf(f, " %" PRId64 ": %" PRId64 " %" PRId64 "\n", i, t->block_translation[i].u.diskoff, t->block_translation[i].size);
}
fprintf(f, "\n");
} else {
fprintf(f, " does not exist\n");
}
else fprintf(f, " does not exist\n");
}
//Only used by toku_ft_dump which is only for debugging purposes
// Only used by toku_ft_dump which is only for debugging purposes
// "pretty" just means we use tabs so we can parse output easier later
void
toku_dump_translation_table_pretty(FILE *f, BLOCK_TABLE bt) {
lock_for_blocktable(bt);
struct translation *t = &bt->checkpointed;
void block_table::dump_translation_table_pretty(FILE *f) {
_mutex_lock();
struct translation *t = &_checkpointed;
assert(t->block_translation != nullptr);
for (int64_t i = 0; i < t->length_of_array; ++i) {
fprintf(f, "%" PRId64 "\t%" PRId64 "\t%" PRId64 "\n", i, t->block_translation[i].u.diskoff, t->block_translation[i].size);
}
unlock_for_blocktable(bt);
_mutex_unlock();
}
//Only used by toku_ft_dump which is only for debugging purposes
void
toku_dump_translation_table(FILE *f, BLOCK_TABLE bt) {
lock_for_blocktable(bt);
// Only used by toku_ft_dump which is only for debugging purposes
void block_table::dump_translation_table(FILE *f) {
_mutex_lock();
fprintf(f, "Current block translation:");
dump_translation(f, &bt->current);
_dump_translation_internal(f, &_current);
fprintf(f, "Checkpoint in progress block translation:");
dump_translation(f, &bt->inprogress);
_dump_translation_internal(f, &_inprogress);
fprintf(f, "Checkpointed block translation:");
dump_translation(f, &bt->checkpointed);
unlock_for_blocktable(bt);
_dump_translation_internal(f, &_checkpointed);
_mutex_unlock();
}
//Only used by ftdump
void
toku_blocknum_dump_translation(BLOCK_TABLE bt, BLOCKNUM b) {
lock_for_blocktable(bt);
// Only used by ftdump
void block_table::blocknum_dump_translation(BLOCKNUM b) {
_mutex_lock();
struct translation *t = &bt->current;
struct translation *t = &_current;
if (b.b < t->length_of_array) {
struct block_translation_pair *bx = &t->block_translation[b.b];
printf("%" PRId64 ": %" PRId64 " %" PRId64 "\n", b.b, bx->u.diskoff, bx->size);
}
unlock_for_blocktable(bt);
}
//Must not call this function when anything else is using the blocktable.
//No one may use the blocktable afterwards.
void
toku_blocktable_destroy(BLOCK_TABLE *btp) {
BLOCK_TABLE bt = *btp;
*btp = NULL;
if (bt->current.block_translation) toku_free(bt->current.block_translation);
if (bt->inprogress.block_translation) toku_free(bt->inprogress.block_translation);
if (bt->checkpointed.block_translation) toku_free(bt->checkpointed.block_translation);
bt->bt_block_allocator.destroy();
blocktable_lock_destroy(bt);
nb_mutex_destroy(&bt->safe_file_size_lock);
toku_free(bt);
_mutex_unlock();
}
static BLOCK_TABLE
blocktable_create_internal (void) {
// Effect: Fill it in, including the translation table, which is uninitialized
BLOCK_TABLE XCALLOC(bt);
blocktable_lock_init(bt);
nb_mutex_init(&bt->safe_file_size_lock);
//There are two headers, so we reserve space for two.
uint64_t reserve_per_header = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE;
//Must reserve in multiples of BLOCK_ALLOCATOR_ALIGNMENT
//Round up the per-header usage if necessary.
//We want each header aligned.
uint64_t remainder = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE % block_allocator::BLOCK_ALLOCATOR_ALIGNMENT;
if (remainder != 0) {
reserve_per_header += block_allocator::BLOCK_ALLOCATOR_ALIGNMENT;
reserve_per_header -= remainder;
}
assert(2 * reserve_per_header == block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
bt->bt_block_allocator.create(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE, block_allocator::BLOCK_ALLOCATOR_ALIGNMENT);
return bt;
// Must not call this function when anything else is using the blocktable.
// No one may use the blocktable afterwards.
void block_table::destroy(void) {
// TODO: translation.destroy();
toku_free(_current.block_translation);
toku_free(_inprogress.block_translation);
toku_free(_checkpointed.block_translation);
_bt_block_allocator.destroy();
toku_mutex_destroy(&_mutex);
nb_mutex_destroy(&_safe_file_size_lock);
}
static void
translation_default(struct translation *t) { // destination into which to create a default translation
t->type = TRANSLATION_CHECKPOINTED;
t->smallest_never_used_blocknum = make_blocknum(RESERVED_BLOCKNUMS);
t->length_of_array = t->smallest_never_used_blocknum.b;
t->blocknum_freelist_head = freelist_null;
XMALLOC_N(t->length_of_array, t->block_translation);
int64_t i;
for (i = 0; i < t->length_of_array; i++) {
t->block_translation[i].size = 0;
t->block_translation[i].u.diskoff = diskoff_unused;
}
}
static int
translation_deserialize_from_buffer(struct translation *t, // destination into which to deserialize
DISKOFF location_on_disk, //Location of translation_buffer
uint64_t size_on_disk,
unsigned char * translation_buffer) { // buffer with serialized translation
int block_table::_translation_deserialize_from_buffer(struct translation *t,
DISKOFF location_on_disk,
uint64_t size_on_disk,
// out: buffer with serialized translation
unsigned char *translation_buffer) {
int r = 0;
assert(location_on_disk!=0);
assert(location_on_disk != 0);
t->type = TRANSLATION_CHECKPOINTED;
{
// check the checksum
uint32_t x1764 = toku_x1764_memory(translation_buffer, size_on_disk - 4);
uint64_t offset = size_on_disk - 4;
//printf("%s:%d read from %ld (x1764 offset=%ld) size=%ld\n", __FILE__, __LINE__, block_translation_address_on_disk, offset, block_translation_size_on_disk);
uint32_t stored_x1764 = toku_dtoh32(*(int*)(translation_buffer + offset));
if (x1764 != stored_x1764) {
fprintf(stderr, "Translation table checksum failure: calc=0x%08x read=0x%08x\n", x1764, stored_x1764);
r = TOKUDB_BAD_CHECKSUM;
goto exit;
}
// check the checksum
uint32_t x1764 = toku_x1764_memory(translation_buffer, size_on_disk - 4);
uint64_t offset = size_on_disk - 4;
uint32_t stored_x1764 = toku_dtoh32(*(int*)(translation_buffer + offset));
if (x1764 != stored_x1764) {
fprintf(stderr, "Translation table checksum failure: calc=0x%08x read=0x%08x\n", x1764, stored_x1764);
r = TOKUDB_BAD_CHECKSUM;
goto exit;
}
struct rbuf rt;
rt.buf = translation_buffer;
rt.ndone = 0;
rt.size = size_on_disk-4;//4==checksum
t->smallest_never_used_blocknum = rbuf_blocknum(&rt);
struct rbuf rb;
rb.buf = translation_buffer;
rb.ndone = 0;
rb.size = size_on_disk-4;//4==checksum
t->smallest_never_used_blocknum = rbuf_blocknum(&rb);
t->length_of_array = t->smallest_never_used_blocknum.b;
assert(t->smallest_never_used_blocknum.b >= RESERVED_BLOCKNUMS);
t->blocknum_freelist_head = rbuf_blocknum(&rt);
invariant(t->smallest_never_used_blocknum.b >= RESERVED_BLOCKNUMS);
t->blocknum_freelist_head = rbuf_blocknum(&rb);
XMALLOC_N(t->length_of_array, t->block_translation);
for (int64_t i = 0; i < t->length_of_array; i++) {
t->block_translation[i].u.diskoff = rbuf_DISKOFF(&rt);
t->block_translation[i].size = rbuf_DISKOFF(&rt);
}
assert(calculate_size_on_disk(t) == (int64_t)size_on_disk);
assert(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].size == (int64_t)size_on_disk);
assert(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff == location_on_disk);
exit:
return r;
}
// We just initialized a translation, inform block allocator to reserve space for each blocknum in use.
static void blocktable_note_translation(block_allocator *ba, struct translation *t) {
//This is where the space for them will be reserved (in addition to normal blocks).
//See RESERVED_BLOCKNUMS
// Previously this added blocks one at a time. Now we make an array and pass it in so it can be sorted and merged. See #3218.
struct block_allocator::blockpair *XMALLOC_N(t->smallest_never_used_blocknum.b, pairs);
uint64_t n_pairs = 0;
for (int64_t i=0; i<t->smallest_never_used_blocknum.b; i++) {
struct block_translation_pair pair = t->block_translation[i];
if (pair.size > 0) {
paranoid_invariant(pair.u.diskoff != diskoff_unused);
int cur_pair = n_pairs++;
pairs[cur_pair] = block_allocator::blockpair(pair.u.diskoff, pair.size);
}
}
ba->alloc_blocks_at(n_pairs, pairs);
toku_free(pairs);
}
// Fill in the checkpointed translation from buffer, and copy checkpointed to current.
// The one read from disk is the last known checkpointed one, so we are keeping it in
// place and then setting current (which is never stored on disk) for current use.
// The translation_buffer has translation only, we create the rest of the block_table.
int
toku_blocktable_create_from_buffer(int fd,
BLOCK_TABLE *btp,
DISKOFF location_on_disk, //Location of translation_buffer
DISKOFF size_on_disk,
unsigned char *translation_buffer) {
BLOCK_TABLE bt = blocktable_create_internal();
int r = translation_deserialize_from_buffer(&bt->checkpointed, location_on_disk, size_on_disk, translation_buffer);
if (r != 0) {
goto exit;
t->block_translation[i].u.diskoff = rbuf_DISKOFF(&rb);
t->block_translation[i].size = rbuf_DISKOFF(&rb);
}
blocktable_note_translation(&bt->bt_block_allocator, &bt->checkpointed);
// we just filled in checkpointed, now copy it to current.
copy_translation(&bt->current, &bt->checkpointed, TRANSLATION_CURRENT);
invariant(_calculate_size_on_disk(t) == (int64_t) size_on_disk);
invariant(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].size == (int64_t) size_on_disk);
invariant(t->block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff == location_on_disk);
int64_t file_size;
r = toku_os_get_file_size(fd, &file_size);
lazy_assert_zero(r);
invariant(file_size >= 0);
bt->safe_file_size = file_size;
*btp = bt;
exit:
return r;
}
void
toku_blocktable_create_new(BLOCK_TABLE *btp) {
BLOCK_TABLE bt = blocktable_create_internal();
translation_default(&bt->checkpointed); // create default btt (empty except for reserved blocknums)
blocktable_note_translation(&bt->bt_block_allocator, &bt->checkpointed);
// we just created a default checkpointed, now copy it to current.
copy_translation(&bt->current, &bt->checkpointed, TRANSLATION_CURRENT);
*btp = bt;
}
int
toku_blocktable_iterate (BLOCK_TABLE bt, enum translation_type type, BLOCKTABLE_CALLBACK f, void *extra, bool data_only, bool used_only) {
int block_table::iterate(enum translation_type type,
BLOCKTABLE_CALLBACK f, void *extra, bool data_only, bool used_only) {
struct translation *src;
int r = 0;
switch (type) {
case TRANSLATION_CURRENT: src = &bt->current; break;
case TRANSLATION_INPROGRESS: src = &bt->inprogress; break;
case TRANSLATION_CHECKPOINTED: src = &bt->checkpointed; break;
default: r = EINVAL; break;
case TRANSLATION_CURRENT:
src = &_current;
break;
case TRANSLATION_INPROGRESS:
src = &_inprogress;
break;
case TRANSLATION_CHECKPOINTED:
src = &_checkpointed;
break;
default:
r = EINVAL;
}
struct translation fakecurrent;
struct translation *t = &fakecurrent;
if (r==0) {
lock_for_blocktable(bt);
copy_translation(t, src, TRANSLATION_DEBUG);
if (r == 0) {
_mutex_lock();
_copy_translation(t, src, TRANSLATION_DEBUG);
t->block_translation[RESERVED_BLOCKNUM_TRANSLATION] =
src->block_translation[RESERVED_BLOCKNUM_TRANSLATION];
unlock_for_blocktable(bt);
src->block_translation[RESERVED_BLOCKNUM_TRANSLATION];
_mutex_unlock();
int64_t i;
for (i=0; i<t->smallest_never_used_blocknum.b; i++) {
struct block_translation_pair pair = t->block_translation[i];
......@@ -1035,8 +900,7 @@ typedef struct {
int64_t total_space;
} frag_extra;
static int
frag_helper(BLOCKNUM UU(b), int64_t size, int64_t address, void *extra) {
static int frag_helper(BLOCKNUM UU(b), int64_t size, int64_t address, void *extra) {
frag_extra *info = (frag_extra *) extra;
if (size + address > info->total_space)
......@@ -1045,96 +909,86 @@ frag_helper(BLOCKNUM UU(b), int64_t size, int64_t address, void *extra) {
return 0;
}
void
toku_blocktable_internal_fragmentation (BLOCK_TABLE bt, int64_t *total_sizep, int64_t *used_sizep) {
frag_extra info = {0,0};
int r = toku_blocktable_iterate(bt, TRANSLATION_CHECKPOINTED, frag_helper, &info, false, true);
void block_table::internal_fragmentation(int64_t *total_sizep, int64_t *used_sizep) {
frag_extra info = { 0, 0 };
int r = iterate(TRANSLATION_CHECKPOINTED, frag_helper, &info, false, true);
assert_zero(r);
if (total_sizep) *total_sizep = info.total_space;
if (used_sizep) *used_sizep = info.used_space;
}
void
toku_realloc_descriptor_on_disk_unlocked(BLOCK_TABLE bt, DISKOFF size, DISKOFF *offset, FT ft) {
toku_mutex_assert_locked(&bt->mutex);
void block_table::_realloc_descriptor_on_disk_unlocked(DISKOFF size, DISKOFF *offset, FT ft) {
toku_mutex_assert_locked(&_mutex);
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_DESCRIPTOR);
blocknum_realloc_on_disk_internal(bt, b, size, offset, ft, false);
_realloc_on_disk_internal(b, size, offset, ft, false);
}
void
toku_realloc_descriptor_on_disk(BLOCK_TABLE bt, DISKOFF size, DISKOFF *offset, FT ft, int fd) {
lock_for_blocktable(bt);
toku_realloc_descriptor_on_disk_unlocked(bt, size, offset, ft);
ensure_safe_write_unlocked(bt, fd, size, *offset);
unlock_for_blocktable(bt);
void block_table::realloc_descriptor_on_disk(DISKOFF size, DISKOFF *offset, FT ft, int fd) {
_mutex_lock();
_realloc_descriptor_on_disk_unlocked(size, offset, ft);
_ensure_safe_write_unlocked(fd, size, *offset);
_mutex_unlock();
}
void
toku_get_descriptor_offset_size(BLOCK_TABLE bt, DISKOFF *offset, DISKOFF *size) {
lock_for_blocktable(bt);
void block_table::get_descriptor_offset_size(DISKOFF *offset, DISKOFF *size) {
_mutex_lock();
BLOCKNUM b = make_blocknum(RESERVED_BLOCKNUM_DESCRIPTOR);
translate_blocknum_to_offset_size_unlocked(bt, b, offset, size);
unlock_for_blocktable(bt);
_translate_blocknum_to_offset_size_unlocked(b, offset, size);
_mutex_unlock();
}
void
toku_block_table_get_fragmentation_unlocked(BLOCK_TABLE bt, TOKU_DB_FRAGMENTATION report) {
//Requires: blocktable lock is held.
//Requires: report->file_size_bytes is already filled in.
void block_table::get_fragmentation_unlocked(TOKU_DB_FRAGMENTATION report) {
// Requires: blocktable lock is held.
// Requires: report->file_size_bytes is already filled in.
//Count the headers.
report->data_bytes = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE;
report->data_blocks = 1;
report->checkpoint_bytes_additional = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE;
// Count the headers.
report->data_bytes = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE;
report->data_blocks = 1;
report->checkpoint_bytes_additional = block_allocator::BLOCK_ALLOCATOR_HEADER_RESERVE;
report->checkpoint_blocks_additional = 1;
struct translation *current = &bt->current;
int64_t i;
for (i = 0; i < current->length_of_array; i++) {
struct translation *current = &_current;
for (int64_t i = 0; i < current->length_of_array; i++) {
struct block_translation_pair *pair = &current->block_translation[i];
if (pair->size > 0) {
report->data_bytes += pair->size;
report->data_blocks++;
}
}
struct translation *checkpointed = &bt->checkpointed;
for (i = 0; i < checkpointed->length_of_array; i++) {
struct block_translation_pair *pair = &checkpointed->block_translation[i];
if (pair->size > 0 &&
!(i < current->length_of_array &&
current->block_translation[i].size > 0 &&
current->block_translation[i].u.diskoff == pair->u.diskoff)
) {
struct translation *checkpointed = &_checkpointed;
for (int64_t i = 0; i < checkpointed->length_of_array; i++) {
struct block_translation_pair *pair = &_checkpointed.block_translation[i];
if (pair->size > 0 && !(i < current->length_of_array &&
current->block_translation[i].size > 0 &&
current->block_translation[i].u.diskoff == pair->u.diskoff)) {
report->checkpoint_bytes_additional += pair->size;
report->checkpoint_blocks_additional++;
}
}
struct translation *inprogress = &bt->inprogress;
for (i = 0; i < inprogress->length_of_array; i++) {
struct block_translation_pair *pair = &inprogress->block_translation[i];
if (pair->size > 0 &&
!(i < current->length_of_array &&
current->block_translation[i].size > 0 &&
current->block_translation[i].u.diskoff == pair->u.diskoff) &&
!(i < checkpointed->length_of_array &&
checkpointed->block_translation[i].size > 0 &&
checkpointed->block_translation[i].u.diskoff == pair->u.diskoff)
) {
report->checkpoint_bytes_additional += pair->size;
report->checkpoint_blocks_additional++;
struct translation *inprogress = &_inprogress;
for (int64_t i = 0; i < inprogress->length_of_array; i++) {
struct block_translation_pair *pair = &_inprogress.block_translation[i];
if (pair->size > 0 && !(i < current->length_of_array &&
current->block_translation[i].size > 0 &&
current->block_translation[i].u.diskoff == pair->u.diskoff) &&
!(i < checkpointed->length_of_array &&
checkpointed->block_translation[i].size > 0 &&
checkpointed->block_translation[i].u.diskoff == pair->u.diskoff)) {
report->checkpoint_bytes_additional += pair->size;
report->checkpoint_blocks_additional++;
}
}
bt->bt_block_allocator.get_unused_statistics(report);
_bt_block_allocator.get_unused_statistics(report);
}
void
toku_blocktable_get_info64(BLOCK_TABLE bt, struct ftinfo64 *s) {
lock_for_blocktable(bt);
void block_table::get_info64(struct ftinfo64 *s) {
_mutex_lock();
struct translation *current = &bt->current;
struct translation *current = &_current;
s->num_blocks_allocated = current->length_of_array;
s->num_blocks_in_use = 0;
s->size_allocated = 0;
......@@ -1154,31 +1008,30 @@ toku_blocktable_get_info64(BLOCK_TABLE bt, struct ftinfo64 *s) {
}
}
unlock_for_blocktable(bt);
_mutex_unlock();
}
int
toku_blocktable_iterate_translation_tables(BLOCK_TABLE bt, uint64_t checkpoint_count,
int (*iter)(uint64_t checkpoint_count,
int64_t total_num_rows,
int64_t blocknum,
int64_t diskoff,
int64_t size,
void *extra),
void *iter_extra) {
int block_table::iterate_translation_tables(uint64_t checkpoint_count,
int (*iter)(uint64_t checkpoint_count,
int64_t total_num_rows,
int64_t blocknum,
int64_t diskoff,
int64_t size,
void *extra),
void *iter_extra) {
int error = 0;
lock_for_blocktable(bt);
_mutex_lock();
int64_t total_num_rows = bt->current.length_of_array + bt->checkpointed.length_of_array;
for (int64_t i = 0; error == 0 && i < bt->current.length_of_array; ++i) {
struct block_translation_pair *block = &bt->current.block_translation[i];
int64_t total_num_rows = _current.length_of_array + _checkpointed.length_of_array;
for (int64_t i = 0; error == 0 && i < _current.length_of_array; ++i) {
struct block_translation_pair *block = &_current.block_translation[i];
error = iter(checkpoint_count, total_num_rows, i, block->u.diskoff, block->size, iter_extra);
}
for (int64_t i = 0; error == 0 && i < bt->checkpointed.length_of_array; ++i) {
struct block_translation_pair *block = &bt->checkpointed.block_translation[i];
for (int64_t i = 0; error == 0 && i < _checkpointed.length_of_array; ++i) {
struct block_translation_pair *block = &_checkpointed.block_translation[i];
error = iter(checkpoint_count - 1, total_num_rows, i, block->u.diskoff, block->size, iter_extra);
}
unlock_for_blocktable(bt);
_mutex_unlock();
return error;
}
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id$"
/*
COPYING CONDITIONS NOTICE:
......@@ -30,7 +30,7 @@ COPYING CONDITIONS NOTICE:
COPYRIGHT NOTICE:
TokuDB, Tokutek Fractal Tree Indexing Library.
Copyright (C) 2007-2013 Tokutek, Inc.
Copyright (C) 2007-2014 Tokutek, Inc.
DISCLAIMER:
......@@ -88,98 +88,25 @@ PATENT RIGHTS GRANT:
#pragma once
#ident "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved."
#ident "Copyright (c) 2007-2014 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 <db.h>
#include "portability/toku_stdint.h"
#include "portability/toku_pthread.h"
struct ft;
#include "ft/serialize/block_allocator.h"
#include "util/nb_mutex.h"
typedef struct block_table *BLOCK_TABLE;
struct ft;
typedef struct blocknum_s { int64_t b; } BLOCKNUM;
static inline BLOCKNUM make_blocknum(int64_t b) {
BLOCKNUM result = { .b = b };
return result;
}
static const BLOCKNUM ROLLBACK_NONE = { .b = 0 };
// Offset in a disk. -1 is the 'null' pointer.
typedef int64_t DISKOFF;
// Needed by tests, ftdump
struct block_translation_pair {
union { // If in the freelist, use next_free_blocknum, otherwise diskoff.
DISKOFF diskoff;
BLOCKNUM next_free_blocknum;
} u;
DISKOFF size; // set to 0xFFFFFFFFFFFFFFFF for free
};
void toku_blocktable_create_new(BLOCK_TABLE *btp);
int toku_blocktable_create_from_buffer(int fd, BLOCK_TABLE *btp, DISKOFF location_on_disk, DISKOFF size_on_disk, unsigned char *translation_buffer);
void toku_blocktable_destroy(BLOCK_TABLE *btp);
void toku_ft_lock(struct ft *ft);
void toku_ft_unlock(struct ft *ft);
void toku_block_translation_note_start_checkpoint_unlocked(BLOCK_TABLE bt);
void toku_block_translation_note_end_checkpoint(BLOCK_TABLE bt, int fd);
void toku_block_translation_note_skipped_checkpoint(BLOCK_TABLE bt);
void toku_maybe_truncate_file_on_open(BLOCK_TABLE bt, int fd);
//Blocknums
void toku_allocate_blocknum(BLOCK_TABLE bt, BLOCKNUM *res, struct ft *ft);
void toku_allocate_blocknum_unlocked(BLOCK_TABLE bt, BLOCKNUM *res, struct ft *ft);
void toku_free_blocknum(BLOCK_TABLE bt, BLOCKNUM *b, struct ft *ft, bool for_checkpoint);
void toku_verify_blocknum_allocated(BLOCK_TABLE bt, BLOCKNUM b);
void toku_block_verify_no_data_blocks_except_root(BLOCK_TABLE bt, BLOCKNUM root);
void toku_free_unused_blocknums(BLOCK_TABLE bt, BLOCKNUM root);
void toku_block_verify_no_free_blocknums(BLOCK_TABLE bt);
void toku_realloc_descriptor_on_disk(BLOCK_TABLE bt, DISKOFF size, DISKOFF *offset, struct ft *ft, int fd);
void toku_realloc_descriptor_on_disk_unlocked(BLOCK_TABLE bt, DISKOFF size, DISKOFF *offset, struct ft *ft);
void toku_get_descriptor_offset_size(BLOCK_TABLE bt, DISKOFF *offset, DISKOFF *size);
//Blocks and Blocknums
void toku_blocknum_realloc_on_disk(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF size, DISKOFF *offset, struct ft *ft, int fd, bool for_checkpoint);
void toku_translate_blocknum_to_offset_size(BLOCK_TABLE bt, BLOCKNUM b, DISKOFF *offset, DISKOFF *size);
//Serialization
void toku_serialize_translation_to_wbuf(BLOCK_TABLE bt, int fd, struct wbuf *w, int64_t *address, int64_t *size);
void toku_block_table_swap_for_redirect(BLOCK_TABLE old_bt, BLOCK_TABLE new_bt);
//DEBUG ONLY (ftdump included), tests included
void toku_blocknum_dump_translation(BLOCK_TABLE bt, BLOCKNUM b);
void toku_dump_translation_table_pretty(FILE *f, BLOCK_TABLE bt);
void toku_dump_translation_table(FILE *f, BLOCK_TABLE bt);
void toku_block_free(BLOCK_TABLE bt, uint64_t offset);
typedef int (*BLOCKTABLE_CALLBACK)(BLOCKNUM b, int64_t size, int64_t address, void *extra);
enum translation_type {
TRANSLATION_NONE = 0,
TRANSLATION_CURRENT,
TRANSLATION_INPROGRESS,
TRANSLATION_CHECKPOINTED,
TRANSLATION_DEBUG
};
int toku_blocktable_iterate(BLOCK_TABLE bt, enum translation_type type, BLOCKTABLE_CALLBACK f, void *extra, bool data_only, bool used_only);
void toku_blocktable_internal_fragmentation(BLOCK_TABLE bt, int64_t *total_sizep, int64_t *used_sizep);
void toku_block_table_get_fragmentation_unlocked(BLOCK_TABLE bt, TOKU_DB_FRAGMENTATION report);
//Requires: blocktable lock is held.
//Requires: report->file_size_bytes is already filled in.
int64_t toku_block_get_blocks_in_use_unlocked(BLOCK_TABLE bt);
void toku_blocktable_get_info64(BLOCK_TABLE, struct ftinfo64 *);
int toku_blocktable_iterate_translation_tables(BLOCK_TABLE, uint64_t, int (*)(uint64_t, int64_t, int64_t, int64_t, int64_t, void *), void *);
//Unmovable reserved first, then reallocable.
// Unmovable reserved first, then reallocable.
// We reserve one blocknum for the translation table itself.
enum {
RESERVED_BLOCKNUM_NULL = 0,
......@@ -188,6 +115,197 @@ enum {
RESERVED_BLOCKNUMS
};
typedef int (*BLOCKTABLE_CALLBACK)(BLOCKNUM b, int64_t size, int64_t address, void *extra);
static inline BLOCKNUM make_blocknum(int64_t b) {
BLOCKNUM result = { .b = b };
return result;
}
static const BLOCKNUM ROLLBACK_NONE = { .b = 0 };
/**
* There are three copies of the translation table (btt) in the block table:
*
* checkpointed Is initialized by deserializing from disk,
* and is the only version ever read from disk.
* When read from disk it is copied to current.
* It is immutable. It can be replaced by an inprogress btt.
*
* inprogress Is only filled by copying from current,
* and is the only version ever serialized to disk.
* (It is serialized to disk on checkpoint and clean shutdown.)
* At end of checkpoint it replaces 'checkpointed'.
* During a checkpoint, any 'pending' dirty writes will update
* inprogress.
*
* current Is initialized by copying from checkpointed,
* is the only version ever modified while the database is in use,
* and is the only version ever copied to inprogress.
* It is never stored on disk.
*/
class block_table {
public:
enum translation_type {
TRANSLATION_NONE = 0,
TRANSLATION_CURRENT,
TRANSLATION_INPROGRESS,
TRANSLATION_CHECKPOINTED,
TRANSLATION_DEBUG
};
void create();
int create_from_buffer(int fd, DISKOFF location_on_disk, DISKOFF size_on_disk, unsigned char *translation_buffer);
void destroy();
// Checkpointing
void note_start_checkpoint_unlocked();
void note_end_checkpoint(int fd);
void note_skipped_checkpoint();
void maybe_truncate_file_on_open(int fd);
// Blocknums
void allocate_blocknum(BLOCKNUM *res, struct ft *ft);
void realloc_on_disk(BLOCKNUM b, DISKOFF size, DISKOFF *offset, struct ft *ft, int fd, bool for_checkpoint);
void free_blocknum(BLOCKNUM *b, struct ft *ft, bool for_checkpoint);
void translate_blocknum_to_offset_size(BLOCKNUM b, DISKOFF *offset, DISKOFF *size);
void free_unused_blocknums(BLOCKNUM root);
void realloc_descriptor_on_disk(DISKOFF size, DISKOFF *offset, struct ft *ft, int fd);
void get_descriptor_offset_size(DISKOFF *offset, DISKOFF *size);
// External verfication
void verify_blocknum_allocated(BLOCKNUM b);
void verify_no_data_blocks_except_root(BLOCKNUM root);
void verify_no_free_blocknums();
// Serialization
void serialize_translation_to_wbuf(int fd, struct wbuf *w, int64_t *address, int64_t *size);
// DEBUG ONLY (ftdump included), tests included
void blocknum_dump_translation(BLOCKNUM b);
void dump_translation_table_pretty(FILE *f);
void dump_translation_table(FILE *f);
void block_free(uint64_t offset);
int iterate(enum translation_type type, BLOCKTABLE_CALLBACK f, void *extra, bool data_only, bool used_only);
void internal_fragmentation(int64_t *total_sizep, int64_t *used_sizep);
// Requires: blocktable lock is held.
// Requires: report->file_size_bytes is already filled in.
void get_fragmentation_unlocked(TOKU_DB_FRAGMENTATION report);
int64_t get_blocks_in_use_unlocked();
void get_info64(struct ftinfo64 *);
int iterate_translation_tables(uint64_t, int (*)(uint64_t, int64_t, int64_t, int64_t, int64_t, void *), void *);
private:
struct block_translation_pair {
// If in the freelist, use next_free_blocknum, otherwise diskoff.
union {
DISKOFF diskoff;
BLOCKNUM next_free_blocknum;
} u;
// Set to 0xFFFFFFFFFFFFFFFF for free
DISKOFF size;
};
// This is the BTT (block translation table)
// When the translation (btt) is stored on disk:
// In Header:
// size_on_disk
// location_on_disk
// In block translation table (in order):
// smallest_never_used_blocknum
// blocknum_freelist_head
// array
// a checksum
struct translation {
enum translation_type type;
// Number of elements in array (block_translation). always >= smallest_never_used_blocknum
int64_t length_of_array;
BLOCKNUM smallest_never_used_blocknum;
// Next (previously used) unused blocknum (free list)
BLOCKNUM blocknum_freelist_head;
struct block_translation_pair *block_translation;
// size_on_disk is stored in block_translation[RESERVED_BLOCKNUM_TRANSLATION].size
// location_on is stored in block_translation[RESERVED_BLOCKNUM_TRANSLATION].u.diskoff
};
void _create_internal();
int _translation_deserialize_from_buffer(struct translation *t, // destination into which to deserialize
DISKOFF location_on_disk, // location of translation_buffer
uint64_t size_on_disk,
unsigned char * translation_buffer); // buffer with serialized translation
void _copy_translation(struct translation *dst, struct translation *src, enum translation_type newtype);
void _maybe_optimize_translation(struct translation *t);
void _maybe_expand_translation(struct translation *t);
bool _translation_prevents_freeing(struct translation *t, BLOCKNUM b, struct block_translation_pair *old_pair);
void _free_blocknum_in_translation(struct translation *t, BLOCKNUM b);
int64_t _calculate_size_on_disk(struct translation *t);
bool _pair_is_unallocated(struct block_translation_pair *pair);
void _alloc_inprogress_translation_on_disk_unlocked();
void _dump_translation_internal(FILE *f, struct translation *t);
// Blocknum management
void _allocate_blocknum_unlocked(BLOCKNUM *res, struct ft *ft);
void _free_blocknum_unlocked(BLOCKNUM *bp, struct ft *ft, bool for_checkpoint);
void _realloc_descriptor_on_disk_unlocked(DISKOFF size, DISKOFF *offset, struct ft *ft);
void _realloc_on_disk_internal(BLOCKNUM b, DISKOFF size, DISKOFF *offset, struct ft *ft, bool for_checkpoint);
void _translate_blocknum_to_offset_size_unlocked(BLOCKNUM b, DISKOFF *offset, DISKOFF *size);
// File management
void _maybe_truncate_file(int fd, uint64_t size_needed_before);
void _ensure_safe_write_unlocked(int fd, DISKOFF block_size, DISKOFF block_offset);
// Verification
bool _is_valid_blocknum(struct translation *t, BLOCKNUM b);
void _verify_valid_blocknum(struct translation *t, BLOCKNUM b);
bool _is_valid_freeable_blocknum(struct translation *t, BLOCKNUM b);
void _verify_valid_freeable_blocknum(struct translation *t, BLOCKNUM b);
bool _no_data_blocks_except_root(BLOCKNUM root);
bool _blocknum_allocated(BLOCKNUM b);
// Locking
//
// TODO: Move the lock to the FT
void _mutex_lock();
void _mutex_unlock();
// The current translation is the one used by client threads.
// It is not represented on disk.
struct translation _current;
// The translation used by the checkpoint currently in progress.
// If the checkpoint thread allocates a block, it must also update the current translation.
struct translation _inprogress;
// The translation for the data that shall remain inviolate on disk until the next checkpoint finishes,
// after which any blocks used only in this translation can be freed.
struct translation _checkpointed;
// The in-memory data structure for block allocation.
// There is no on-disk data structure for block allocation.
// Note: This is *allocation* not *translation* - the block allocator is unaware of which
// blocks are used for which translation, but simply allocates and deallocates blocks.
block_allocator _bt_block_allocator;
toku_mutex_t _mutex;
struct nb_mutex _safe_file_size_lock;
bool _checkpoint_skipped;
uint64_t _safe_file_size;
// Because the lock is in a weird place right now
friend void toku_ft_lock(struct ft *ft);
friend void toku_ft_unlock(struct ft *ft);
};
// For serialize / deserialize
#include "ft/serialize/wbuf.h"
......
......@@ -161,12 +161,12 @@ deserialize_descriptor_from_rbuf(struct rbuf *rb, DESCRIPTOR desc, int layout_ve
}
static int
deserialize_descriptor_from(int fd, BLOCK_TABLE bt, DESCRIPTOR desc, int layout_version) {
deserialize_descriptor_from(int fd, block_table *bt, DESCRIPTOR desc, int layout_version) {
int r = 0;
DISKOFF offset;
DISKOFF size;
unsigned char *dbuf = NULL;
toku_get_descriptor_offset_size(bt, &offset, &size);
unsigned char *dbuf = nullptr;
bt->get_descriptor_offset_size(&offset, &size);
memset(desc, 0, sizeof(*desc));
if (size > 0) {
lazy_assert(size>=4); //4 for checksum
......@@ -274,11 +274,10 @@ int deserialize_ft_versioned(int fd, struct rbuf *rb, FT *ftp, uint32_t version)
assert(readsz <= (ssize_t)size_to_read);
}
// Create table and read in data.
r = toku_blocktable_create_from_buffer(fd,
&ft->blocktable,
translation_address_on_disk,
translation_size_on_disk,
tbuf);
r = ft->blocktable.create_from_buffer(fd,
translation_address_on_disk,
translation_size_on_disk,
tbuf);
toku_free(tbuf);
if (r != 0) {
goto exit;
......@@ -426,7 +425,7 @@ int deserialize_ft_versioned(int fd, struct rbuf *rb, FT *ftp, uint32_t version)
}
invariant((uint32_t) ft->layout_version_read_from_disk == version);
r = deserialize_descriptor_from(fd, ft->blocktable, &ft->descriptor, version);
r = deserialize_descriptor_from(fd, &ft->blocktable, &ft->descriptor, version);
if (r != 0) {
goto exit;
}
......@@ -804,18 +803,20 @@ void toku_serialize_ft_to_wbuf (
lazy_assert(wbuf->ndone == wbuf->size);
}
void toku_serialize_ft_to (int fd, FT_HEADER h, BLOCK_TABLE blocktable, CACHEFILE cf) {
void toku_serialize_ft_to(int fd, FT_HEADER h, block_table *bt, CACHEFILE cf) {
lazy_assert(h->type==FT_CHECKPOINT_INPROGRESS);
struct wbuf w_translation;
int64_t size_translation;
int64_t address_translation;
//Must serialize translation first, to get address,size for header.
toku_serialize_translation_to_wbuf(blocktable, fd, &w_translation,
&address_translation,
&size_translation);
assert(size_translation == w_translation.ndone); // the bytes written are the size
assert(w_translation.size % 512 == 0); // the number of bytes available in the buffer is 0 mod 512, and those last bytes are all initialized.
// Must serialize translation first, to get address,size for header.
bt->serialize_translation_to_wbuf(fd, &w_translation,
&address_translation,
&size_translation);
assert(size_translation == w_translation.ndone);
// the number of bytes available in the buffer is 0 mod 512, and those last bytes are all initialized.
assert(w_translation.size % 512 == 0);
struct wbuf w_main;
size_t size_main = toku_serialize_ft_size(h);
......
......@@ -845,8 +845,8 @@ toku_serialize_ftnode_to (int fd, BLOCKNUM blocknum, FTNODE node, FTNODE_DISK_DA
invariant(blocknum.b>=0);
DISKOFF offset;
toku_blocknum_realloc_on_disk(ft->blocktable, blocknum, n_to_write, &offset,
ft, fd, for_checkpoint); //dirties h
ft->blocktable.realloc_on_disk(blocknum, n_to_write, &offset,
ft, fd, for_checkpoint); //dirties h
tokutime_t t0 = toku_time_now();
toku_os_full_pwrite(fd, compressed_buf, n_to_write, offset);
......@@ -1085,7 +1085,7 @@ void read_block_from_fd_into_rbuf(
{
// get the file offset and block size for the block
DISKOFF offset, size;
toku_translate_blocknum_to_offset_size(ft->blocktable, blocknum, &offset, &size);
ft->blocktable.translate_blocknum_to_offset_size(blocknum, &offset, &size);
DISKOFF size_aligned = roundup_to_multiple(512, size);
uint8_t *XMALLOC_N_ALIGNED(512, size_aligned, raw_block);
rbuf_init(rb, raw_block, size);
......@@ -1101,11 +1101,12 @@ static const int read_header_heuristic_max = 32*1024;
#define MIN(a,b) (((a)>(b)) ? (b) : (a))
#endif
static void read_ftnode_header_from_fd_into_rbuf_if_small_enough (int fd, BLOCKNUM blocknum, FT ft, struct rbuf *rb, struct ftnode_fetch_extra *bfe)
// Effect: If the header part of the node is small enough, then read it into the rbuf. The rbuf will be allocated to be big enough in any case.
{
static void read_ftnode_header_from_fd_into_rbuf_if_small_enough(int fd, BLOCKNUM blocknum,
FT ft, struct rbuf *rb,
struct ftnode_fetch_extra *bfe) {
DISKOFF offset, size;
toku_translate_blocknum_to_offset_size(ft->blocktable, blocknum, &offset, &size);
ft->blocktable.translate_blocknum_to_offset_size(blocknum, &offset, &size);
DISKOFF read_size = roundup_to_multiple(512, MIN(read_header_heuristic_max, size));
uint8_t *XMALLOC_N_ALIGNED(512, roundup_to_multiple(512, size), raw_block);
rbuf_init(rb, raw_block, read_size);
......@@ -1937,10 +1938,8 @@ deserialize_and_upgrade_ftnode(FTNODE node,
// we read the different sub-sections.
// get the file offset and block size for the block
DISKOFF offset, size;
toku_translate_blocknum_to_offset_size(bfe->ft->blocktable,
blocknum,
&offset,
&size);
bfe->ft->blocktable.translate_blocknum_to_offset_size(blocknum, &offset, &size);
struct rbuf rb;
r = read_and_decompress_block_from_fd_into_rbuf(fd,
blocknum,
......@@ -2218,16 +2217,13 @@ toku_deserialize_bp_from_disk(FTNODE node, FTNODE_DISK_DATA ndd, int childnum, i
//
// get the file offset and block size for the block
DISKOFF node_offset, total_node_disk_size;
toku_translate_blocknum_to_offset_size(
bfe->ft->blocktable,
node->blocknum,
&node_offset,
&total_node_disk_size
);
bfe->ft->blocktable.translate_blocknum_to_offset_size(node->blocknum, &node_offset, &total_node_disk_size);
uint32_t curr_offset = BP_START(ndd, childnum);
uint32_t curr_size = BP_SIZE (ndd, childnum);
struct rbuf rb = {.buf = NULL, .size = 0, .ndone = 0};
uint32_t curr_size = BP_SIZE (ndd, childnum);
struct rbuf rb;
rbuf_init(&rb, nullptr, 0);
uint32_t pad_at_beginning = (node_offset+curr_offset)%512;
uint32_t padded_size = roundup_to_multiple(512, pad_at_beginning + curr_size);
......@@ -2530,20 +2526,22 @@ toku_serialize_rollback_log_to (int fd, ROLLBACK_LOG_NODE log, SERIALIZED_ROLLBA
serialized_log = &serialized_local;
toku_serialize_rollback_log_to_memory_uncompressed(log, serialized_log);
}
BLOCKNUM blocknum = serialized_log->blocknum;
invariant(blocknum.b >= 0);
//Compress and malloc buffer to write
// Compress and malloc buffer to write
serialize_uncompressed_block_to_memory(serialized_log->data,
serialized_log->n_sub_blocks, serialized_log->sub_block,
ft->h->compression_method, &n_to_write, &compressed_buf);
serialized_log->n_sub_blocks,
serialized_log->sub_block,
ft->h->compression_method,
&n_to_write, &compressed_buf);
{
lazy_assert(blocknum.b>=0);
DISKOFF offset;
toku_blocknum_realloc_on_disk(ft->blocktable, blocknum, n_to_write, &offset,
ft, fd, for_checkpoint); //dirties h
toku_os_full_pwrite(fd, compressed_buf, n_to_write, offset);
}
// Dirties the ft
DISKOFF offset;
ft->blocktable.realloc_on_disk(blocknum, n_to_write, &offset,
ft, fd, for_checkpoint);
toku_os_full_pwrite(fd, compressed_buf, n_to_write, offset);
toku_free(compressed_buf);
if (!is_serialized) {
toku_static_serialized_rollback_log_destroy(&serialized_local);
......@@ -2803,16 +2801,19 @@ cleanup:
return r;
}
// Read rollback log node from file into struct. Perform version upgrade if necessary.
int
toku_deserialize_rollback_log_from (int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE *logp, FT ft) {
// Read rollback log node from file into struct.
// Perform version upgrade if necessary.
int toku_deserialize_rollback_log_from(int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE *logp, FT ft) {
int layout_version = 0;
int r;
struct rbuf rb = {.buf = NULL, .size = 0, .ndone = 0};
struct rbuf rb;
rbuf_init(&rb, nullptr, 0);
// get the file offset and block size for the block
DISKOFF offset, size;
toku_translate_blocknum_to_offset_size(ft->blocktable, blocknum, &offset, &size);
ft->blocktable.translate_blocknum_to_offset_size(blocknum, &offset, &size);
// if the size is 0, then the blocknum is unused
if (size == 0) {
// blocknum is unused, just create an empty one and get out
......@@ -2838,7 +2839,9 @@ toku_deserialize_rollback_log_from (int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE
r = deserialize_rollback_log_from_rbuf_versioned(layout_version, blocknum, logp, &rb);
cleanup:
if (rb.buf) toku_free(rb.buf);
if (rb.buf) {
toku_free(rb.buf);
}
return r;
}
......
......@@ -90,12 +90,6 @@ PATENT RIGHTS GRANT:
#include "test.h"
static void ba_alloc_at(block_allocator *ba, uint64_t size, uint64_t offset) {
ba->validate();
ba->alloc_block_at(size * 512, offset * 512);
ba->validate();
}
static void ba_alloc(block_allocator *ba, uint64_t size, uint64_t *answer) {
ba->validate();
uint64_t actual_answer;
......@@ -133,28 +127,8 @@ static void
test_ba0 (void) {
block_allocator allocator;
block_allocator *ba = &allocator;
uint64_t b0, b1;
ba->create(100*512, 1*512);
assert(ba->allocated_limit()==100*512);
ba_alloc_at(ba, 50, 100);
assert(ba->allocated_limit()==150*512);
ba_alloc_at(ba, 25, 150);
ba_alloc (ba, 10, &b0);
ba_check_l (ba, 0, 0, 100);
ba_check_l (ba, 1, 100, 50);
ba_check_l (ba, 2, 150, 25);
ba_check_l (ba, 3, b0, 10);
ba_check_none (ba, 4);
assert(b0==175);
ba_free(ba, 150);
ba_alloc_at(ba, 10, 150);
ba_alloc(ba, 10, &b0);
assert(b0==160);
ba_alloc(ba, 10, &b0);
ba_alloc(ba, 113, &b1);
assert(113*512==ba->block_size(b1 *512));
assert(10 *512==ba->block_size(b0 *512));
assert(50 *512==ba->block_size(100*512));
uint64_t b2, b3, b4, b5, b6, b7;
ba_alloc(ba, 100, &b2);
......
......@@ -422,22 +422,22 @@ test_prefetching(void) {
16);
ft_h->cmp.create(int64_key_cmp, nullptr);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -450,8 +450,8 @@ test_prefetching(void) {
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
ft_h->cmp.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
......
......@@ -358,22 +358,22 @@ test_serialize_nonleaf(void) {
ft_h->cmp.create(string_key_cmp, nullptr);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -387,8 +387,8 @@ test_serialize_nonleaf(void) {
toku_destroy_ftnode_internals(&sn);
toku_free(ndd);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
ft_h->cmp.destroy();
toku_free(ft_h);
......@@ -438,22 +438,22 @@ test_serialize_leaf(void) {
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -466,8 +466,8 @@ test_serialize_leaf(void) {
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......
......@@ -198,22 +198,22 @@ test_serialize_leaf(int valsize, int nelts, double entropy, int ser_runs, int de
ft_h->cmp.create(long_key_cmp, nullptr);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -277,8 +277,8 @@ test_serialize_leaf(int valsize, int nelts, double entropy, int ser_runs, int de
toku_ftnode_free(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
ft_h->cmp.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
......@@ -361,22 +361,22 @@ test_serialize_nonleaf(int valsize, int nelts, double entropy, int ser_runs, int
ft_h->cmp.create(long_key_cmp, nullptr);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -412,8 +412,8 @@ test_serialize_nonleaf(int valsize, int nelts, double entropy, int ser_runs, int
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
ft_h->cmp.destroy();
toku_free(ft_h);
......
......@@ -301,23 +301,23 @@ test_serialize_leaf_check_msn(enum ftnode_verify_type bft, bool do_clone) {
TOKU_DEFAULT_COMPRESSION_METHOD,
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -373,8 +373,8 @@ test_serialize_leaf_check_msn(enum ftnode_verify_type bft, bool do_clone) {
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -435,22 +435,22 @@ test_serialize_leaf_with_large_pivots(enum ftnode_verify_type bft, bool do_clone
TOKU_DEFAULT_COMPRESSION_METHOD,
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -508,8 +508,8 @@ test_serialize_leaf_with_large_pivots(enum ftnode_verify_type bft, bool do_clone
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -561,22 +561,22 @@ test_serialize_leaf_with_many_rows(enum ftnode_verify_type bft, bool do_clone) {
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -636,8 +636,8 @@ test_serialize_leaf_with_many_rows(enum ftnode_verify_type bft, bool do_clone) {
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -696,22 +696,22 @@ test_serialize_leaf_with_large_rows(enum ftnode_verify_type bft, bool do_clone)
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -773,8 +773,8 @@ test_serialize_leaf_with_large_rows(enum ftnode_verify_type bft, bool do_clone)
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -832,22 +832,22 @@ test_serialize_leaf_with_empty_basement_nodes(enum ftnode_verify_type bft, bool
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -901,8 +901,8 @@ test_serialize_leaf_with_empty_basement_nodes(enum ftnode_verify_type bft, bool
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -952,22 +952,22 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum ftnode_verify_type b
16);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -1000,8 +1000,8 @@ test_serialize_leaf_with_multiple_empty_basement_nodes(enum ftnode_verify_type b
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
toku_free(ft);
......@@ -1075,22 +1075,22 @@ test_serialize_nonleaf(enum ftnode_verify_type bft, bool do_clone) {
ft_h->cmp.create(string_key_cmp, nullptr);
ft->ft = ft_h;
toku_blocktable_create_new(&ft_h->blocktable);
ft_h->blocktable.create();
{ int r_truncate = ftruncate(fd, 0); CKERR(r_truncate); }
//Want to use block #20
BLOCKNUM b = make_blocknum(0);
while (b.b < 20) {
toku_allocate_blocknum(ft_h->blocktable, &b, ft_h);
ft_h->blocktable.allocate_blocknum(&b, ft_h);
}
assert(b.b == 20);
{
DISKOFF offset;
DISKOFF size;
toku_blocknum_realloc_on_disk(ft_h->blocktable, b, 100, &offset, ft_h, fd, false);
ft_h->blocktable.realloc_on_disk(b, 100, &offset, ft_h, fd, false);
assert(offset==(DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_translate_blocknum_to_offset_size(ft_h->blocktable, b, &offset, &size);
ft_h->blocktable.translate_blocknum_to_offset_size(b, &offset, &size);
assert(offset == (DISKOFF)block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
assert(size == 100);
}
......@@ -1123,8 +1123,8 @@ test_serialize_nonleaf(enum ftnode_verify_type bft, bool do_clone) {
toku_ftnode_free(&dn);
toku_destroy_ftnode_internals(&sn);
toku_block_free(ft_h->blocktable, block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
toku_blocktable_destroy(&ft_h->blocktable);
ft_h->blocktable.block_free(block_allocator::BLOCK_ALLOCATOR_TOTAL_HEADER_RESERVE);
ft_h->blocktable.destroy();
ft_h->cmp.destroy();
toku_free(ft_h->h);
toku_free(ft_h);
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
#ident "$Id$"
/*
COPYING CONDITIONS NOTICE:
This program is free software; you can redistribute it and/or modify
it under the terms of version 2 of the GNU General Public License as
published by the Free Software Foundation, and provided that the
following conditions are met:
* Redistributions of source code must retain this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below).
* Redistributions in binary form must reproduce this COPYING
CONDITIONS NOTICE, the COPYRIGHT NOTICE (below), the
DISCLAIMER (below), the UNIVERSITY PATENT NOTICE (below), the
PATENT MARKING NOTICE (below), and the PATENT RIGHTS
GRANT (below) in the documentation and/or other materials
provided with the distribution.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
02110-1301, USA.
COPYRIGHT NOTICE:
TokuDB, Tokutek Fractal Tree Indexing Library.
Copyright (C) 2007-2013 Tokutek, Inc.
DISCLAIMER:
This program is distributed in the hope that it will be useful, but
WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
General Public License for more details.
UNIVERSITY PATENT NOTICE:
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.
PATENT MARKING NOTICE:
This software is covered by US Patent No. 8,185,551.
This software is covered by US Patent No. 8,489,638.
PATENT RIGHTS GRANT:
"THIS IMPLEMENTATION" means the copyrightable works distributed by
Tokutek as part of the Fractal Tree project.
"PATENT CLAIMS" means the claims of patents that are owned or
licensable by Tokutek, both currently or in the future; and that in
the absence of this license would be infringed by THIS
IMPLEMENTATION or by using or running THIS IMPLEMENTATION.
"PATENT CHALLENGE" shall mean a challenge to the validity,
patentability, enforceability and/or non-infringement of any of the
PATENT CLAIMS or otherwise opposing any of the PATENT CLAIMS.
Tokutek hereby grants to you, for the term and geographical scope of
the PATENT CLAIMS, a non-exclusive, no-charge, royalty-free,
irrevocable (except as stated in this section) patent license to
make, have made, use, offer to sell, sell, import, transfer, and
otherwise run, modify, and propagate the contents of THIS
IMPLEMENTATION, where such license applies only to the PATENT
CLAIMS. This grant does not include claims that would be infringed
only as a consequence of further modifications of THIS
IMPLEMENTATION. If you or your agent or licensee institute or order
or agree to the institution of patent litigation against any entity
(including a cross-claim or counterclaim in a lawsuit) alleging that
THIS IMPLEMENTATION constitutes direct or contributory patent
infringement, or inducement of patent infringement, then any rights
granted to you under this License shall terminate as of the date
such litigation is filed. If you or your agent or exclusive
licensee institute or order or agree to the institution of a PATENT
CHALLENGE, then Tokutek may terminate any rights granted to you
under this License.
*/
#ident "Copyright (c) 2009-2013 Tokutek Inc. All rights reserved."
#include "ft/serialize/block_allocator.h"
#include <memory.h>
#include <assert.h>
// Test the merger.
int verbose = 0;
static void
print_array (uint64_t n, const struct block_allocator::blockpair a[/*n*/]) {
printf("{");
for (uint64_t i=0; i<n; i++) printf(" %016lx", (long)a[i].offset);
printf("}\n");
}
static int
compare_blockpairs (const void *av, const void *bv) {
const struct block_allocator::blockpair *CAST_FROM_VOIDP(a, av);
const struct block_allocator::blockpair *CAST_FROM_VOIDP(b, bv);
if (a->offset < b->offset) return -1;
if (a->offset > b->offset) return +1;
return 0;
}
static void
test_merge (uint64_t an, const struct block_allocator::blockpair a[/*an*/],
uint64_t bn, const struct block_allocator::blockpair b[/*bn*/]) {
if (verbose>1) { printf("a:"); print_array(an, a); }
if (verbose>1) { printf("b:"); print_array(bn, b); }
struct block_allocator::blockpair *MALLOC_N(an+bn, q);
struct block_allocator::blockpair *MALLOC_N(an+bn, m);
if (q==0 || m==0) {
fprintf(stderr, "malloc failed, continuing\n");
goto malloc_failed;
}
for (uint64_t i=0; i<an; i++) {
q[i] = m[i] = a[i];
}
for (uint64_t i=0; i<bn; i++) {
q[an+i] = b[i];
}
if (verbose) printf("qsort\n");
qsort(q, an+bn, sizeof(*q), compare_blockpairs);
if (verbose>1) { printf("q:"); print_array(an+bn, q); }
if (verbose) printf("merge\n");
block_allocator::merge_blockpairs_into(an, m, bn, b);
if (verbose) printf("compare\n");
if (verbose>1) { printf("m:"); print_array(an+bn, m); }
for (uint64_t i=0; i<an+bn; i++) {
assert(q[i].offset == m[i].offset);
}
malloc_failed:
toku_free(q);
toku_free(m);
}
static uint64_t
compute_a (uint64_t i, int mode) {
if (mode==0) return (((uint64_t)random()) << 32) + i;
if (mode==1) return 2*i;
if (mode==2) return i;
if (mode==3) return (1LL<<50) + i;
abort();
}
static uint64_t
compute_b (uint64_t i, int mode) {
if (mode==0) return (((uint64_t)random()) << 32) + i;
if (mode==1) return 2*i+1;
if (mode==2) return (1LL<<50) + i;
if (mode==3) return i;
abort();
}
static void
test_merge_n_m (uint64_t n, uint64_t m, int mode)
{
struct block_allocator::blockpair *MALLOC_N(n, na);
struct block_allocator::blockpair *MALLOC_N(m, ma);
if (na==0 || ma==0) {
fprintf(stderr, "malloc failed, continuing\n");
goto malloc_failed;
}
if (verbose) printf("Filling a[%" PRIu64 "]\n", n);
for (uint64_t i=0; i<n; i++) {
na[i].offset = compute_a(i, mode);
}
if (verbose) printf("Filling b[%" PRIu64 "]\n", m);
for (uint64_t i=0; i<m; i++) {
if (verbose && i % (1+m/10) == 0) { printf("."); fflush(stdout); }
ma[i].offset = compute_b(i, mode);
}
qsort(na, n, sizeof(*na), compare_blockpairs);
qsort(ma, m, sizeof(*ma), compare_blockpairs);
if (verbose) fprintf(stderr, "\ntest_merge\n");
test_merge(n, na, m, ma);
malloc_failed:
toku_free(na);
toku_free(ma);
}
static void
test_big_merge (void) {
uint64_t G = 1024LL * 1024LL * 1024LL;
if (toku_os_get_phys_memory_size() < 40 * G) {
fprintf(stderr, "Skipping big merge because there is only %4.1fGiB physical memory\n", toku_os_get_phys_memory_size()/(1024.0*1024.0*1024.0));
} else {
uint64_t twoG = 2*G;
uint64_t an = twoG;
uint64_t bn = 1;
struct block_allocator::blockpair *MALLOC_N(an+bn, a);
struct block_allocator::blockpair *MALLOC_N(bn, b);
if (a == nullptr) {
fprintf(stderr, "%s:%u malloc failed, continuing\n", __FUNCTION__, __LINE__);
goto malloc_failed;
}
if (b == nullptr) {
fprintf(stderr, "%s:%u malloc failed, continuing\n", __FUNCTION__, __LINE__);
goto malloc_failed;
}
assert(a);
assert(b);
for (uint64_t i=0; i<an; i++) a[i].offset=i+1;
b[0].offset = 0;
block_allocator::merge_blockpairs_into(an, a, bn, b);
for (uint64_t i=0; i<an+bn; i++) assert(a[i].offset == i);
malloc_failed:
toku_free(a);
toku_free(b);
}
}
int main (int argc __attribute__((__unused__)), char *argv[] __attribute__((__unused__))) {
test_merge_n_m(4, 4, 0);
test_merge_n_m(16, 16, 0);
test_merge_n_m(0, 100, 0);
test_merge_n_m(100, 0, 0);
test_merge_n_m(1000000, 1000000, 0);
// Cannot run this on my laptop, or even on pointy
#if 0
uint64_t too_big = 1024LL * 1024LL * 1024LL * 2;
test_merge_n_m(too_big, too_big);
test_merge_n_m(1, too_big, 0);
#endif
test_big_merge();
return 0;
}
......@@ -126,8 +126,7 @@ toku_rollback_flush_unused_log(
{
if (write_me) {
DISKOFF offset;
toku_blocknum_realloc_on_disk(ft->blocktable, logname, 0, &offset,
ft, fd, for_checkpoint);
ft->blocktable.realloc_on_disk(logname, 0, &offset, ft, fd, for_checkpoint);
}
if (!keep_me && !is_clone) {
toku_free(log);
......
......@@ -98,7 +98,7 @@ PATENT RIGHTS GRANT:
static void rollback_unpin_remove_callback(CACHEKEY* cachekey, bool for_checkpoint, void* extra) {
FT CAST_FROM_VOIDP(ft, extra);
toku_free_blocknum(ft->blocktable, cachekey, ft, for_checkpoint);
ft->blocktable.free_blocknum(cachekey, ft, for_checkpoint);
}
void toku_rollback_log_unpin_and_remove(TOKUTXN txn, ROLLBACK_LOG_NODE log) {
......@@ -216,7 +216,7 @@ static void rollback_log_create (
CACHEFILE cf = txn->logger->rollback_cachefile;
FT CAST_FROM_VOIDP(ft, toku_cachefile_get_userdata(cf));
rollback_initialize_for_txn(log, txn, previous);
toku_allocate_blocknum(ft->blocktable, &log->blocknum, ft);
ft->blocktable.allocate_blocknum(&log->blocknum, ft);
const uint32_t hash = toku_cachetable_hash(ft->cf, log->blocknum);
*result = log;
toku_cachetable_put(cf, log->blocknum, hash,
......
......@@ -412,10 +412,8 @@ cleanup:
// Passes our check_block() function to be called as we iterate over
// the block table. This will print any interesting failures and
// update us on our progress.
static void
check_block_table(int fd, BLOCK_TABLE bt, struct ft *h)
{
int64_t num_blocks = toku_block_get_blocks_in_use_unlocked(bt);
static void check_block_table(int fd, block_table *bt, struct ft *h) {
int64_t num_blocks = bt->get_blocks_in_use_unlocked();
printf("Starting verification of checkpoint containing");
printf(" %" PRId64 " blocks.\n", num_blocks);
fflush(stdout);
......@@ -425,13 +423,11 @@ check_block_table(int fd, BLOCK_TABLE bt, struct ft *h)
.blocks_failed = 0,
.total_blocks = num_blocks,
.h = h };
int r = 0;
r = toku_blocktable_iterate(bt,
TRANSLATION_CURRENT,
check_block,
&extra,
true,
true);
int r = bt->iterate(block_table::TRANSLATION_CURRENT,
check_block,
&extra,
true,
true);
if (r != 0) {
// We can print more information here if necessary.
}
......@@ -493,11 +489,11 @@ main(int argc, char const * const argv[])
// walk over the block table and check blocks
if (h1) {
printf("Checking dictionary from header 1.\n");
check_block_table(dictfd, h1->blocktable, h1);
check_block_table(dictfd, &h1->blocktable, h1);
}
if (h2) {
printf("Checking dictionary from header 2.\n");
check_block_table(dictfd, h2->blocktable, h2);
check_block_table(dictfd, &h2->blocktable, h2);
}
if (h1 == NULL && h2 == NULL) {
printf("Both headers have a corruption and could not be used.\n");
......
......@@ -237,7 +237,7 @@ static void dump_node(int fd, BLOCKNUM blocknum, FT ft) {
assert(n!=0);
printf("ftnode\n");
DISKOFF disksize, diskoffset;
toku_translate_blocknum_to_offset_size(ft->blocktable, blocknum, &diskoffset, &disksize);
ft->blocktable.translate_blocknum_to_offset_size(blocknum, &diskoffset, &disksize);
printf(" diskoffset =%" PRId64 "\n", diskoffset);
printf(" disksize =%" PRId64 "\n", disksize);
printf(" serialize_size =%u\n", toku_serialize_ftnode_size(n));
......@@ -334,13 +334,13 @@ ok:
}
static void dump_block_translation(FT ft, uint64_t offset) {
toku_blocknum_dump_translation(ft->blocktable, make_blocknum(offset));
ft->blocktable.blocknum_dump_translation(make_blocknum(offset));
}
static void dump_fragmentation(int UU(f), FT ft, int tsv) {
int64_t used_space;
int64_t total_space;
toku_blocktable_internal_fragmentation(ft->blocktable, &total_space, &used_space);
ft->blocktable.internal_fragmentation(&total_space, &used_space);
int64_t fragsizes = total_space - used_space;
if (tsv) {
......@@ -386,8 +386,8 @@ static void dump_nodesizes(int fd, FT ft) {
memset(&info, 0, sizeof(info));
info.fd = fd;
info.ft = ft;
toku_blocktable_iterate(ft->blocktable, TRANSLATION_CHECKPOINTED,
nodesizes_helper, &info, true, true);
ft->blocktable.iterate(block_table::TRANSLATION_CHECKPOINTED,
nodesizes_helper, &info, true, true);
printf("leafblocks\t%" PRIu64 "\n", info.leafblocks);
printf("blocksizes\t%" PRIu64 "\n", info.blocksizes);
printf("leafsizes\t%" PRIu64 "\n", info.leafsizes);
......@@ -476,7 +476,7 @@ static void verify_block(unsigned char *cp, uint64_t file_offset, uint64_t size)
static void dump_block(int fd, BLOCKNUM blocknum, FT ft) {
DISKOFF offset, size;
toku_translate_blocknum_to_offset_size(ft->blocktable, blocknum, &offset, &size);
ft->blocktable.translate_blocknum_to_offset_size(blocknum, &offset, &size);
printf("%" PRId64 " at %" PRId64 " size %" PRId64 "\n", blocknum.b, offset, size);
unsigned char *CAST_FROM_VOIDP(vp, toku_malloc(size));
......@@ -688,22 +688,22 @@ int main (int argc, const char *const argv[]) {
dump_fragmentation(fd, ft, do_tsv);
}
if (do_translation_table) {
toku_dump_translation_table_pretty(stdout, ft->blocktable);
ft->blocktable.dump_translation_table_pretty(stdout);
}
if (do_garbage) {
dump_garbage_stats(fd, ft);
}
if (!do_header && !do_rootnode && !do_fragmentation && !do_translation_table && !do_garbage) {
printf("Block translation:");
toku_dump_translation_table(stdout, ft->blocktable);
ft->blocktable.dump_translation_table(stdout);
dump_header(ft);
struct __dump_node_extra info;
info.fd = fd;
info.ft = ft;
toku_blocktable_iterate(ft->blocktable, TRANSLATION_CHECKPOINTED,
dump_node_wrapper, &info, true, true);
ft->blocktable.iterate(block_table::TRANSLATION_CHECKPOINTED,
dump_node_wrapper, &info, true, true);
}
}
toku_cachefile_close(&cf, false, ZERO_LSN);
......
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