Commit 77c7cdd7 authored by Leif Walsh's avatar Leif Walsh

implement db->get_key_after_bytes closes #18

squashed commits:

create db->get_key_after_bytes() api down to the ft layer, and start a unit test
refs #18

setup/teardown for get_key_after_bytes test
refs #18

rename test because it really is going to be a small unit test
refs #18

implement a few initial checks
refs #18

adding actually_skipped parameter because we will want it in splitVector
refs #18

NULL -> nullptr, I'm rusty from mongo work
refs #18

fix some old apis to fit what get_key_after_bytes needs:

make keyrange_compare_s hold an FT instead of FT_HANDLE to prepare for reuse refs #18

make keyrange_compare_s hold a const dbt refs #18

fix const-correctness of fill_bfe functions refs #18

add unit test for an empty dictionary
refs #18

implement get_key_after_bytes for a single basement node, passes simple tests
refs #18

add test for multiple basement nodes
refs #18

fix usage of ft_search_t, we do need a full one after all
refs #18

check actually_skipped for correctness (even though it's an estimate)
refs #18

restructure test to be faster, and test keys before the beginning of the table
refs #18

don't try to read in the right basement node, just check the bns you have
refs #18

implement get_key_after_bytes for height > 0 trees
refs #18

return the amount skipped even if we hit the end of the table
refs #18

add inexact test for height > 0 trees, adjust constants so it's faster
refs #18

don't do I/O to bring in basement nodes just for get_key_after_bytes
refs #18

don't blindly check the first basement node, it's not guaranteed to be there anymore
refs #18

fix leak in get_key_after_bytes_unit.tdb
refs #18

add get_key_after_bytes to test_stress5
refs #18

switch to a callback API to save mallocs and memcpys
refs #18
parent 61c35212
......@@ -508,6 +508,7 @@ static void print_db_struct (void) {
STRUCT_SETUP(DB, verify, "int (*%s) (DB *, const char *, const char *, FILE *, uint32_t)");
const char *extra[]={
"int (*key_range64)(DB*, DB_TXN *, DBT *, uint64_t *less, uint64_t *equal, uint64_t *greater, int *is_exact)",
"int (*get_key_after_bytes)(DB *, DB_TXN *, const DBT *, uint64_t, void (*callback)(const DBT *, uint64_t, void *), void *, uint32_t); /* given start_key and skip_len, find largest end_key such that the elements in [start_key,end_key) sum to <= skip_len bytes */",
"int (*keys_range64)(DB*, DB_TXN *, DBT *keyleft, DBT *keyright, uint64_t *less, uint64_t *left, uint64_t *between, uint64_t *right, uint64_t *greater, bool *middle_3_exact)",
"int (*stat64)(DB *, DB_TXN *, DB_BTREE_STAT64 *)",
"int (*pre_acquire_table_lock)(DB*, DB_TXN*)",
......
......@@ -840,8 +840,8 @@ static inline void fill_bfe_for_full_read(struct ftnode_fetch_extra *bfe, FT h)
static inline void fill_bfe_for_keymatch(
struct ftnode_fetch_extra *bfe,
FT h,
DBT *left,
DBT *right,
const DBT *left,
const DBT *right,
bool disable_prefetching,
bool read_all_partitions
)
......@@ -880,8 +880,8 @@ static inline void fill_bfe_for_subset_read(
struct ftnode_fetch_extra *bfe,
FT h,
ft_search_t* search,
DBT *left,
DBT *right,
const DBT *left,
const DBT *right,
bool left_is_neg_infty,
bool right_is_pos_infty,
bool disable_prefetching,
......
......@@ -5654,8 +5654,8 @@ toku_ft_cursor_delete(FT_CURSOR cursor, int flags, TOKUTXN txn) {
struct keyrange_compare_s {
FT_HANDLE ft_handle;
DBT *key;
FT ft;
const DBT *key;
};
static int
......@@ -5667,8 +5667,8 @@ keyrange_compare (OMTVALUE lev, void *extra) {
toku_fill_dbt(&omt_dbt, key, keylen);
struct keyrange_compare_s *CAST_FROM_VOIDP(s, extra);
// TODO: maybe put a const fake_db in the header
FAKE_DB(db, &s->ft_handle->ft->cmp_descriptor);
return s->ft_handle->ft->compare_fun(&db, &omt_dbt, s->key);
FAKE_DB(db, &s->ft->cmp_descriptor);
return s->ft->compare_fun(&db, &omt_dbt, s->key);
}
static void
......@@ -5689,7 +5689,7 @@ keysrange_in_leaf_partition (FT_HANDLE brt, FTNODE node,
if (BP_STATE(node, left_child_number) == PT_AVAIL) {
int r;
// The partition is in main memory then get an exact count.
struct keyrange_compare_s s_left = {brt, key_left};
struct keyrange_compare_s s_left = {brt->ft, key_left};
BASEMENTNODE bn = BLB(node, left_child_number);
OMTVALUE datav;
uint32_t idx_left = 0;
......@@ -5702,7 +5702,7 @@ keysrange_in_leaf_partition (FT_HANDLE brt, FTNODE node,
uint32_t idx_right = size;
r = -1;
if (single_basement && key_right) {
struct keyrange_compare_s s_right = {brt, key_right};
struct keyrange_compare_s s_right = {brt->ft, key_right};
r = toku_omt_find_zero(bn->buffer, keyrange_compare, &s_right, &datav, &idx_right);
}
*middle = idx_right - idx_left - *equal_left;
......@@ -5928,6 +5928,167 @@ try_again:
}
}
struct get_key_after_bytes_iterate_extra {
uint64_t skip_len;
uint64_t *skipped;
void (*callback)(const DBT *, uint64_t, void *);
void *cb_extra;
};
static int get_key_after_bytes_iterate(OMTVALUE lev, uint32_t UU(idx), void *extra) {
struct get_key_after_bytes_iterate_extra *CAST_FROM_VOIDP(e, extra);
LEAFENTRY CAST_FROM_VOIDP(le, lev);
uint32_t keylen;
void *key = le_key_and_len(le, &keylen);
// only checking the latest val, mvcc will make this inaccurate
uint64_t pairlen = keylen + le_latest_vallen(le);
if (*e->skipped + pairlen > e->skip_len) {
// found our key!
DBT end_key;
toku_fill_dbt(&end_key, key, keylen);
e->callback(&end_key, *e->skipped, e->cb_extra);
return 1;
} else {
*e->skipped += pairlen;
return 0;
}
}
static int get_key_after_bytes_in_basementnode(FT ft, BASEMENTNODE bn, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *, uint64_t, void *), void *cb_extra, uint64_t *skipped) {
int r;
uint32_t idx_left = 0;
if (start_key != nullptr) {
struct keyrange_compare_s cmp = {ft, start_key};
OMTVALUE v;
r = toku_omt_find_zero(bn->buffer, keyrange_compare, &cmp, &v, &idx_left);
assert(r == 0 || r == DB_NOTFOUND);
}
struct get_key_after_bytes_iterate_extra iter_extra = {skip_len, skipped, callback, cb_extra};
r = toku_omt_iterate_on_range(bn->buffer, idx_left, toku_omt_size(bn->buffer), get_key_after_bytes_iterate, &iter_extra);
// Invert the sense of r == 0 (meaning the iterate finished, which means we didn't find what we wanted)
if (r == 1) {
r = 0;
} else {
r = DB_NOTFOUND;
}
return r;
}
static int get_key_after_bytes_in_subtree(FT_HANDLE ft_h, FT ft, FTNODE node, UNLOCKERS unlockers, ANCESTORS ancestors, PIVOT_BOUNDS bounds, FTNODE_FETCH_EXTRA bfe, ft_search_t *search, uint64_t subtree_bytes, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *, uint64_t, void *), void *cb_extra, uint64_t *skipped);
static int get_key_after_bytes_in_child(FT_HANDLE ft_h, FT ft, FTNODE node, UNLOCKERS unlockers, ANCESTORS ancestors, PIVOT_BOUNDS bounds, FTNODE_FETCH_EXTRA bfe, ft_search_t *search, int childnum, uint64_t subtree_bytes, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *, uint64_t, void *), void *cb_extra, uint64_t *skipped) {
int r;
struct ancestors next_ancestors = {node, childnum, ancestors};
BLOCKNUM childblocknum = BP_BLOCKNUM(node, childnum);
uint32_t fullhash = compute_child_fullhash(ft->cf, node, childnum);
FTNODE child;
bool msgs_applied = false;
r = toku_pin_ftnode_batched(ft_h, childblocknum, fullhash, unlockers, &next_ancestors, bounds, bfe, PL_READ, false, &child, &msgs_applied);
paranoid_invariant(!msgs_applied);
if (r == TOKUDB_TRY_AGAIN) {
return r;
}
assert_zero(r);
struct unlock_ftnode_extra unlock_extra = {ft_h, child, false};
struct unlockers next_unlockers = {true, unlock_ftnode_fun, (void *) &unlock_extra, unlockers};
const struct pivot_bounds next_bounds = next_pivot_keys(node, childnum, bounds);
return get_key_after_bytes_in_subtree(ft_h, ft, child, &next_unlockers, &next_ancestors, &next_bounds, bfe, search, subtree_bytes, start_key, skip_len, callback, cb_extra, skipped);
}
static int get_key_after_bytes_in_subtree(FT_HANDLE ft_h, FT ft, FTNODE node, UNLOCKERS unlockers, ANCESTORS ancestors, PIVOT_BOUNDS bounds, FTNODE_FETCH_EXTRA bfe, ft_search_t *search, uint64_t subtree_bytes, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *, uint64_t, void *), void *cb_extra, uint64_t *skipped) {
int r;
int childnum = toku_ft_search_which_child(&ft->cmp_descriptor, ft->compare_fun, node, search);
const uint64_t child_subtree_bytes = subtree_bytes / node->n_children;
if (node->height == 0) {
r = DB_NOTFOUND;
for (int i = childnum; r == DB_NOTFOUND && i < node->n_children; ++i) {
// The theory here is that a leaf node could only be very
// unbalanced if it's dirty, which means all its basements are
// available. So if a basement node is available, we should
// check it as carefully as possible, but if it's compressed
// or on disk, then it should be fairly well balanced so we
// can trust the fanout calculation.
if (BP_STATE(node, i) == PT_AVAIL) {
r = get_key_after_bytes_in_basementnode(ft, BLB(node, i), (i == childnum) ? start_key : nullptr, skip_len, callback, cb_extra, skipped);
} else {
*skipped += child_subtree_bytes;
if (*skipped >= skip_len && i < node->n_children - 1) {
callback(&node->childkeys[i], *skipped, cb_extra);
r = 0;
}
// Otherwise, r is still DB_NOTFOUND. If this is the last
// basement node, we'll return DB_NOTFOUND and that's ok.
// Some ancestor in the call stack will check the next
// node over and that will call the callback, or if no
// such node exists, we're at the max key and we should
// return DB_NOTFOUND up to the top.
}
}
} else {
r = get_key_after_bytes_in_child(ft_h, ft, node, unlockers, ancestors, bounds, bfe, search, childnum, child_subtree_bytes, start_key, skip_len, callback, cb_extra, skipped);
for (int i = childnum + 1; r == DB_NOTFOUND && i < node->n_children; ++i) {
if (*skipped + child_subtree_bytes < skip_len) {
*skipped += child_subtree_bytes;
} else {
r = get_key_after_bytes_in_child(ft_h, ft, node, unlockers, ancestors, bounds, bfe, search, i, child_subtree_bytes, nullptr, skip_len, callback, cb_extra, skipped);
}
}
}
if (r != TOKUDB_TRY_AGAIN) {
assert(unlockers->locked);
toku_unpin_ftnode_read_only(ft, node);
unlockers->locked = false;
}
return r;
}
int toku_ft_get_key_after_bytes(FT_HANDLE ft_h, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *end_key, uint64_t actually_skipped, void *extra), void *cb_extra)
// Effect:
// Call callback with end_key set to the largest key such that the sum of the sizes of the key/val pairs in the range [start_key, end_key) is <= skip_len.
// Call callback with actually_skipped set to the sum of the sizes of the key/val pairs in the range [start_key, end_key).
// Notes:
// start_key == nullptr is interpreted as negative infinity.
// end_key == nullptr is interpreted as positive infinity.
// Only the latest val is counted toward the size, in the case of MVCC data.
// Implementation:
// This is an estimated calculation. We assume for a node that each of its subtrees have equal size. If the tree is a single basement node, then we will be accurate, but otherwise we could be quite off.
// Returns:
// 0 on success
// an error code otherwise
{
FT ft = ft_h->ft;
struct ftnode_fetch_extra bfe;
fill_bfe_for_min_read(&bfe, ft);
while (true) {
FTNODE root;
{
uint32_t fullhash;
CACHEKEY root_key;
toku_calculate_root_offset_pointer(ft, &root_key, &fullhash);
toku_pin_ftnode_off_client_thread_batched(ft, root_key, fullhash, &bfe, PL_READ, 0, nullptr, &root);
}
struct unlock_ftnode_extra unlock_extra = {ft_h, root, false};
struct unlockers unlockers = {true, unlock_ftnode_fun, (void*)&unlock_extra, (UNLOCKERS) nullptr};
ft_search_t search;
ft_search_init(&search, (start_key == nullptr ? ft_cursor_compare_one : ft_cursor_compare_set_range), FT_SEARCH_LEFT, start_key, ft_h);
int r;
paranoid_invariant(ft->in_memory_stats.numbytes >= 0);
uint64_t numbytes = (uint64_t) ft->in_memory_stats.numbytes;
uint64_t skipped = 0;
r = get_key_after_bytes_in_subtree(ft_h, ft, root, &unlockers, nullptr, &infinite_bounds, &bfe, &search, numbytes, start_key, skip_len, callback, cb_extra, &skipped);
assert(!unlockers.locked);
if (r != TOKUDB_TRY_AGAIN) {
if (r == DB_NOTFOUND) {
callback(nullptr, skipped, cb_extra);
r = 0;
}
return r;
}
}
}
//Test-only wrapper for the old one-key range function
void toku_ft_keyrange(FT_HANDLE brt, DBT *key, uint64_t *less, uint64_t *equal, uint64_t *greater) {
uint64_t zero_equal_right, zero_greater;
......
......@@ -291,6 +291,8 @@ enum ft_flags {
void toku_ft_keyrange(FT_HANDLE brt, DBT *key, uint64_t *less, uint64_t *equal, uint64_t *greater);
void toku_ft_keysrange(FT_HANDLE brt, DBT* key_left, DBT* key_right, uint64_t *less_p, uint64_t* equal_left_p, uint64_t* middle_p, uint64_t* equal_right_p, uint64_t* greater_p, bool* middle_3_exact_p);
int toku_ft_get_key_after_bytes(FT_HANDLE ft_h, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *end_key, uint64_t actually_skipped, void *extra), void *cb_extra);
struct ftstat64_s {
uint64_t nkeys; /* estimate how many unique keys (even when flattened this may be an estimate) */
uint64_t ndata; /* estimate the number of pairs (exact when flattened and committed) */
......
......@@ -69,6 +69,7 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS)
env_startup
execute-updates
filesize
get_key_after_bytes_unit
helgrind1
helgrind2
helgrind3
......
/* -*- mode: C++; c-basic-offset: 4; indent-tabs-mode: nil -*- */
// vim: ft=cpp:expandtab:ts=8:sw=4:softtabstop=4:
/*
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.
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) 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 "test.h"
#include <db.h>
#include <algorithm>
// Unit test for db->get_key_after_bytes.
static const int num_keys = 1<<10;
static void setup(DB_ENV **envp, DB **dbp, uint32_t nodesize, uint32_t basementnodesize) {
int r;
toku_os_recursive_delete(TOKU_TEST_FILENAME);
r = toku_os_mkdir(TOKU_TEST_FILENAME, S_IRWXU|S_IRWXG|S_IRWXO);
CKERR(r);
r = db_env_create(envp, 0);
CKERR(r);
DB_ENV *env = *envp;
r = env->set_default_bt_compare(env, int_dbt_cmp);
CKERR(r);
env->set_errfile(env, stderr);
r = env->open(env, TOKU_TEST_FILENAME, DB_INIT_LOCK|DB_INIT_LOG|DB_INIT_MPOOL|DB_INIT_TXN|DB_CREATE|DB_PRIVATE, S_IRWXU|S_IRWXG|S_IRWXO);
CKERR(r);
r = db_create(dbp, env, 0);
CKERR(r);
DB *db = *dbp;
{
r = db->set_pagesize(db, nodesize);
CKERR(r);
r = db->set_readpagesize(db, basementnodesize);
CKERR(r);
DB_TXN *txn;
r = env->txn_begin(env, 0, &txn, 0);
CKERR(r);
r = db->open(db, txn, "foo.db", 0, DB_BTREE, DB_CREATE, S_IRWXU|S_IRWXG|S_IRWXO);
CKERR(r);
r = txn->commit(txn, 0);
CKERR(r);
}
}
static void fill(DB_ENV *env, DB *db) {
int r;
DB_TXN *txn;
r = env->txn_begin(env, 0, &txn, 0);
CKERR(r);
int k, v;
DBT key, val;
dbt_init(&key, &k, sizeof k);
dbt_init(&val, &v, sizeof v);
for (int i = 0; i < num_keys; ++i) {
k = i;
v = i;
r = db->put(db, txn, &key, &val, 0);
CKERR(r);
}
r = txn->commit(txn, 0);
CKERR(r);
}
struct check_extra {
int start_key;
uint64_t skip_len;
bool filled;
bool exact;
};
static void check_callback(const DBT *end_key, uint64_t actually_skipped, void *extra) {
struct check_extra *CAST_FROM_VOIDP(e, extra);
int real_start_key = std::min(std::max(e->start_key, 0), num_keys);
int expected_key = std::min(real_start_key + (e->skip_len / (2 * sizeof(int))), (uint64_t) num_keys);
if (e->exact) {
if (!e->filled || expected_key >= num_keys) {
expected_key = -1;
}
assert(actually_skipped <= e->skip_len);
if (expected_key == -1) {
assert(end_key == nullptr);
} else {
assert(e->skip_len - actually_skipped < 2 * (int) sizeof(int));
assert(end_key != nullptr);
assert(end_key->size == sizeof expected_key);
assert((*(int *) end_key->data) == expected_key);
}
} else {
// no sense in doing an inexact check if the table's empty
assert(e->filled);
int found;
if (end_key == nullptr) {
found = num_keys;
} else {
assert(end_key->size == sizeof found);
found = *(int *) end_key->data;
}
// These are just guesses. I don't have a good reason but they
// seem like alright bounds.
double skipped_portion = (double) e->skip_len / (num_keys * 2 * sizeof(int));
int key_slack = num_keys * std::max(std::min(skipped_portion, 0.25), 0.01);
int size_slack = key_slack * 2 * sizeof(int);
assert(found <= expected_key + key_slack);
assert(found >= expected_key - key_slack);
assert(actually_skipped <= e->skip_len + size_slack);
if (end_key != nullptr) {
// if we hit the end of the table, this definitely won't hold up
assert((int) actually_skipped >= (int) e->skip_len - size_slack);
}
}
}
static void check(DB_ENV *env, DB *db, int start_key, uint64_t skip_len, bool filled, bool exact) {
int r;
DB_TXN *txn;
r = env->txn_begin(env, 0, &txn, 0);
CKERR(r);
DBT start_dbt, end_key;
dbt_init(&start_dbt, &start_key, sizeof start_key);
dbt_init(&end_key, nullptr, 0);
struct check_extra extra = {start_key, skip_len, filled, exact};
r = db->get_key_after_bytes(db, txn, (start_key == -2 ? nullptr : &start_dbt), skip_len, check_callback, &extra, 0);
CKERR(r);
r = txn->commit(txn, 0);
CKERR(r);
}
static void teardown(DB_ENV *env, DB *db) {
int r;
r = db->close(db, 0);
CKERR(r);
r = env->close(env, 0);
CKERR(r);
}
int test_main(int argc, char * const argv[]) {
int r;
default_parse_args(argc, argv);
DB_ENV *env;
DB *db;
setup(&env, &db, 4<<20, 64<<10);
// if the table is empty, always say DB_NOTFOUND
for (int start_key = -2; start_key <= 1; ++start_key) {
for (int skip_len = 0; skip_len < 2; ++skip_len) {
check(env, db, start_key, skip_len, false, true);
}
}
fill(env, db);
// if start_key is bigger than any key, assert that we get DB_NOTFOUND
for (int extra_key = 0; extra_key < 10; extra_key += 5) {
for (int skip_len = 0; skip_len < 24; ++skip_len) {
check(env, db, num_keys + extra_key, skip_len, true, true);
}
}
// if start_key is nullptr or the first key or before the first key, we start at the beginning
for (int start_key = -2; start_key <= 0; ++start_key) {
for (int skip_len = 0; skip_len < 48; ++skip_len) {
check(env, db, start_key, skip_len, true, true);
}
}
// check a bunch of places in the middle too (use prime increments to get a good distribution of stuff)
for (int start_key = 0; start_key <= num_keys; start_key += 31) {
for (int skip_len = 0; skip_len < (num_keys + 1 - start_key) * (2 * (int) sizeof(int)); skip_len += 67) {
check(env, db, start_key, skip_len, true, true);
}
}
// TODO: test mvcc stuff (check that we only look at the latest val, which is the current behavior)
teardown(env, db);
// Try many bn and nodesizes
for (int basementnodesize = 1<<10; basementnodesize <= 64<<10; basementnodesize <<= 1) {
for (int nodesize = basementnodesize; nodesize <= 128<<10; nodesize <<= 2) {
setup(&env, &db, nodesize, basementnodesize);
fill(env, db);
// forces a rebalance of the root, to get multiple bns
r = env->txn_checkpoint(env, 0, 0, 0);
CKERR(r);
// near the beginning
for (int start_key = -2; start_key <= 1; ++start_key) {
for (int skip_len = 0; skip_len <= (num_keys + 1 - start_key) * (2 * (int) sizeof(int)); skip_len += 41) {
check(env, db, start_key, skip_len, true, false);
}
}
// near the end
for (int start_key = num_keys - 1; start_key <= num_keys + 1; ++start_key) {
for (int skip_len = 0; skip_len <= (num_keys + 1 - start_key) * (2 * (int) sizeof(int)); skip_len += 41) {
check(env, db, start_key, skip_len, true, false);
}
}
for (int start_key = 0; start_key <= num_keys; start_key += 17) {
for (int skip_len = 0; skip_len <= (num_keys + 1 - start_key) * (2 * (int) sizeof(int)); skip_len += 31) {
check(env, db, start_key, skip_len, true, false);
}
}
}
}
return 0;
}
......@@ -132,16 +132,17 @@ stress_table(DB_ENV *env, DB **dbp, struct cli_args *cli_args) {
// make the guy that updates the db
myargs[2].operation = loader_op;
myargs[3].operation = keyrange_op;
myargs[4].operation = get_key_after_bytes_op;
struct update_op_args uoe = get_update_op_args(cli_args, NULL);
// make the guy that updates the db
for (int i = 4; i < 4 + cli_args->num_update_threads; ++i) {
for (int i = 5; i < 5 + cli_args->num_update_threads; ++i) {
myargs[i].operation_extra = &uoe;
myargs[i].operation = update_op;
}
// make the guy that does point queries
for (int i = 4 + cli_args->num_update_threads; i < num_threads; i++) {
for (int i = 5 + cli_args->num_update_threads; i < num_threads; i++) {
myargs[i].operation = ptquery_op;
}
run_workers(myargs, num_threads, cli_args->num_seconds, false, cli_args);
......
......@@ -974,6 +974,28 @@ static int UU() keyrange_op(DB_TXN *txn, ARG arg, void* UU(operation_extra), voi
return r;
}
static void UU() get_key_after_bytes_callback(const DBT *UU(end_key), uint64_t UU(skipped), void *UU(extra)) {
// nothing
}
static int UU() get_key_after_bytes_op(DB_TXN *txn, ARG arg, void* UU(operation_extra), void *UU(stats_extra)) {
// Pick a random DB, do a get_key_after_bytes operation.
int db_index = myrandom_r(arg->random_data)%arg->cli->num_DBs;
DB* db = arg->dbp[db_index];
int r = 0;
uint8_t keybuf[arg->cli->key_size];
DBT start_key, end_key;
dbt_init(&start_key, keybuf, sizeof keybuf);
fill_key_buf_random(arg->random_data, keybuf, arg);
uint64_t skip_len = myrandom_r(arg->random_data) % (2<<30);
dbt_init(&end_key, nullptr, 0);
r = db->get_key_after_bytes(db, txn, &start_key, skip_len, get_key_after_bytes_callback, nullptr, 0);
return r;
}
static int verify_progress_callback(void *UU(extra), float UU(progress)) {
if (!run_test) {
return -1;
......
......@@ -746,6 +746,12 @@ toku_db_key_range64(DB* db, DB_TXN* txn, DBT* key, uint64_t* less_p, uint64_t* e
return 0;
}
static int toku_db_get_key_after_bytes(DB *db, DB_TXN *txn, const DBT *start_key, uint64_t skip_len, void (*callback)(const DBT *end_key, uint64_t actually_skipped, void *extra), void *cb_extra, uint32_t UU(flags)) {
HANDLE_PANICKED_DB(db);
HANDLE_DB_ILLEGAL_WORKING_PARENT_TXN(db, txn);
return toku_ft_get_key_after_bytes(db->i->ft_handle, start_key, skip_len, callback, cb_extra);
}
// needed by loader.c
int
toku_db_pre_acquire_table_lock(DB *db, DB_TXN *txn) {
......@@ -1019,6 +1025,7 @@ toku_db_create(DB ** db, DB_ENV * env, uint32_t flags) {
USDB(pre_acquire_fileops_lock);
USDB(key_range64);
USDB(keys_range64);
USDB(get_key_after_bytes);
USDB(hot_optimize);
USDB(stat64);
USDB(get_fractal_tree_info64);
......
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