Commit 7ad1f1b9 authored by John Esmet's avatar John Esmet

FT-298 Start breaking up ft-internal.h by moving serialization code to

its own header
parent 7ffd1fab
......@@ -99,8 +99,9 @@ PATENT RIGHTS GRANT:
typedef int (*ft_compare_func)(DB *db, const DBT *a, const DBT *b);
// TODO: this should really all be encapsulated in ft/comparator.cc
int toku_builtin_compare_fun(DB *db, const DBT *a, const DBT *b) __attribute__((__visibility__("default")));
int toku_keycompare(const void *key1, uint32_t key1len, const void *key2, uint32_t key2len);
int toku_builtin_compare_fun (DB *, const DBT *, const DBT*) __attribute__((__visibility__("default")));
namespace toku {
......
......@@ -140,7 +140,7 @@ cachetable_put_empty_node_with_dep_nodes(
dependent_dirty_bits,
name,
fullhash,
toku_node_save_ct_pair);
toku_ftnode_save_ct_pair);
*result = new_node;
}
......
......@@ -95,6 +95,7 @@ PATENT RIGHTS GRANT:
#include "ft/ft-internal.h"
#include "ft/ft-flusher.h"
#include "ft/ft-flusher-internal.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
#include "portability/toku_assert.h"
#include "portability/toku_atomic.h"
......
......@@ -124,10 +124,12 @@ enum { FT_SEQINSERT_SCORE_THRESHOLD = 100 };
uint32_t compute_child_fullhash (CACHEFILE cf, FTNODE node, int childnum);
// The ft_header is not managed by the cachetable. Instead, it hangs off the cachefile as userdata.
enum ft_type {FT_CURRENT=1, FT_CHECKPOINT_INPROGRESS};
enum ft_type {
FT_CURRENT = 1,
FT_CHECKPOINT_INPROGRESS
};
// The ft_header is not managed by the cachetable. Instead, it hangs off the cachefile as userdata.
struct ft_header {
enum ft_type type;
......@@ -292,9 +294,11 @@ struct ft_handle {
struct ft_options options;
};
// TODO: Move to cachetable header
PAIR_ATTR make_ftnode_pair_attr(FTNODE node);
PAIR_ATTR make_invalid_pair_attr(void);
// Only exported for tests.
// Cachetable callbacks for ftnodes.
void toku_ftnode_clone_callback(void* value_data, void** cloned_value_data, long* clone_size, PAIR_ATTR* new_attr, bool for_checkpoint, void* write_extraargs);
void toku_ftnode_checkpoint_complete_callback(void *value_data);
......@@ -309,91 +313,11 @@ int toku_ftnode_cleaner_callback( void *ftnode_pv, BLOCKNUM blocknum, uint32_t f
CACHETABLE_WRITE_CALLBACK get_write_callbacks_for_node(FT ft);
/* serialization code */
void toku_create_compressed_partition_from_available(FTNODE node, int childnum,
enum toku_compression_method compression_method,
SUB_BLOCK sb);
int toku_serialize_ftnode_to_memory (FTNODE node,
FTNODE_DISK_DATA* ndd,
unsigned int basementnodesize,
enum toku_compression_method compression_method,
bool do_rebalancing,
bool in_parallel,
/*out*/ size_t *n_bytes_to_write,
/*out*/ size_t *n_uncompressed_bytes,
/*out*/ char **bytes_to_write);
int toku_serialize_ftnode_to(int fd, BLOCKNUM, FTNODE node, FTNODE_DISK_DATA* ndd, bool do_rebalancing, FT ft, bool for_checkpoint);
int toku_serialize_rollback_log_to (int fd, ROLLBACK_LOG_NODE log, SERIALIZED_ROLLBACK_LOG_NODE serialized_log, bool is_serialized,
FT ft, bool for_checkpoint);
void toku_serialize_rollback_log_to_memory_uncompressed(ROLLBACK_LOG_NODE log, SERIALIZED_ROLLBACK_LOG_NODE serialized);
int toku_deserialize_rollback_log_from (int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE *logp, FT ft);
int toku_deserialize_bp_from_disk(FTNODE node, FTNODE_DISK_DATA ndd, int childnum, int fd, struct ftnode_fetch_extra* bfe);
int toku_deserialize_bp_from_compressed(FTNODE node, int childnum, struct ftnode_fetch_extra *bfe);
int toku_deserialize_ftnode_from (int fd, BLOCKNUM off, uint32_t /*fullhash*/, FTNODE *ftnode, FTNODE_DISK_DATA* ndd, struct ftnode_fetch_extra* bfe);
// <CER> For verifying old, non-upgraded nodes (versions 13 and 14).
int
decompress_from_raw_block_into_rbuf(uint8_t *raw_block, size_t raw_block_size, struct rbuf *rb, BLOCKNUM blocknum);
//
//////////////// <CER> TODO: Move these function declarations
int
deserialize_ft_from_fd_into_rbuf(int fd,
toku_off_t offset_of_header,
struct rbuf *rb,
uint64_t *checkpoint_count,
LSN *checkpoint_lsn,
uint32_t * version_p);
int
deserialize_ft_versioned(int fd, struct rbuf *rb, FT *ft, uint32_t version);
void read_block_from_fd_into_rbuf(
int fd,
BLOCKNUM blocknum,
FT ft,
struct rbuf *rb
);
int
read_compressed_sub_block(struct rbuf *rb, struct sub_block *sb);
int
verify_ftnode_sub_block (struct sub_block *sb);
void
just_decompress_sub_block(struct sub_block *sb);
/* Beginning of ft-node-deserialize.c helper functions. */
void initialize_ftnode(FTNODE node, BLOCKNUM blocknum);
int read_and_check_magic(struct rbuf *rb);
int read_and_check_version(FTNODE node, struct rbuf *rb);
void read_node_info(FTNODE node, struct rbuf *rb, int version);
void allocate_and_read_partition_offsets(FTNODE node, struct rbuf *rb, FTNODE_DISK_DATA *ndd);
int check_node_info_checksum(struct rbuf *rb);
void read_legacy_node_info(FTNODE node, struct rbuf *rb, int version);
int check_legacy_end_checksum(struct rbuf *rb);
/* End of ft-node-deserialization.c helper functions. */
unsigned int toku_serialize_ftnode_size(FTNODE node); /* How much space will it take? */
void toku_verify_or_set_counts(FTNODE);
size_t toku_serialize_ft_size (FT_HEADER h);
void toku_serialize_ft_to (int fd, FT_HEADER h, struct block_table *blocktable, CACHEFILE cf);
void toku_serialize_ft_to_wbuf (
struct wbuf *wbuf,
FT_HEADER h,
DISKOFF translation_location_on_disk,
DISKOFF translation_size_on_disk
);
int toku_deserialize_ft_from (int fd, LSN max_acceptable_lsn, FT *ft);
void toku_serialize_descriptor_contents_to_fd(int fd, DESCRIPTOR desc, DISKOFF offset);
void toku_serialize_descriptor_contents_to_wbuf(struct wbuf *wb, DESCRIPTOR desc);
// This is only exported for tests.
// append a child node to a parent node
void toku_ft_nonleaf_append_child(FTNODE node, FTNODE child, const DBT *pivotkey);
// This is only exported for tests.
// append a message to a nonleaf node child buffer
void toku_ft_append_to_child_buffer(const toku::comparator &cmp, FTNODE node, int childnum, enum ft_msg_type type, MSN msn, XIDS xids, bool is_fresh, const DBT *key, const DBT *val);
......@@ -406,14 +330,7 @@ STAT64INFO_S toku_get_and_clear_basement_stats(FTNODE leafnode);
#define VERIFY_NODE(t,n) ((void)0)
#endif
void toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe);
void toku_ft_status_update_flush_reason(FTNODE node, uint64_t uncompressed_bytes_flushed, uint64_t bytes_written, tokutime_t write_time, bool for_checkpoint);
void toku_ft_status_update_serialize_times(FTNODE node, tokutime_t serialize_time, tokutime_t compress_time);
void toku_ft_status_update_deserialize_times(FTNODE node, tokutime_t deserialize_time, tokutime_t decompress_time);
void toku_ft_status_note_msn_discard(void);
void toku_ft_status_note_update(bool broadcast);
void toku_ft_status_note_msg_bytes_out(size_t buffsize);
void toku_ft_status_note_ftnode(int height, bool created); // created = false means destroyed
void toku_verify_or_set_counts(FTNODE);
//
// Helper function to fill a ftnode_fetch_extra with data
......@@ -456,6 +373,7 @@ void fill_bfe_for_prefetch(struct ftnode_fetch_extra *bfe, FT ft, struct ft_curs
void destroy_bfe_for_prefetch(struct ftnode_fetch_extra *bfe);
// TODO: consider moving this to ft/pivotkeys.cc
class pivot_bounds {
public:
pivot_bounds(const DBT &lbe_dbt, const DBT &ubi_dbt);
......@@ -477,13 +395,10 @@ private:
const DBT _upper_bound_inclusive;
};
bool
toku_bfe_wants_child_available (struct ftnode_fetch_extra* bfe, int childnum);
int
toku_bfe_leftmost_child_wanted(struct ftnode_fetch_extra *bfe, FTNODE node);
int
toku_bfe_rightmost_child_wanted(struct ftnode_fetch_extra *bfe, FTNODE node);
// TODO: move into the ftnode_fetch_extra class
bool toku_bfe_wants_child_available (struct ftnode_fetch_extra* bfe, int childnum);
int toku_bfe_leftmost_child_wanted(struct ftnode_fetch_extra *bfe, FTNODE node);
int toku_bfe_rightmost_child_wanted(struct ftnode_fetch_extra *bfe, FTNODE node);
// allocate a block number
// allocate and initialize a ftnode
......@@ -668,10 +583,20 @@ typedef struct {
TOKU_ENGINE_STATUS_ROW_S status[FT_STATUS_NUM_ROWS];
} FT_STATUS_S, *FT_STATUS;
void toku_ft_status_update_pivot_fetch_reason(struct ftnode_fetch_extra *bfe);
void toku_ft_status_update_flush_reason(FTNODE node, uint64_t uncompressed_bytes_flushed, uint64_t bytes_written, tokutime_t write_time, bool for_checkpoint);
void toku_ft_status_update_serialize_times(FTNODE node, tokutime_t serialize_time, tokutime_t compress_time);
void toku_ft_status_update_deserialize_times(FTNODE node, tokutime_t deserialize_time, tokutime_t decompress_time);
void toku_ft_status_note_msn_discard(void);
void toku_ft_status_note_update(bool broadcast);
void toku_ft_status_note_msg_bytes_out(size_t buffsize);
void toku_ft_status_note_ftnode(int height, bool created); // created = false means destroyed
void toku_ft_get_status(FT_STATUS);
void toku_flusher_thread_set_callback(void (*callback_f)(int, void*), void* extra);
// For upgrade
int toku_upgrade_subtree_estimates_to_stat64info(int fd, FT ft) __attribute__((nonnull));
int toku_upgrade_msn_from_root_to_header(int fd, FT ft) __attribute__((nonnull));
......
......@@ -90,7 +90,8 @@ PATENT RIGHTS GRANT:
#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 "ft/node.h"
#include "ft-internal.h"
#include "ft/ft-internal.h"
#include "ft/ft_node-serialize.h"
/*
* ft-node-deserialize.c -
......
......@@ -207,7 +207,9 @@ basement nodes, bulk fetch, and partial fetch:
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-flusher.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/ft_layout_version.h"
#include "ft/ft_node-serialize.h"
#include "ft/msg.h"
#include "ft/leafentry.h"
#include "ft/log-internal.h"
......
......@@ -94,6 +94,7 @@ PATENT RIGHTS GRANT:
#include "ft/compress.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/msg.h"
// not version-sensitive because we only serialize a descriptor using the current layout_version
......
/* -*- 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.
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.
*/
#pragma once
#include "ft/block_table.h"
#include "ft/ft.h"
size_t toku_serialize_ft_size(struct ft_header *h);
void toku_serialize_ft_to(int fd, struct ft_header *h, struct block_table *blocktable, CACHEFILE cf);
void toku_serialize_ft_to_wbuf(struct wbuf *wbuf, struct ft_header *h, DISKOFF translation_location_on_disk, DISKOFF translation_size_on_disk);
void toku_serialize_descriptor_contents_to_fd(int fd, DESCRIPTOR desc, DISKOFF offset);
void toku_serialize_descriptor_contents_to_wbuf(struct wbuf *wb, DESCRIPTOR desc);
int toku_deserialize_ft_from(int fd, LSN max_acceptable_lsn, FT *ft);
// TODO rename
int deserialize_ft_from_fd_into_rbuf(int fd,
toku_off_t offset_of_header,
struct rbuf *rb,
uint64_t *checkpoint_count,
LSN *checkpoint_lsn,
uint32_t *version_p);
// used by verify
// TODO rename
int deserialize_ft_versioned(int fd, struct rbuf *rb, FT *ft, uint32_t version);
......@@ -93,6 +93,7 @@ PATENT RIGHTS GRANT:
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-internal.h"
#include "ft/ft-flusher.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
#include "ft/ule.h"
......
......@@ -93,6 +93,8 @@ PATENT RIGHTS GRANT:
#include "ft/ft.h"
#include "ft/ft-cachetable-wrappers.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/ft_node-serialize.h"
#include "ft/log-internal.h"
#include "ft/log_header.h"
#include "ft/node.h"
......@@ -360,11 +362,6 @@ static void ft_note_unpin_by_checkpoint (CACHEFILE UU(cachefile), void *header_v
// End of Functions that are callbacks to the cachefile
/////////////////////////////////////////////////////////////////////////
void toku_node_save_ct_pair(CACHEKEY UU(key), void *value_data, PAIR p) {
FTNODE CAST_FROM_VOIDP(node, value_data);
node->ct_pair = p;
}
static void setup_initial_ft_root_node(FT ft, BLOCKNUM blocknum) {
FTNODE XCALLOC(node);
toku_initialize_empty_ftnode(node, blocknum, 0, 1, ft->h->layout_version, ft->h->flags);
......@@ -375,7 +372,7 @@ static void setup_initial_ft_root_node(FT ft, BLOCKNUM blocknum) {
toku_cachetable_put(ft->cf, blocknum, fullhash,
node, make_ftnode_pair_attr(node),
get_write_callbacks_for_node(ft),
toku_node_save_ct_pair);
toku_ftnode_save_ct_pair);
toku_unpin_ftnode(ft, node);
}
......
......@@ -198,7 +198,6 @@ void toku_ft_set_compression_method(FT ft, enum toku_compression_method method);
void toku_ft_get_compression_method(FT ft, enum toku_compression_method *methodp);
void toku_ft_set_fanout(FT ft, unsigned int fanout);
void toku_ft_get_fanout(FT ft, unsigned int *fanout);
void toku_node_save_ct_pair(CACHEKEY UU(key), void *value_data, PAIR p);
// mark the ft as a blackhole. any message injections will be a no op.
void toku_ft_set_blackhole(FT_HANDLE ft_handle);
......@@ -207,12 +206,14 @@ void toku_ft_set_blackhole(FT_HANDLE ft_handle);
// The difference between the two is MVCC garbage.
void toku_ft_get_garbage(FT ft, uint64_t *total_space, uint64_t *used_space);
// TODO: Should be in portability
int get_num_cores(void);
// TODO: Use the cachetable's worker pool instead of something managed by the FT...
struct toku_thread_pool *get_ft_pool(void);
void dump_bad_block(unsigned char *vp, uint64_t size);
// TODO: Should be in portability
int toku_single_process_lock(const char *lock_dir, const char *which, int *lockfd);
int toku_single_process_unlock(int *lockfd);
void tokudb_update_product_name_strings(void);
......@@ -229,6 +230,3 @@ struct toku_product_name_strings_struct {
extern struct toku_product_name_strings_struct toku_product_name_strings;
extern int tokudb_num_envs;
int toku_keycompare (const void *key1, uint32_t key1len, const void *key2, uint32_t key2len);
int toku_builtin_compare_fun (DB *, const DBT *, const DBT*) __attribute__((__visibility__("default")));
......@@ -94,6 +94,7 @@ PATENT RIGHTS GRANT:
#include "ft/compress.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
#include "ft/log-internal.h"
#include "ft/rollback.h"
......
/* -*- 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.
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.
*/
#pragma once
#include "ft/block_table.h"
#include "ft/node.h"
#include "ft/ft.h"
#include "ft/sub_block.h"
#include "ft/rbuf.h"
#include "ft/wbuf.h"
unsigned int toku_serialize_ftnode_size(FTNODE node);
int toku_serialize_ftnode_to_memory(FTNODE node, FTNODE_DISK_DATA *ndd,
unsigned int basementnodesize,
enum toku_compression_method compression_method,
bool do_rebalancing, bool in_parallel,
size_t *n_bytes_to_write, size_t *n_uncompressed_bytes,
char **bytes_to_write);
int toku_serialize_ftnode_to(int fd, BLOCKNUM, FTNODE node, FTNODE_DISK_DATA *ndd, bool do_rebalancing, FT ft, bool for_checkpoint);
int toku_serialize_rollback_log_to(int fd, ROLLBACK_LOG_NODE log, SERIALIZED_ROLLBACK_LOG_NODE serialized_log, bool is_serialized,
FT ft, bool for_checkpoint);
void toku_serialize_rollback_log_to_memory_uncompressed(ROLLBACK_LOG_NODE log, SERIALIZED_ROLLBACK_LOG_NODE serialized);
int toku_deserialize_rollback_log_from(int fd, BLOCKNUM blocknum, ROLLBACK_LOG_NODE *logp, FT ft);
int toku_deserialize_bp_from_disk(FTNODE node, FTNODE_DISK_DATA ndd, int childnum, int fd, struct ftnode_fetch_extra *bfe);
int toku_deserialize_bp_from_compressed(FTNODE node, int childnum, struct ftnode_fetch_extra *bfe);
int toku_deserialize_ftnode_from(int fd, BLOCKNUM off, uint32_t fullhash, FTNODE *node, FTNODE_DISK_DATA *ndd, struct ftnode_fetch_extra *bfe);
// used by nonleaf node partial eviction
void toku_create_compressed_partition_from_available(FTNODE node, int childnum,
enum toku_compression_method compression_method, SUB_BLOCK sb);
// <CER> For verifying old, non-upgraded nodes (versions 13 and 14).
int decompress_from_raw_block_into_rbuf(uint8_t *raw_block, size_t raw_block_size, struct rbuf *rb, BLOCKNUM blocknum);
// used by verify
int deserialize_ft_versioned(int fd, struct rbuf *rb, FT *ft, uint32_t version);
void read_block_from_fd_into_rbuf(int fd, BLOCKNUM blocknum, FT ft, struct rbuf *rb);
int read_compressed_sub_block(struct rbuf *rb, struct sub_block *sb);
int verify_ftnode_sub_block(struct sub_block *sb);
void just_decompress_sub_block(struct sub_block *sb);
// used by ft-node-deserialize.cc
void initialize_ftnode(FTNODE node, BLOCKNUM blocknum);
int read_and_check_magic(struct rbuf *rb);
int read_and_check_version(FTNODE node, struct rbuf *rb);
void read_node_info(FTNODE node, struct rbuf *rb, int version);
void allocate_and_read_partition_offsets(FTNODE node, struct rbuf *rb, FTNODE_DISK_DATA *ndd);
int check_node_info_checksum(struct rbuf *rb);
void read_legacy_node_info(FTNODE node, struct rbuf *rb, int version);
int check_legacy_end_checksum(struct rbuf *rb);
// exported so the loader can dump bad blocks
void dump_bad_block(unsigned char *vp, uint64_t size);
......@@ -100,7 +100,9 @@ PATENT RIGHTS GRANT:
#include "ft/block_allocator.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/ft_layout_version.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
#include "ft/rbuf.h"
#include "ft/sub_block.h"
......
......@@ -96,8 +96,8 @@ PATENT RIGHTS GRANT:
#include "portability/toku_assert.h"
#include "portability/memory.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft_node-serialize.h"
#include "loader/dbufio.h"
#include "loader/loader-internal.h"
......
......@@ -103,6 +103,8 @@ PATENT RIGHTS GRANT:
#include "ft/block_table.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/ft_node-serialize.h"
#include "ft/leafentry.h"
#include "ft/loader/loader-internal.h"
#include "ft/loader/pqueue.h"
......
......@@ -91,6 +91,7 @@ PATENT RIGHTS GRANT:
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
#include "ft/rbuf.h"
#include "ft/wbuf.h"
......@@ -1673,6 +1674,11 @@ int toku_ftnode_hot_next_child(FTNODE node, const DBT *k, const toku::comparator
return low;
}
void toku_ftnode_save_ct_pair(CACHEKEY UU(key), void *value_data, PAIR p) {
FTNODE CAST_FROM_VOIDP(node, value_data);
node->ct_pair = p;
}
static void
ft_nonleaf_msg_all(const toku::comparator &cmp, FTNODE node, const ft_msg &msg, bool is_fresh, size_t flow_deltas[])
// Effect: Put the message into a nonleaf node. We put it into all children, possibly causing the children to become reactive.
......
......@@ -364,6 +364,7 @@ void toku_initialize_empty_ftnode(FTNODE node, BLOCKNUM blocknum, int height, in
int layout_version, unsigned int flags);
int toku_ftnode_which_child(FTNODE node, const DBT *k, const toku::comparator &cmp);
void toku_ftnode_save_ct_pair(CACHEKEY key, void *value_data, PAIR p);
//
// Field in ftnode_fetch_extra that tells the
......
......@@ -94,6 +94,7 @@ PATENT RIGHTS GRANT:
#include "ft/block_table.h"
#include "ft/ft-internal.h"
#include "ft/ft_node-serialize.h"
#include "ft/rollback.h"
#include "ft/rollback-ct-callbacks.h"
......
......@@ -101,16 +101,18 @@ PATENT RIGHTS GRANT:
#include <string.h>
#include <portability/toku_path.h>
#include "ft/ft.h"
#include "ft/node.h"
#include "ft/block_allocator.h"
#include "ft/block_table.h"
#include "ft/log-internal.h"
#include "ft/logger.h"
#include "ft/ft-ops.h"
#include "ft/cursor.h"
#include "ft/cachetable.h"
#include "ft/cachetable-internal.h"
#include "ft/cursor.h"
#include "ft/ft.h"
#include "ft/ft-ops.h"
#include "ft/ft-serialize.h"
#include "ft/ft_node-serialize.h"
#include "ft/log-internal.h"
#include "ft/logger.h"
#include "ft/node.h"
#include "util/bytestring.h"
#define CKERR(r) ({ int __r = r; if (__r!=0) fprintf(stderr, "%s:%d error %d %s\n", __FILE__, __LINE__, __r, strerror(r)); assert(__r==0); })
......
......@@ -102,6 +102,8 @@ PATENT RIGHTS GRANT:
#include "ft/cachetable.h"
#include "ft/ft.h"
#include "ft/ft-internal.h"
#include "ft/ft-serialize.h"
#include "ft/ft_node-serialize.h"
#include "ft/node.h"
static int do_dump_data = 1;
......
......@@ -92,46 +92,36 @@ PATENT RIGHTS GRANT:
extern const char *toku_patent_string;
const char *toku_copyright_string = "Copyright (c) 2007-2013 Tokutek Inc. All rights reserved.";
#include <toku_portability.h>
#include <toku_pthread.h>
#include <toku_assert.h>
#include <db.h>
#include <ctype.h>
#include <errno.h>
#include <limits.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <fcntl.h>
#include <unistd.h>
#include <memory.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <util/status.h>
#include <util/context.h>
#include <ft/ft-flusher.h>
#include <ft/cachetable.h>
#include <ft/log.h>
#include <ft/checkpoint.h>
#include <ft/loader/loader.h>
#include <ft/log_header.h>
#include <ft/ft.h>
#include <ft/txn_manager.h>
#include "ydb.h"
#include "ydb-internal.h"
#include "ydb_cursor.h"
#include "ydb_row_lock.h"
#include "ydb_env_func.h"
#include "ydb_db.h"
#include "ydb_write.h"
#include "ydb_txn.h"
#include "loader.h"
#include "indexer.h"
#include "portability/memory.h"
#include "portability/toku_assert.h"
#include "portability/toku_portability.h"
#include "portability/toku_pthread.h"
#include "portability/toku_stdlib.h"
#include "ft/ft-flusher.h"
#include "ft/cachetable.h"
#include "ft/log.h"
#include "ft/checkpoint.h"
#include "ft/loader/loader.h"
#include "ft/log_header.h"
#include "ft/ft.h"
#include "ft/txn_manager.h"
#include "src/ydb.h"
#include "src/ydb-internal.h"
#include "src/ydb_cursor.h"
#include "src/ydb_row_lock.h"
#include "src/ydb_env_func.h"
#include "src/ydb_db.h"
#include "src/ydb_write.h"
#include "src/ydb_txn.h"
#include "src/loader.h"
#include "src/indexer.h"
#include "util/status.h"
#include "util/context.h"
// Include ydb_lib.cc here so that its constructor/destructor gets put into
// ydb.o, to make sure they don't get erased at link time (when linking to
......
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