diff --git a/cmake_modules/TokuSetupCompiler.cmake b/cmake_modules/TokuSetupCompiler.cmake index 1d59ab0f8..b18be8252 100644 --- a/cmake_modules/TokuSetupCompiler.cmake +++ b/cmake_modules/TokuSetupCompiler.cmake @@ -108,6 +108,8 @@ set_cflags_if_supported( -fno-rtti -fno-exceptions -Wno-error=nonnull-compare + -Wno-missing-braces + -Wno-address-of-packed-member ) ## set_cflags_if_supported_named("-Weffc++" -Weffcpp) diff --git a/ft/cachetable/cachetable-internal.h b/ft/cachetable/cachetable-internal.h index 05fb771de..6676b678f 100644 --- a/ft/cachetable/cachetable-internal.h +++ b/ft/cachetable/cachetable-internal.h @@ -44,6 +44,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include +#include ////////////////////////////////////////////////////////////////////////////// // @@ -498,12 +499,12 @@ class evictor { pair_list* m_pl; cachefile_list* m_cf_list; - int64_t m_size_current; // the sum of the sizes of the pairs in the cachetable + std::atomic_int64_t m_size_current; // the sum of the sizes of the pairs in the cachetable int64_t m_size_cloned_data; // stores amount of cloned data we have, only used for engine status // changes to these two values are protected // by ev_thread_lock int64_t m_size_reserved; // How much memory is reserved (e.g., by the loader) - int64_t m_size_evicting; // the sum of the sizes of the pairs being written + std::atomic_int64_t m_size_evicting; // the sum of the sizes of the pairs being written // these are constants int64_t m_low_size_watermark; // target max size of cachetable that eviction thread aims for @@ -531,7 +532,7 @@ class evictor { // in init, set to false during destroy bool m_run_thread; // bool that states if the eviction thread is currently running - bool m_ev_thread_is_running; + std::atomic_bool m_ev_thread_is_running; // period which the eviction thread sleeps uint32_t m_period_in_seconds; // condition variable on which client threads wait on when sleeping diff --git a/ft/cachetable/cachetable.cc b/ft/cachetable/cachetable.cc index ae2bb2846..f988a5454 100644 --- a/ft/cachetable/cachetable.cc +++ b/ft/cachetable/cachetable.cc @@ -615,7 +615,7 @@ static void cachetable_free_pair(PAIR p) { void *write_extraargs = p->write_extraargs; PAIR_ATTR old_attr = p->attr; - cachetable_evictions++; + toku_unsafe_inc(&cachetable_evictions); PAIR_ATTR new_attr = p->attr; // Note that flush_callback is called with write_me false, so the only purpose of this // call is to tell the ft layer to evict the node (keep_me is false). @@ -1715,8 +1715,8 @@ int toku_cachetable_get_and_pin_with_dep_pairs ( // The pair being fetched will be marked as pending if a checkpoint happens during the // fetch because begin_checkpoint will mark as pending any pair that is locked even if it is clean. cachetable_fetch_pair(ct, cachefile, p, fetch_callback, read_extraargs, true); - cachetable_miss++; - cachetable_misstime += get_tnow() - t0; + toku_unsafe_inc(&cachetable_miss); + toku_unsafe_add(&cachetable_misstime, get_tnow() - t0); // If the lock_type requested was a PL_READ, we downgrade to PL_READ, // but if the request was for a PL_WRITE_CHEAP, we don't bother @@ -2063,8 +2063,8 @@ int toku_cachetable_get_and_pin_nonblocking( // no list lock is held uint64_t t0 = get_tnow(); cachetable_fetch_pair(ct, cf, p, fetch_callback, read_extraargs, false); - cachetable_miss++; - cachetable_misstime += get_tnow() - t0; + toku_unsafe_inc(&cachetable_miss); + toku_unsafe_add(&cachetable_misstime, get_tnow() - t0); if (ct->ev.should_client_thread_sleep()) { ct->ev.wait_for_cache_pressure_to_subside(); @@ -2206,7 +2206,7 @@ int toku_cachefile_prefetch(CACHEFILE cf, CACHEKEY key, uint32_t fullhash, p = ct->list.find_pair(cf, key, fullhash); // if not found then create a pair and fetch it if (p == NULL) { - cachetable_prefetches++; + toku_unsafe_inc(&cachetable_prefetches); ct->list.pair_unlock_by_fullhash(fullhash); ct->list.write_list_lock(); ct->list.pair_lock_by_fullhash(fullhash); @@ -2764,7 +2764,7 @@ void toku_cachetable_begin_checkpoint (CHECKPOINTER cp, TOKULOGGER UU(logger)) { // This is used by the cachetable_race test. -static volatile int toku_checkpointing_user_data_status = 0; +static std::atomic_int toku_checkpointing_user_data_status = {0}; static void toku_cachetable_set_checkpointing_user_data_status (int v) { toku_checkpointing_user_data_status = v; } @@ -3076,7 +3076,7 @@ int cleaner::run_cleaner(void) { int r; uint32_t num_iterations = this->get_iterations(); for (uint32_t i = 0; i < num_iterations; ++i) { - cleaner_executions++; + toku_unsafe_inc(&cleaner_executions); m_pl->read_list_lock(); PAIR best_pair = NULL; int n_seen = 0; @@ -3779,7 +3779,7 @@ void evictor::change_pair_attr(PAIR_ATTR old_attr, PAIR_ATTR new_attr) { // the size of the cachetable. // void evictor::add_to_size_current(long size) { - (void) toku_sync_fetch_and_add(&m_size_current, size); + m_size_current += size; // atomic add fetch } // @@ -3787,7 +3787,7 @@ void evictor::add_to_size_current(long size) { // approximation of the cachetable size. // void evictor::remove_from_size_current(long size) { - (void) toku_sync_fetch_and_sub(&m_size_current, size); + m_size_current -= size; // atomic sub fetch } // @@ -3812,14 +3812,11 @@ void evictor::remove_cloned_data_size(long size) { uint64_t evictor::reserve_memory(double fraction, uint64_t upper_bound) { toku_mutex_lock(&m_ev_thread_lock); uint64_t reserved_memory = fraction * (m_low_size_watermark - m_size_reserved); - if (0) { // debug - fprintf(stderr, "%s %" PRIu64 " %" PRIu64 "\n", __PRETTY_FUNCTION__, reserved_memory, upper_bound); - } if (upper_bound > 0 && reserved_memory > upper_bound) { reserved_memory = upper_bound; } m_size_reserved += reserved_memory; - (void) toku_sync_fetch_and_add(&m_size_current, reserved_memory); + m_size_current += reserved_memory; // atomic add fetch this->signal_eviction_thread_locked(); toku_mutex_unlock(&m_ev_thread_lock); @@ -3833,7 +3830,7 @@ uint64_t evictor::reserve_memory(double fraction, uint64_t upper_bound) { // TODO: (Zardosht) comment this function // void evictor::release_reserved_memory(uint64_t reserved_memory){ - (void) toku_sync_fetch_and_sub(&m_size_current, reserved_memory); + m_size_current -= reserved_memory; // atomic sub fetch toku_mutex_lock(&m_ev_thread_lock); m_size_reserved -= reserved_memory; // signal the eviction thread in order to possibly wake up sleeping clients @@ -3992,7 +3989,7 @@ bool evictor::run_eviction_on_pair(PAIR curr_in_clock) { // extract and use these values so that we don't risk them changing // out from underneath us in calculations below. n_in_table = m_pl->m_n_in_table; - size_current = m_size_current; + size_current = unsafe_read_size_current(); // now that we have the pair mutex we care about, we can // release the read list lock and reacquire it at the end of the function diff --git a/ft/cachetable/checkpoint.cc b/ft/cachetable/checkpoint.cc index 13ff5eff5..d82931c97 100644 --- a/ft/cachetable/checkpoint.cc +++ b/ft/cachetable/checkpoint.cc @@ -195,7 +195,7 @@ checkpoint_safe_checkpoint_unlock(void) { void toku_multi_operation_client_lock(void) { - if (locked_mo) + if (toku_unsafe_fetch(locked_mo)) (void) toku_sync_fetch_and_add(&CP_STATUS_VAL(CP_CLIENT_WAIT_ON_MO), 1); toku_pthread_rwlock_rdlock(&multi_operation_lock); } @@ -215,7 +215,7 @@ void toku_low_priority_multi_operation_client_unlock(void) { void toku_checkpoint_safe_client_lock(void) { - if (locked_cs) + if (toku_unsafe_fetch(locked_cs)) (void) toku_sync_fetch_and_add(&CP_STATUS_VAL(CP_CLIENT_WAIT_ON_CS), 1); toku_mutex_lock(&checkpoint_safe_mutex); checkpoint_safe_lock.read_lock(); diff --git a/ft/ft-flusher.cc b/ft/ft-flusher.cc index e6452f60c..c0c2f861a 100644 --- a/ft/ft-flusher.cc +++ b/ft/ft-flusher.cc @@ -99,22 +99,22 @@ find_heaviest_child(FTNODE node) static void update_flush_status(FTNODE child, int cascades) { - FL_STATUS_VAL(FT_FLUSHER_FLUSH_TOTAL)++; + FL_STATUS_INC(FT_FLUSHER_FLUSH_TOTAL); if (cascades > 0) { - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES)++; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES); switch (cascades) { case 1: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_1)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_1); break; case 2: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_2)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_2); break; case 3: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_3)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_3); break; case 4: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_4)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_4); break; case 5: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_5)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_5); break; default: - FL_STATUS_VAL(FT_FLUSHER_FLUSH_CASCADES_GT_5)++; break; + FL_STATUS_INC(FT_FLUSHER_FLUSH_CASCADES_GT_5); break; } } bool flush_needs_io = false; @@ -124,9 +124,9 @@ update_flush_status(FTNODE child, int cascades) { } } if (flush_needs_io) { - FL_STATUS_VAL(FT_FLUSHER_FLUSH_NEEDED_IO)++; + FL_STATUS_INC(FT_FLUSHER_FLUSH_NEEDED_IO); } else { - FL_STATUS_VAL(FT_FLUSHER_FLUSH_IN_MEMORY)++; + FL_STATUS_INC(FT_FLUSHER_FLUSH_IN_MEMORY); } } @@ -685,7 +685,7 @@ ftleaf_split( { paranoid_invariant(node->height == 0); - FL_STATUS_VAL(FT_FLUSHER_SPLIT_LEAF)++; + FL_STATUS_INC(FT_FLUSHER_SPLIT_LEAF); if (node->n_children) { // First move all the accumulated stat64info deltas into the first basement. // After the split, either both nodes or neither node will be included in the next checkpoint. @@ -862,7 +862,7 @@ ft_nonleaf_split( FTNODE* dependent_nodes) { //VERIFY_NODE(t,node); - FL_STATUS_VAL(FT_FLUSHER_SPLIT_NONLEAF)++; + FL_STATUS_INC(FT_FLUSHER_SPLIT_NONLEAF); toku_ftnode_assert_fully_in_memory(node); int old_n_children = node->n_children; int n_children_in_a = old_n_children/2; @@ -1018,7 +1018,7 @@ flush_this_child( static void merge_leaf_nodes(FTNODE a, FTNODE b) { - FL_STATUS_VAL(FT_FLUSHER_MERGE_LEAF)++; + FL_STATUS_INC(FT_FLUSHER_MERGE_LEAF); toku_ftnode_assert_fully_in_memory(a); toku_ftnode_assert_fully_in_memory(b); paranoid_invariant(a->height == 0); @@ -1094,7 +1094,7 @@ static void balance_leaf_nodes( // If b is bigger then move stuff from b to a until b is the smaller. // If a is bigger then move stuff from a to b until a is the smaller. { - FL_STATUS_VAL(FT_FLUSHER_BALANCE_LEAF)++; + FL_STATUS_INC(FT_FLUSHER_BALANCE_LEAF); // first merge all the data into a merge_leaf_nodes(a,b); // now split them @@ -1173,7 +1173,7 @@ maybe_merge_pinned_nonleaf_nodes( *did_rebalance = false; toku_init_dbt(splitk); - FL_STATUS_VAL(FT_FLUSHER_MERGE_NONLEAF)++; + FL_STATUS_INC(FT_FLUSHER_MERGE_NONLEAF); } static void @@ -1627,16 +1627,16 @@ update_cleaner_status( FTNODE node, int childnum) { - FL_STATUS_VAL(FT_FLUSHER_CLEANER_TOTAL_NODES)++; + FL_STATUS_INC(FT_FLUSHER_CLEANER_TOTAL_NODES); if (node->height == 1) { - FL_STATUS_VAL(FT_FLUSHER_CLEANER_H1_NODES)++; + FL_STATUS_INC(FT_FLUSHER_CLEANER_H1_NODES); } else { - FL_STATUS_VAL(FT_FLUSHER_CLEANER_HGT1_NODES)++; + FL_STATUS_INC(FT_FLUSHER_CLEANER_HGT1_NODES); } unsigned int nbytesinbuf = toku_bnc_nbytesinbuf(BNC(node, childnum)); if (nbytesinbuf == 0) { - FL_STATUS_VAL(FT_FLUSHER_CLEANER_EMPTY_NODES)++; + FL_STATUS_INC(FT_FLUSHER_CLEANER_EMPTY_NODES); } else { if (nbytesinbuf > FL_STATUS_VAL(FT_FLUSHER_CLEANER_MAX_BUFFER_SIZE)) { FL_STATUS_VAL(FT_FLUSHER_CLEANER_MAX_BUFFER_SIZE) = nbytesinbuf; diff --git a/ft/ft-ops.cc b/ft/ft-ops.cc index 60885ed9f..d036366dd 100644 --- a/ft/ft-ops.cc +++ b/ft/ft-ops.cc @@ -4880,6 +4880,94 @@ static void toku_pfs_keys_init(const char *toku_instr_group_name) { toku_instr_probe_1 = new toku_instr_probe(*fti_probe_1_key); } +static void toku_pfs_keys_destroy(void) { + delete kibbutz_mutex_key; + delete minicron_p_mutex_key; + delete queue_result_mutex_key; + delete tpool_lock_mutex_key; + delete workset_lock_mutex_key; + delete bjm_jobs_lock_mutex_key; + delete log_internal_lock_mutex_key; + delete cachetable_ev_thread_lock_mutex_key; + delete cachetable_disk_nb_mutex_key; + delete safe_file_size_lock_mutex_key; + delete cachetable_m_mutex_key; + delete checkpoint_safe_mutex_key; + delete ft_ref_lock_mutex_key; + delete ft_open_close_lock_mutex_key; + delete loader_error_mutex_key; + delete bfs_mutex_key; + delete loader_bl_mutex_key; + delete loader_fi_lock_mutex_key; + delete loader_out_mutex_key; + delete result_output_condition_lock_mutex_key; + delete block_table_mutex_key; + delete rollback_log_node_cache_mutex_key; + delete txn_lock_mutex_key; + delete txn_state_lock_mutex_key; + delete txn_child_manager_mutex_key; + delete txn_manager_lock_mutex_key; + delete treenode_mutex_key; + delete locktree_request_info_mutex_key; + delete locktree_request_info_retry_mutex_key; + delete manager_mutex_key; + delete manager_escalation_mutex_key; + delete db_txn_struct_i_txn_mutex_key; + delete manager_escalator_mutex_key; + delete indexer_i_indexer_lock_mutex_key; + delete indexer_i_indexer_estimate_lock_mutex_key; + + delete tokudb_file_data_key; + delete tokudb_file_load_key; + delete tokudb_file_tmp_key; + delete tokudb_file_log_key; + + delete fti_probe_1_key; + + delete extractor_thread_key; + delete fractal_thread_key; + delete io_thread_key; + delete eviction_thread_key; + delete kibbutz_thread_key; + delete minicron_thread_key; + delete tp_internal_thread_key; + + delete result_state_cond_key; + delete bjm_jobs_wait_key; + delete cachetable_p_refcount_wait_key; + delete cachetable_m_flow_control_cond_key; + delete cachetable_m_ev_thread_cond_key; + delete bfs_cond_key; + delete result_output_condition_key; + delete manager_m_escalator_done_key; + delete lock_request_m_wait_cond_key; + delete queue_result_cond_key; + delete ws_worker_wait_key; + delete rwlock_wait_read_key; + delete rwlock_wait_write_key; + delete rwlock_cond_key; + delete tp_thread_wait_key; + delete tp_pool_wait_free_key; + delete frwlock_m_wait_read_key; + delete kibbutz_k_cond_key; + delete minicron_p_condvar_key; + delete locktree_request_info_retry_cv_key; + + delete multi_operation_lock_key; + delete low_priority_multi_operation_lock_key; + delete cachetable_m_list_lock_key; + delete cachetable_m_pending_lock_expensive_key; + delete cachetable_m_pending_lock_cheap_key; + delete cachetable_m_lock_key; + delete result_i_open_dbs_rwlock_key; + delete checkpoint_safe_rwlock_key; + delete cachetable_value_key; + delete safe_file_size_lock_rwlock_key; + + delete cachetable_disk_nb_rwlock_key; + delete toku_instr_probe_1; +} + int toku_ft_layer_init(void) { int r = 0; @@ -4916,8 +5004,7 @@ void toku_ft_layer_destroy(void) { toku_status_destroy(); partitioned_counters_destroy(); toku_scoped_malloc_destroy(); - - delete toku_instr_probe_1; + toku_pfs_keys_destroy(); // Portability must be cleaned up last toku_portability_destroy(); diff --git a/ft/ft-status.h b/ft/ft-status.h index b17616168..5fa785983 100644 --- a/ft/ft-status.h +++ b/ft/ft-status.h @@ -435,7 +435,13 @@ extern FT_FLUSHER_STATUS_S fl_status; #define FL_STATUS_VAL(x) fl_status.status[FT_FLUSHER_STATUS_S::x].value.num - +#define FL_STATUS_INC(x) FL_STATUS_VAL(x)++ +#if defined(__has_feature) +#if __has_feature(thread_sanitizer) +#undef FL_STATUS_INC +#define FL_STATUS_INC(x) toku_unsafe_inc(&FL_STATUS_VAL(x)) +#endif +#endif // // Hot Flusher diff --git a/ft/ft.cc b/ft/ft.cc index 454bf1179..79a1af50c 100644 --- a/ft/ft.cc +++ b/ft/ft.cc @@ -196,7 +196,7 @@ static void ft_checkpoint (CACHEFILE cf, int fd, void *header_v) { ch->checkpoint_count++; ft_hack_highest_unused_msn_for_upgrade_for_checkpoint(ft); ch->on_disk_logical_rows = - ft->h->on_disk_logical_rows = ft->in_memory_logical_rows; + ft->h->on_disk_logical_rows = toku_unsafe_fetch(ft->in_memory_logical_rows); // write translation and header to disk (or at least to OS internal buffer) toku_serialize_ft_to(fd, ch, &ft->blocktable, ft->cf); diff --git a/ft/serialize/ft-serialize.cc b/ft/serialize/ft-serialize.cc index fcab9fc67..bc604f68d 100644 --- a/ft/serialize/ft-serialize.cc +++ b/ft/serialize/ft-serialize.cc @@ -417,8 +417,10 @@ static size_t serialize_ft_min_size(uint32_t version) { switch (version) { case FT_LAYOUT_VERSION_29: size += sizeof(uint64_t); // logrows in ft + /* fall through */ case FT_LAYOUT_VERSION_28: size += sizeof(uint32_t); // fanout in ft + /* fall through */ case FT_LAYOUT_VERSION_27: case FT_LAYOUT_VERSION_26: case FT_LAYOUT_VERSION_25: @@ -427,10 +429,12 @@ static size_t serialize_ft_min_size(uint32_t version) { case FT_LAYOUT_VERSION_22: case FT_LAYOUT_VERSION_21: size += sizeof(MSN); // max_msn_in_ft + /* fall through */ case FT_LAYOUT_VERSION_20: case FT_LAYOUT_VERSION_19: size += 1; // compression method size += sizeof(MSN); // highest_unused_msn_for_upgrade + /* fall through */ case FT_LAYOUT_VERSION_18: size += sizeof(uint64_t); // time_of_last_optimize_begin size += sizeof(uint64_t); // time_of_last_optimize_end @@ -438,9 +442,11 @@ static size_t serialize_ft_min_size(uint32_t version) { size += sizeof(MSN); // msn_at_start_of_last_completed_optimize size -= 8; // removed num_blocks_to_upgrade_14 size -= 8; // removed num_blocks_to_upgrade_13 + /* fall through */ case FT_LAYOUT_VERSION_17: size += 16; invariant(sizeof(STAT64INFO_S) == 16); + /* fall through */ case FT_LAYOUT_VERSION_16: case FT_LAYOUT_VERSION_15: size += 4; // basement node size @@ -448,8 +454,10 @@ static size_t serialize_ft_min_size(uint32_t version) { // num_blocks_to_upgrade, now one int each for upgrade // from 13, 14 size += 8; // time of last verification + /* fall through */ case FT_LAYOUT_VERSION_14: size += 8; // TXNID that created + /* fall through */ case FT_LAYOUT_VERSION_13: size += (4 // build_id + @@ -459,7 +467,7 @@ static size_t serialize_ft_min_size(uint32_t version) { + 8 // time_of_last_modification ); - // fall through + /* fall through */ case FT_LAYOUT_VERSION_12: size += (+8 // "tokudata" + diff --git a/ft/tests/CMakeLists.txt b/ft/tests/CMakeLists.txt index 270ec9766..73930dda6 100644 --- a/ft/tests/CMakeLists.txt +++ b/ft/tests/CMakeLists.txt @@ -123,22 +123,4 @@ if(BUILD_TESTING OR BUILD_FT_TESTS) get_filename_component(test_basename "${test}" NAME) add_ft_test_aux(test-${test_basename} test-upgrade-recovery-logs ${test}) endforeach(test) - - ## give some tests, that time out normally, 1 hour to complete - set(long_tests - ft/ftloader-test-extractor-3a - ft/log-test7 - ft/recovery-bad-last-entry - ft/subblock-test-compression - ft/upgrade_test_simple - ) - set_tests_properties(${long_tests} PROPERTIES TIMEOUT 3600) - ## some take even longer, with valgrind - set(extra_long_tests - ft/benchmark-test - ft/benchmark-test_256 - ft/is_empty - ft/subblock-test-checksum - ) - set_tests_properties(${extra_long_tests} PROPERTIES TIMEOUT 7200) endif(BUILD_TESTING OR BUILD_FT_TESTS) diff --git a/ft/tests/cachetable-5097.cc b/ft/tests/cachetable-5097.cc index 5ab0df88e..5b8a542a8 100644 --- a/ft/tests/cachetable-5097.cc +++ b/ft/tests/cachetable-5097.cc @@ -41,11 +41,11 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. CACHEFILE f1; CACHEFILE f2; -bool check_flush; -bool dirty_flush_called; -bool check_pe_callback; -bool pe_callback_called; -bool enable_partial_eviction; +std::atomic_bool check_flush; +std::atomic_bool dirty_flush_called; +std::atomic_bool check_pe_callback; +std::atomic_bool pe_callback_called; +std::atomic_bool enable_partial_eviction; CACHETABLE ct; diff --git a/ft/tests/cachetable-checkpoint-pending.cc b/ft/tests/cachetable-checkpoint-pending.cc index 5e87fed74..e90fc1e07 100644 --- a/ft/tests/cachetable-checkpoint-pending.cc +++ b/ft/tests/cachetable-checkpoint-pending.cc @@ -51,7 +51,7 @@ int *values; static const int item_size = sizeof(int); -static volatile int n_flush, n_write_me, n_keep_me, n_fetch; +static std::atomic_int n_flush, n_write_me, n_keep_me, n_fetch; static void sleep_random (void) @@ -84,9 +84,9 @@ flush ( int *CAST_FROM_VOIDP(v, value); if (*v!=expect_value) printf("got %d expect %d\n", *v, expect_value); assert(*v==expect_value); - (void)toku_sync_fetch_and_add(&n_flush, 1); - if (write_me) (void)toku_sync_fetch_and_add(&n_write_me, 1); - if (keep_me) (void)toku_sync_fetch_and_add(&n_keep_me, 1); + (void)n_flush.fetch_add(1); + if (write_me) (void)n_write_me.fetch_add(1); + if (keep_me) (void)n_keep_me.fetch_add(1); sleep_random(); } diff --git a/ft/tests/cachetable-cleaner-thread-simple.cc b/ft/tests/cachetable-cleaner-thread-simple.cc index 8a5aa983f..b53a3aa5e 100644 --- a/ft/tests/cachetable-cleaner-thread-simple.cc +++ b/ft/tests/cachetable-cleaner-thread-simple.cc @@ -44,7 +44,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. // CACHEFILE f1; -bool my_cleaner_callback_called; +std::atomic_bool my_cleaner_callback_called; static int my_cleaner_callback( diff --git a/ft/tests/cachetable-clock-eviction.cc b/ft/tests/cachetable-clock-eviction.cc index 4f8516008..72332dbd8 100644 --- a/ft/tests/cachetable-clock-eviction.cc +++ b/ft/tests/cachetable-clock-eviction.cc @@ -38,10 +38,9 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "test.h" -int num_entries; -bool flush_may_occur; -int expected_flushed_key; -bool check_flush; +std::atomic_bool flush_may_occur; +std::atomic_int expected_flushed_key; +std::atomic_bool check_flush; static void flush (CACHEFILE f __attribute__((__unused__)), @@ -59,7 +58,7 @@ flush (CACHEFILE f __attribute__((__unused__)), ) { /* Do nothing */ if (check_flush && !keep) { - if (verbose) { printf("FLUSH: %d write_me %d expected %d\n", (int)k.b, w, expected_flushed_key); } + if (verbose) { printf("FLUSH: %d write_me %d expected %d\n", (int)k.b, w, (int)expected_flushed_key); } assert(flush_may_occur); assert(!w); assert(expected_flushed_key == (int)k.b); @@ -88,7 +87,6 @@ fetch (CACHEFILE f __attribute__((__unused__)), static void cachetable_test (void) { const int test_limit = 4; - num_entries = 0; int r; CACHETABLE ct; toku_cachetable_create(&ct, test_limit, ZERO_LSN, nullptr); diff --git a/ft/tests/cachetable-clock-eviction3.cc b/ft/tests/cachetable-clock-eviction3.cc index 97ae526a6..100a0e0d0 100644 --- a/ft/tests/cachetable-clock-eviction3.cc +++ b/ft/tests/cachetable-clock-eviction3.cc @@ -38,8 +38,8 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "test.h" -bool flush_may_occur; -long expected_bytes_to_free; +std::atomic_bool flush_may_occur; +std::atomic_long expected_bytes_to_free; static void flush (CACHEFILE f __attribute__((__unused__)), diff --git a/ft/tests/cachetable-clone-checkpoint.cc b/ft/tests/cachetable-clone-checkpoint.cc index 99d595b1f..0908f4f6a 100644 --- a/ft/tests/cachetable-clone-checkpoint.cc +++ b/ft/tests/cachetable-clone-checkpoint.cc @@ -47,8 +47,8 @@ clone_callback(void* UU(value_data), void** cloned_value_data, long* clone_size, new_attr->is_valid = false; } -bool clone_flush_started; -bool clone_flush_completed; +std::atomic_bool clone_flush_started; +std::atomic_bool clone_flush_completed; CACHETABLE ct; static void diff --git a/ft/tests/cachetable-flush-during-cleaner.cc b/ft/tests/cachetable-flush-during-cleaner.cc index c52444d8e..c44199516 100644 --- a/ft/tests/cachetable-flush-during-cleaner.cc +++ b/ft/tests/cachetable-flush-during-cleaner.cc @@ -40,7 +40,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. CACHEFILE f1; -bool should_close; +std::atomic_bool should_close; static int cleaner_callback( diff --git a/ft/tests/cachetable-pin-checkpoint.cc b/ft/tests/cachetable-pin-checkpoint.cc index 65b02aeba..786ad6c24 100644 --- a/ft/tests/cachetable-pin-checkpoint.cc +++ b/ft/tests/cachetable-pin-checkpoint.cc @@ -56,7 +56,7 @@ int64_t checkpointed_data[NUM_ELEMENTS]; PAIR data_pair[NUM_ELEMENTS]; uint32_t time_of_test; -bool run_test; +std::atomic_bool run_test; static void clone_callback( diff --git a/ft/tests/cachetable-prefetch-checkpoint-test.cc b/ft/tests/cachetable-prefetch-checkpoint-test.cc index 13c4f2efa..ea00dd725 100644 --- a/ft/tests/cachetable-prefetch-checkpoint-test.cc +++ b/ft/tests/cachetable-prefetch-checkpoint-test.cc @@ -48,7 +48,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. const int item_size = 1; -int n_flush, n_write_me, n_keep_me, n_fetch; +std::atomic_int n_flush, n_write_me, n_keep_me, n_fetch; static void flush( CACHEFILE UU(cf), diff --git a/ft/tests/cachetable-put-checkpoint.cc b/ft/tests/cachetable-put-checkpoint.cc index 4cf167844..387d17f7e 100644 --- a/ft/tests/cachetable-put-checkpoint.cc +++ b/ft/tests/cachetable-put-checkpoint.cc @@ -59,7 +59,7 @@ int64_t checkpointed_data[NUM_ELEMENTS]; PAIR data_pair[NUM_ELEMENTS]; uint32_t time_of_test; -bool run_test; +std::atomic_bool run_test; static void put_callback_pair( diff --git a/ft/tests/cachetable-simple-close.cc b/ft/tests/cachetable-simple-close.cc index f5024806f..70f827c05 100644 --- a/ft/tests/cachetable-simple-close.cc +++ b/ft/tests/cachetable-simple-close.cc @@ -39,8 +39,8 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "test.h" #include "cachetable-test.h" -bool close_called; -bool free_called; +std::atomic_bool close_called; +std::atomic_bool free_called; static void close_usr(CACHEFILE UU(cf), int UU(i), void* UU(p), bool UU(b), LSN UU(lsn)) { close_called = true; diff --git a/ft/tests/ft-clock-test.cc b/ft/tests/ft-clock-test.cc index 26a3dae67..00ff8cf20 100644 --- a/ft/tests/ft-clock-test.cc +++ b/ft/tests/ft-clock-test.cc @@ -184,11 +184,11 @@ static void test2(int fd, FT ft_h, FTNODE *dn) { PAIR_ATTR attr; memset(&attr, 0, sizeof(attr)); toku_ftnode_pe_callback(*dn, attr, ft_h, def_pe_finalize_impl, nullptr); - invariant(BP_STATE(*dn, 0) == (is_leaf) ? PT_ON_DISK : PT_COMPRESSED); + invariant(BP_STATE(*dn, 0) == ((is_leaf) ? PT_ON_DISK : PT_COMPRESSED)); invariant(BP_STATE(*dn, 1) == PT_AVAIL); invariant(BP_SHOULD_EVICT(*dn, 1)); toku_ftnode_pe_callback(*dn, attr, ft_h, def_pe_finalize_impl, nullptr); - invariant(BP_STATE(*dn, 1) == (is_leaf) ? PT_ON_DISK : PT_COMPRESSED); + invariant(BP_STATE(*dn, 1) == ((is_leaf) ? PT_ON_DISK : PT_COMPRESSED)); bool req = toku_ftnode_pf_req_callback(*dn, &bfe_subset); invariant(req); diff --git a/ft/tests/log-test4.cc b/ft/tests/log-test4.cc index e0bbedb95..019852bb7 100644 --- a/ft/tests/log-test4.cc +++ b/ft/tests/log-test4.cc @@ -54,7 +54,7 @@ test_main (int argc __attribute__((__unused__)), { ml_lock(&logger->input_lock); toku_logger_make_space_in_inbuf(logger, 5); - snprintf(logger->inbuf.buf+logger->inbuf.n_in_buf, 5, "a1234"); + memcpy(logger->inbuf.buf+logger->inbuf.n_in_buf, "a1234", 5); logger->inbuf.n_in_buf+=5; logger->lsn.lsn++; logger->inbuf.max_lsn_in_buf = logger->lsn; diff --git a/ft/tests/test-checkpoint-during-flush.cc b/ft/tests/test-checkpoint-during-flush.cc index 06a266148..659e97325 100644 --- a/ft/tests/test-checkpoint-during-flush.cc +++ b/ft/tests/test-checkpoint-during-flush.cc @@ -53,8 +53,8 @@ enum { NODESIZE = 1024, KSIZE=NODESIZE-100, TOKU_PSIZE=20 }; CACHETABLE ct; FT_HANDLE t; -bool checkpoint_called; -bool checkpoint_callback_called; +std::atomic_bool checkpoint_called; +std::atomic_bool checkpoint_callback_called; toku_pthread_t checkpoint_tid; diff --git a/ft/tests/test-checkpoint-during-merge.cc b/ft/tests/test-checkpoint-during-merge.cc index 1029dfef3..baad51c63 100644 --- a/ft/tests/test-checkpoint-during-merge.cc +++ b/ft/tests/test-checkpoint-during-merge.cc @@ -53,8 +53,8 @@ enum { NODESIZE = 1024, KSIZE=NODESIZE-100, TOKU_PSIZE=20 }; CACHETABLE ct; FT_HANDLE t; -bool checkpoint_called; -bool checkpoint_callback_called; +std::atomic_bool checkpoint_called; +std::atomic_bool checkpoint_callback_called; toku_pthread_t checkpoint_tid; diff --git a/ft/tests/test-checkpoint-during-rebalance.cc b/ft/tests/test-checkpoint-during-rebalance.cc index 208ebe3ca..95bbc84ce 100644 --- a/ft/tests/test-checkpoint-during-rebalance.cc +++ b/ft/tests/test-checkpoint-during-rebalance.cc @@ -53,8 +53,8 @@ enum { NODESIZE = 1024, KSIZE=NODESIZE-100, TOKU_PSIZE=20 }; CACHETABLE ct; FT_HANDLE t; -bool checkpoint_called; -bool checkpoint_callback_called; +std::atomic_bool checkpoint_called; +std::atomic_bool checkpoint_callback_called; toku_pthread_t checkpoint_tid; diff --git a/ft/tests/test-checkpoint-during-split.cc b/ft/tests/test-checkpoint-during-split.cc index 2b29de409..48101742c 100644 --- a/ft/tests/test-checkpoint-during-split.cc +++ b/ft/tests/test-checkpoint-during-split.cc @@ -53,8 +53,8 @@ enum { NODESIZE = 1024, KSIZE=NODESIZE-100, TOKU_PSIZE=20 }; CACHETABLE ct; FT_HANDLE t; -bool checkpoint_called; -bool checkpoint_callback_called; +std::atomic_bool checkpoint_called; +std::atomic_bool checkpoint_callback_called; toku_pthread_t checkpoint_tid; diff --git a/ft/tests/test.h b/ft/tests/test.h index 81faba203..6e1faadd9 100644 --- a/ft/tests/test.h +++ b/ft/tests/test.h @@ -46,6 +46,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include +#include #include #include "ft/serialize/block_allocator.h" diff --git a/ft/tests/test3681.cc b/ft/tests/test3681.cc index 9e4a46e8d..83b75b387 100644 --- a/ft/tests/test3681.cc +++ b/ft/tests/test3681.cc @@ -45,13 +45,14 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "cachetable/checkpoint.h" #include "test.h" +#include CACHETABLE ct; FT_HANDLE t; static TOKUTXN const null_txn = 0; -volatile bool done = false; +std::atomic_bool done (false); static void setup (void) { toku_cachetable_create(&ct, 0, ZERO_LSN, nullptr); diff --git a/locktree/locktree.cc b/locktree/locktree.cc index 069aae26f..e1f7d6d75 100644 --- a/locktree/locktree.cc +++ b/locktree/locktree.cc @@ -118,11 +118,11 @@ void lt_lock_request_info::destroy(void) { } void locktree::add_reference(void) { - (void)toku_sync_add_and_fetch(&m_reference_count, 1); + m_reference_count++; } uint32_t locktree::release_reference(void) { - return toku_sync_sub_and_fetch(&m_reference_count, 1); + return --m_reference_count; } uint32_t locktree::get_reference_count(void) { diff --git a/locktree/locktree.h b/locktree/locktree.h index 1ba7a51b1..8b564a080 100644 --- a/locktree/locktree.h +++ b/locktree/locktree.h @@ -181,7 +181,7 @@ namespace toku { // tracks the current number of locks and lock memory uint64_t m_max_lock_memory; - uint64_t m_current_lock_memory; + std::atomic_uint64_t m_current_lock_memory; struct lt_counters m_lt_counters; @@ -318,7 +318,7 @@ namespace toku { private: locktree_manager *m_mgr; DICTIONARY_ID m_dict_id; - uint32_t m_reference_count; + std::atomic_uint32_t m_reference_count; // Since the memory referenced by this comparator is not owned by the // locktree, the user must guarantee it will outlive the locktree. diff --git a/locktree/manager.cc b/locktree/manager.cc index 6bb5c77bf..9678fd3b8 100644 --- a/locktree/manager.cc +++ b/locktree/manager.cc @@ -280,11 +280,11 @@ void locktree_manager::escalate_all_locktrees(void) { } void locktree_manager::note_mem_used(uint64_t mem_used) { - (void) toku_sync_fetch_and_add(&m_current_lock_memory, mem_used); + m_current_lock_memory.fetch_add(mem_used); } void locktree_manager::note_mem_released(uint64_t mem_released) { - uint64_t old_mem_used = toku_sync_fetch_and_sub(&m_current_lock_memory, mem_released); + uint64_t old_mem_used = m_current_lock_memory.fetch_sub(mem_released); invariant(old_mem_used >= mem_released); } diff --git a/locktree/tests/kill_waiter.cc b/locktree/tests/kill_waiter.cc index 8d93c0bbb..a1db80cc5 100644 --- a/locktree/tests/kill_waiter.cc +++ b/locktree/tests/kill_waiter.cc @@ -8,7 +8,6 @@ #include "test.h" #include "locktree_unit_test.h" #include -#include namespace toku { diff --git a/locktree/tests/lock_request_start_release_wait.cc b/locktree/tests/lock_request_start_release_wait.cc index 717628f9d..d6539c500 100644 --- a/locktree/tests/lock_request_start_release_wait.cc +++ b/locktree/tests/lock_request_start_release_wait.cc @@ -8,8 +8,6 @@ // again when lock retries are locked out. #include "lock_request.h" -#include -#include #include "locktree.h" #include "locktree_unit_test.h" #include "test.h" diff --git a/locktree/tests/locktree_escalation_1big7lt_1small.cc b/locktree/tests/locktree_escalation_1big7lt_1small.cc index 32029b5bd..d78ed1bc7 100644 --- a/locktree/tests/locktree_escalation_1big7lt_1small.cc +++ b/locktree/tests/locktree_escalation_1big7lt_1small.cc @@ -46,7 +46,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. using namespace toku; static int verbose = 0; -static int killed = 0; +static std::atomic_int killed (0); static pthread_t big_id, small_id; static void locktree_release_lock(locktree *lt, TXNID txn_id, int64_t left_k, int64_t right_k) { diff --git a/locktree/tests/locktree_escalation_2big_1lt.cc b/locktree/tests/locktree_escalation_2big_1lt.cc index ff59a7bdd..70a97fc8e 100644 --- a/locktree/tests/locktree_escalation_2big_1lt.cc +++ b/locktree/tests/locktree_escalation_2big_1lt.cc @@ -47,7 +47,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. using namespace toku; static int verbose = 0; -static int killed = 0; +static std::atomic_int killed (0); static void locktree_release_lock(locktree *lt, TXNID txn_id, int64_t left_k, int64_t right_k) { range_buffer buffer; diff --git a/locktree/tests/locktree_escalation_2big_2lt.cc b/locktree/tests/locktree_escalation_2big_2lt.cc index be1ddaba9..8c48e05a4 100644 --- a/locktree/tests/locktree_escalation_2big_2lt.cc +++ b/locktree/tests/locktree_escalation_2big_2lt.cc @@ -47,7 +47,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. using namespace toku; static int verbose = 0; -static int killed = 0; +static std::atomic_int killed (0); static void locktree_release_lock(locktree *lt, TXNID txn_id, int64_t left_k, int64_t right_k) { range_buffer buffer; diff --git a/locktree/tests/locktree_escalation_stalls.cc b/locktree/tests/locktree_escalation_stalls.cc index 9dc9596a7..f21a71056 100644 --- a/locktree/tests/locktree_escalation_stalls.cc +++ b/locktree/tests/locktree_escalation_stalls.cc @@ -54,7 +54,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. using namespace toku; static int verbose = 0; -static int killed = 0; +static std::atomic_int killed (0); static void locktree_release_lock(locktree *lt, TXNID txn_id, int64_t left_k, int64_t right_k) { range_buffer buffer; diff --git a/locktree/tests/test.h b/locktree/tests/test.h index 921f2468b..15e4f6099 100644 --- a/locktree/tests/test.h +++ b/locktree/tests/test.h @@ -38,6 +38,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #pragma once +#include #include #include "ft/comparator.h" diff --git a/portability/tests/test-pthread-rwlock-rwr.cc b/portability/tests/test-pthread-rwlock-rwr.cc index 92b30421b..568035acf 100644 --- a/portability/tests/test-pthread-rwlock-rwr.cc +++ b/portability/tests/test-pthread-rwlock-rwr.cc @@ -45,17 +45,17 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. // write a test see if things happen in the right order. -volatile int state = 0; +std::atomic_int state (0); int verbose = 0; static void *f(void *arg) { toku_pthread_rwlock_t *mylock = (toku_pthread_rwlock_t *) arg; - sleep(2); + while (state != 42) sleep(2); assert(state==42); state = 16; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); toku_pthread_rwlock_wrlock(mylock); assert(state==49); state = 17; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); toku_pthread_rwlock_wrunlock(mylock); - sleep(10); + while (state != 52) sleep(10); assert(state==52); state = 20; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); return arg; } @@ -84,15 +84,15 @@ int test_main(int argc , char *const argv[] ) { state = 42; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); - sleep(4); + while (state != 16) sleep(4); assert(state==16); state = 44; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); toku_pthread_rwlock_rdlock(&rwlock); assert(state==44); state = 46; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); toku_pthread_rwlock_rdunlock(&rwlock); - sleep(4); + while (state != 46) sleep(4); assert(state==46); state=49; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); // still have a read lock toku_pthread_rwlock_rdunlock(&rwlock); - sleep(6); + while (state != 17) sleep(6); assert(state==17); state=52; if (verbose) printf("%s:%d\n", __FUNCTION__, __LINE__); r = toku_pthread_join(tid, &retptr); assert(r == 0); diff --git a/portability/tests/test.h b/portability/tests/test.h index c5150553e..951ae65a1 100644 --- a/portability/tests/test.h +++ b/portability/tests/test.h @@ -38,6 +38,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include +#include #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); }) #define CKERR2(r,r2) do { if (r!=r2) fprintf(stderr, "%s:%d error %d %s, expected %d\n", __FILE__, __LINE__, r, strerror(r), r2); assert(r==r2); } while (0) diff --git a/portability/toku_instrumentation.h b/portability/toku_instrumentation.h index 8c9390edc..c300f9275 100644 --- a/portability/toku_instrumentation.h +++ b/portability/toku_instrumentation.h @@ -52,6 +52,8 @@ class toku_instr_key { UU(const char *name)) {} explicit toku_instr_key(UU(pfs_key_t key_id)) {} + + ~toku_instr_key() {} }; typedef toku_instr_probe_empty toku_instr_probe; diff --git a/portability/toku_race_tools.h b/portability/toku_race_tools.h index 8482a164f..8deaacccb 100644 --- a/portability/toku_race_tools.h +++ b/portability/toku_race_tools.h @@ -142,3 +142,13 @@ template void toku_unsafe_set(T &dest, const T src) { toku_unsafe_set(&dest, src); } + +template +void toku_unsafe_add(T *dest, const T i) { + *dest += i; +} + +template +void toku_unsafe_inc(T *dest) { + *dest += 1; +} diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt index c01a8f0d6..4494c2b53 100644 --- a/src/tests/CMakeLists.txt +++ b/src/tests/CMakeLists.txt @@ -197,7 +197,6 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS) endif () add_ydb_drd_test_aux(drd_tiny_${test} ${test} --num_seconds 5 --num_elements 150 --join_timeout 3000) - set_tests_properties(ydb/drd_tiny_${test} PROPERTIES TIMEOUT 3600) add_test(ydb/drd_mid_${test}/prepare ${test} --only_create --num_elements 10000) setup_toku_test_properties(ydb/drd_mid_${test}/prepare drd_mid_${test}) @@ -205,7 +204,6 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS) set_tests_properties(ydb/drd_mid_${test} PROPERTIES DEPENDS ydb/drd_mid_${test}/prepare REQUIRED_FILES "drd_mid_${test}.ctest-data" - TIMEOUT 15000 ) add_test(ydb/drd_large_${test}/prepare ${test} --only_create --num_elements 150000) @@ -214,7 +212,6 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS) set_tests_properties(ydb/drd_large_${test} PROPERTIES DEPENDS ydb/drd_large_${test}/prepare REQUIRED_FILES "drd_large_${test}.ctest-data" - TIMEOUT 30000 ) endif() endforeach(src) @@ -246,14 +243,12 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS) set_tests_properties(${testnamebase}/copy PROPERTIES DEPENDS ${testnamebase}/remove REQUIRED_FILES "${oldenvdir}") - add_test(NAME ${testnamebase} COMMAND ${test} --only_stress --num_elements ${size} --num_seconds 600 --join_timeout 7200) setup_toku_test_properties(${testnamebase} "${envdirbase}") set_tests_properties(${testnamebase} PROPERTIES DEPENDS ${testnamebase}/copy - REQUIRED_FILES "${envdir}" - TIMEOUT 10800) + REQUIRED_FILES "${envdir}") endforeach(size) endif () endforeach(p_or_s) @@ -433,61 +428,4 @@ if(BUILD_TESTING OR BUILD_SRC_TESTS) string(REGEX REPLACE ";" ";ydb/" tdb_tests_that_should_fail "${tdb_tests_that_should_fail}") set_tests_properties(${tdb_tests_that_should_fail} PROPERTIES WILL_FAIL TRUE) - ## give some tests, that time out normally, 1 hour to complete - set(long_tests - ydb/drd_test_groupcommit_count.tdb - ydb/env-put-multiple.tdb - ydb/filesize.tdb - ydb/loader-cleanup-test0.tdb - ydb/loader-cleanup-test0z.tdb - ydb/manyfiles.tdb - ydb/recover-loader-test.abortrecover - ydb/recovery_fileops_stress.tdb - ydb/root_fifo_1.tdb - ydb/root_fifo_2.tdb - ydb/root_fifo_31.tdb - ydb/root_fifo_32.tdb - ydb/shutdown-3344.tdb - ydb/stat64-create-modify-times.tdb - ydb/test1572.tdb - ydb/test_abort4_19_0.tdb - ydb/test_abort4_19_1.tdb - ydb/test_abort5.tdb - ydb/test_archive1.tdb - ydb/test_logmax.tdb - ydb/test_query.tdb - ydb/test_txn_abort5.tdb - ydb/test_txn_abort5a.tdb - ydb/test_txn_abort6.tdb - ydb/test_txn_nested2.tdb - ydb/test_txn_nested4.tdb - ydb/test_txn_nested5.tdb - ydb/test_update_broadcast_stress.tdb - ) - set_tests_properties(${long_tests} PROPERTIES TIMEOUT 3600) - ## some take even longer, with valgrind - set(extra_long_tests - ydb/drd_test_4015.tdb - ydb/hotindexer-with-queries.tdb - ydb/hot-optimize-table-tests.tdb - ydb/loader-cleanup-test2.tdb - ydb/loader-cleanup-test2z.tdb - ydb/loader-dup-test0.tdb - ydb/loader-stress-del.nop.loader - ydb/loader-stress-del.p.loader - ydb/loader-stress-del.comp.loader - ydb/test3039.tdb - ydb/test_update_stress.tdb - ) - set_tests_properties(${extra_long_tests} PROPERTIES TIMEOUT 7200) - ## these really take a long time with valgrind - set(phenomenally_long_tests - ydb/checkpoint_stress.tdb - ydb/loader-stress-test4.tdb - ydb/loader-stress-test4z.tdb - ydb/recover_stress.tdb - ydb/test3529.tdb - ydb/test_insert_unique.tdb - ) - set_tests_properties(${phenomenally_long_tests} PROPERTIES TIMEOUT 14400) endif(BUILD_TESTING OR BUILD_SRC_TESTS) diff --git a/src/tests/checkpoint_fairness.cc b/src/tests/checkpoint_fairness.cc index f62dfbc76..465e71a20 100644 --- a/src/tests/checkpoint_fairness.cc +++ b/src/tests/checkpoint_fairness.cc @@ -50,16 +50,17 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "test.h" #include "toku_pthread.h" #include +#include DB_ENV *env; DB *db; const char *env_dir = TOKU_TEST_FILENAME; const int n_threads = 100; -volatile int reader_start_count = 0; +std::atomic_int reader_start_count = { 0 }; const int W = 10; -volatile int writer_done_count = 0; +std::atomic_int writer_done_count = { 0 }; static void *start_txns (void *e) { int *CAST_FROM_VOIDP(idp, e); @@ -72,7 +73,7 @@ static void *start_txns (void *e) { { int chk_r = env->txn_begin(env, NULL, &txn, 0); CKERR(chk_r); } { int chk_r = db->put(db, txn, &k, &k, 0); CKERR(chk_r); } { int chk_r = txn->commit(txn, 0); CKERR(chk_r); } - if (j==10) (void)toku_sync_fetch_and_add(&reader_start_count, 1); + if (j==10) (void)reader_start_count.fetch_add(1); if (j%1000==999) { printf("."); fflush(stdout); } assert(j<1000); // Get upset if we manage to run this many transactions without the checkpoint thread } @@ -86,7 +87,7 @@ static void start_checkpoints (void) { { int chk_r = env->txn_checkpoint(env, 0, 0, 0); CKERR(chk_r); } if (verbose) printf("ck\n"); sched_yield(); - (void)toku_sync_fetch_and_add(&writer_done_count, 1); + (void)writer_done_count.fetch_add(1); } } diff --git a/src/tests/checkpoint_stress.cc b/src/tests/checkpoint_stress.cc index 135a9843c..d3e5ddd50 100644 --- a/src/tests/checkpoint_stress.cc +++ b/src/tests/checkpoint_stress.cc @@ -351,7 +351,7 @@ test_main (int argc, char * const argv[]) { // arg that suppresses valgrind on this child process break; } - // otherwise, fall through to an error + /* fall through */ // otherwise, fall through to an error case 'h': case '?': usage(argv[0]); diff --git a/src/tests/directory_lock.cc b/src/tests/directory_lock.cc index f040e6809..b28a71704 100644 --- a/src/tests/directory_lock.cc +++ b/src/tests/directory_lock.cc @@ -69,7 +69,7 @@ static void verify_shared_ops_fail(DB_ENV* env, DB* db) { uint32_t flags = 0; DBT key,val; DBT in_key,in_val; - uint32_t in_key_data, in_val_data = 0; + uint32_t in_key_data = 0, in_val_data = 0; memset(&in_key, 0, sizeof(in_key)); memset(&in_val, 0, sizeof(in_val)); in_key.size = sizeof(in_key_data); diff --git a/src/tests/hotindexer-multiclient.cc b/src/tests/hotindexer-multiclient.cc index 004a19ecc..22986ab50 100644 --- a/src/tests/hotindexer-multiclient.cc +++ b/src/tests/hotindexer-multiclient.cc @@ -73,7 +73,7 @@ typedef struct { uint32_t *flags; } client_spec_t, *client_spec; -int client_count = 0; +std::atomic_int client_count = { 0 }; static void * client(void *arg) { diff --git a/src/tests/loader-cleanup-test.cc b/src/tests/loader-cleanup-test.cc index ea894683c..a229cb5b5 100644 --- a/src/tests/loader-cleanup-test.cc +++ b/src/tests/loader-cleanup-test.cc @@ -172,12 +172,12 @@ err_type_str (enum test_type t) { case einval_o: return "open"; case enospc_fc: return "fclose"; case abort_via_poll: return "abort_via_poll"; - case commit: assert(0); - case abort_txn: assert(0); - case abort_loader: assert(0); + case commit: abort(); + case abort_txn: abort(); + case abort_loader: abort(); } // I know that Barry prefers the single-return case, but writing the code this way means that the compiler will complain if I forget something in the enum. -Bradley - assert(0); + abort(); return NULL; } @@ -193,12 +193,12 @@ err_msg_type_str (enum test_type t) { case einval_o: return "EINVAL"; case enospc_fc: return "ENOSPC"; case abort_via_poll: return "non-zero"; - case commit: assert(0); - case abort_txn: assert(0); - case abort_loader: assert(0); + case commit: abort(); + case abort_txn: abort(); + case abort_loader: abort(); } // I know that Barry prefers the single-return case, but writing the code this way means that the compiler will complain if I forget something in the enum. -Bradley - assert(0); + abort(); return NULL; } @@ -873,7 +873,7 @@ static void run_test(enum test_type t, int trigger) case abort_via_poll: poll_count_trigger = trigger; break; default: - assert(0); + abort(); } diff --git a/src/tests/locktree_escalation_stalls.cc b/src/tests/locktree_escalation_stalls.cc index e6c1b18b2..856c77631 100644 --- a/src/tests/locktree_escalation_stalls.cc +++ b/src/tests/locktree_escalation_stalls.cc @@ -82,7 +82,7 @@ static void print_matching_engine_status_rows(DB_ENV *env, const char *pattern) fflush(stderr); } -static volatile int killed = 0; +static std::atomic_int killed = { 0 }; // in a big transaction, insert a bunch of rows. static void big_test(DB_ENV *env, DB *db, uint64_t max_i) { diff --git a/src/tests/recover-del-multiple-abort.cc b/src/tests/recover-del-multiple-abort.cc index a8455c0f4..425c12e1a 100644 --- a/src/tests/recover-del-multiple-abort.cc +++ b/src/tests/recover-del-multiple-abort.cc @@ -81,7 +81,7 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals memcpy(dest_key->data, &pri_data[dbnum], dest_key->size); break; default: - assert(0); + abort(); } if (dest_val) { @@ -95,9 +95,9 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals } break; case DB_DBT_REALLOC: - assert(0); + abort(); default: - assert(0); + abort(); } } diff --git a/src/tests/recover-del-multiple-srcdb-fdelete-all.cc b/src/tests/recover-del-multiple-srcdb-fdelete-all.cc index e823a7462..75479cb69 100644 --- a/src/tests/recover-del-multiple-srcdb-fdelete-all.cc +++ b/src/tests/recover-del-multiple-srcdb-fdelete-all.cc @@ -85,7 +85,7 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals memcpy(dest_key->data, &pri_data[dbnum], dest_key->size); break; default: - assert(0); + abort(); } if (dest_val) { @@ -99,9 +99,9 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals } break; case DB_DBT_REALLOC: - assert(0); + abort(); default: - assert(0); + abort(); } } diff --git a/src/tests/recover-del-multiple.cc b/src/tests/recover-del-multiple.cc index c2ee80c43..9f4b1cd9c 100644 --- a/src/tests/recover-del-multiple.cc +++ b/src/tests/recover-del-multiple.cc @@ -84,7 +84,7 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals memcpy(dest_key->data, &pri_data[dbnum], dest_key->size); break; default: - assert(0); + abort(); } if (dest_val) { @@ -98,9 +98,9 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals } break; case DB_DBT_REALLOC: - assert(0); + abort(); default: - assert(0); + abort(); } } diff --git a/src/tests/recover-put-multiple-abort.cc b/src/tests/recover-put-multiple-abort.cc index d04580096..da40a61f2 100644 --- a/src/tests/recover-put-multiple-abort.cc +++ b/src/tests/recover-put-multiple-abort.cc @@ -81,7 +81,7 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals memcpy(dest_key->data, &pri_data[dbnum], dest_key->size); break; default: - assert(0); + abort(); } if (dest_val) { @@ -95,9 +95,9 @@ put_callback(DB *dest_db, DB *src_db, DBT_ARRAY *dest_keys, DBT_ARRAY *dest_vals } break; case DB_DBT_REALLOC: - assert(0); + abort(); default: - assert(0); + abort(); } } diff --git a/src/tests/recover-test_crash_in_flusher_thread.h b/src/tests/recover-test_crash_in_flusher_thread.h index 5c10d0cb7..29ff86bb1 100644 --- a/src/tests/recover-test_crash_in_flusher_thread.h +++ b/src/tests/recover-test_crash_in_flusher_thread.h @@ -52,8 +52,8 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "threaded_stress_test_helpers.h" toku_pthread_t checkpoint_tid; -static int cnt = 0; -static bool starting_a_chkpt = false; +static std::atomic_int cnt = { 0 }; +static std::atomic_bool starting_a_chkpt = { false }; int state_to_crash = 0; diff --git a/src/tests/recovery_fileops_unit.cc b/src/tests/recovery_fileops_unit.cc index cc99ab560..f2043d802 100644 --- a/src/tests/recovery_fileops_unit.cc +++ b/src/tests/recovery_fileops_unit.cc @@ -217,7 +217,7 @@ do_args(int argc, char * const argv[]) { // arg that suppresses valgrind on this child process break; } - // otherwise, fall through to an error + /* fall through */ // otherwise, fall through to an error default: usage(); break; diff --git a/src/tests/test-prepare3.cc b/src/tests/test-prepare3.cc index 5cb3796a2..f57fc9635 100644 --- a/src/tests/test-prepare3.cc +++ b/src/tests/test-prepare3.cc @@ -128,6 +128,7 @@ static void check_prepared_list (enum prepared_state ps[NTXNS], long count, DB_P goto next; case PREPARED: count_prepared++; + /* fall through */ case MAYBE_COMMITTED: case MAYBE_ABORTED: count_maybe_prepared++; diff --git a/src/tests/test.h b/src/tests/test.h index ff464f558..672f49e81 100644 --- a/src/tests/test.h +++ b/src/tests/test.h @@ -53,6 +53,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include "toku_assert.h" #include #include +#include #include "ydb.h" //TDB uses DB_NOTFOUND for c_del and DB_CURRENT errors. diff --git a/src/tests/test3039.cc b/src/tests/test3039.cc index 6c38fa135..af8e4cc20 100644 --- a/src/tests/test3039.cc +++ b/src/tests/test3039.cc @@ -109,8 +109,13 @@ struct reader_thread_state { int do_local; /* communicate to the thread while running */ - volatile int finish; + std::atomic_int finish; + reader_thread_state(double _elapsed_time, unsigned long long _n_did_read, + signed long long _n_to_read, int _do_local, int _finish) : + elapsed_time(_elapsed_time), n_did_read(_n_did_read), n_to_read(_n_to_read), + do_local(_do_local), finish(_finish) { + } }; static @@ -182,16 +187,10 @@ void* reader_thread (void *arg) static void do_threads (unsigned long long N, int do_nonlocal) { toku_pthread_t ths[2]; - struct reader_thread_state rstates[2] = {{.elapsed_time = 0.0, - .n_did_read = 0, - .n_to_read = (long long signed)N, - .do_local = 1, - .finish = 0}, - {.elapsed_time = 0.0, - .n_did_read = 0, - .n_to_read = -1, - .do_local = 0, - .finish = 0}}; + struct reader_thread_state rstates[2] = { + {0.0, 0, (long long signed)N, 1, 0}, + {0.0, 0, -1, 0, 0} + }; int n_to_create = do_nonlocal ? 2 : 1; for (int i = 0; i < n_to_create; i++) { int r = toku_pthread_create(toku_uninstrumented, @@ -222,10 +221,10 @@ void do_threads (unsigned long long N, int do_nonlocal) { } } -static volatile unsigned long long n_preads; +static std::atomic_ullong n_preads; static ssize_t my_pread (int fd, void *buf, size_t count, off_t offset) { - (void) toku_sync_fetch_and_add(&n_preads, 1); + (void) n_preads.fetch_add(1); usleep(1000); // sleep for a millisecond return pread(fd, buf, count, offset); } @@ -269,16 +268,16 @@ int test_main (int argc, char * const argv[]) { db_env_set_func_pread(my_pread); create_db (N); - if (verbose) printf("%lld preads\n", n_preads); + if (verbose) printf("%lld preads\n", (long long) n_preads); do_threads (M, 0); - if (verbose) printf("%lld preads\n", n_preads); + if (verbose) printf("%lld preads\n", (long long) n_preads); do_threads (M, 0); - if (verbose) printf("%lld preads\n", n_preads); + if (verbose) printf("%lld preads\n", (long long) n_preads); do_threads (M, 1); - if (verbose) printf("%lld preads\n", n_preads); + if (verbose) printf("%lld preads\n", (long long) n_preads); { int r = db->close(db, 0); CKERR(r); } { int r = env->close(env, 0); CKERR(r); } - if (verbose) printf("%lld preads\n", n_preads); + if (verbose) printf("%lld preads\n", (long long) n_preads); return 0; } diff --git a/src/tests/test3219.cc b/src/tests/test3219.cc index 99d1a90a7..af3492700 100644 --- a/src/tests/test3219.cc +++ b/src/tests/test3219.cc @@ -130,7 +130,7 @@ finish (void) { } -volatile int finished = false; +std::atomic_int finished = { false }; // Thread A performs checkpoints static void* diff --git a/src/tests/test_3645.cc b/src/tests/test_3645.cc index e1fa37bef..01a3eb11e 100644 --- a/src/tests/test_3645.cc +++ b/src/tests/test_3645.cc @@ -61,7 +61,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. // If the test runs to completion without crashing, we consider it a success. // -bool run_test; +std::atomic_bool run_test; int time_of_test; int num_elements; diff --git a/src/tests/test_4015.cc b/src/tests/test_4015.cc index 1231e3b4b..d8415c0e5 100644 --- a/src/tests/test_4015.cc +++ b/src/tests/test_4015.cc @@ -56,7 +56,7 @@ static int my_compare (DB *db, const DBT *a, const DBT *b) { DB_ENV *env; DB *db; const char *env_dir = TOKU_TEST_FILENAME; -volatile int done = 0; +std::atomic_int done = { 0 }; static void *startA (void *ignore __attribute__((__unused__))) { for (int i=0;i<999; i++) { @@ -78,7 +78,7 @@ static void *startA (void *ignore __attribute__((__unused__))) { } { int chk_r = txn->commit(txn, 0); CKERR(chk_r); } } - int r __attribute__((__unused__)) = toku_sync_fetch_and_add(&done, 1); + int r __attribute__((__unused__)) = done.fetch_add(1); return NULL; } static void change_descriptor (DB_TXN *txn, int i) { diff --git a/src/tests/test_lock_timeout_callback.cc b/src/tests/test_lock_timeout_callback.cc index 571bae699..ee3173a42 100644 --- a/src/tests/test_lock_timeout_callback.cc +++ b/src/tests/test_lock_timeout_callback.cc @@ -45,13 +45,13 @@ static DB_ENV *env; static DB *db; static DB_TXN *txn1, *txn2; static const int magic_key = 100; -static int callback_calls; +static std::atomic_int callback_calls; toku_pthread_t thread1; static void lock_not_granted(DB *_db, uint64_t requesting_txnid, const DBT *left_key, const DBT *right_key, uint64_t blocking_txnid) { - toku_sync_fetch_and_add(&callback_calls, 1); + callback_calls++; // atomic fetch add invariant(strcmp(_db->get_dname(_db), db->get_dname(db)) == 0); if (requesting_txnid == txn2->id64(txn2)) { invariant(blocking_txnid == txn1->id64(txn1)); diff --git a/src/tests/threaded_stress_test_helpers.h b/src/tests/threaded_stress_test_helpers.h index e232f327d..60722eb12 100644 --- a/src/tests/threaded_stress_test_helpers.h +++ b/src/tests/threaded_stress_test_helpers.h @@ -77,7 +77,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. static const size_t min_val_size = sizeof(int32_t); static const size_t min_key_size = sizeof(int32_t); -volatile bool run_test; // should be volatile since we are communicating through this variable. +std::atomic_bool run_test; // should be volatile since we are communicating through this variable. typedef struct arg *ARG; typedef int (*operation_t)(DB_TXN *txn, ARG arg, void *operation_extra, void *stats_extra); @@ -1712,9 +1712,10 @@ static void *test_time(void *arg) { if (verbose) { printf("should now end test\n"); } - toku_sync_bool_compare_and_swap(&run_test, true, false); // make this atomic to make valgrind --tool=drd happy. + bool expected = true; + run_test.compare_exchange_strong(expected, false); // toku_sync_bool_compare_and_swap(&run_test, true, false); // make this atomic to make valgrind --tool=drd happy. if (verbose) { - printf("run_test %d\n", run_test); + printf("run_test %d\n", (bool)run_test); } if (tte->crash_at_end) { toku_hard_crash_on_purpose(); diff --git a/src/ydb_write.cc b/src/ydb_write.cc index 8cd7e2201..fd9fb4e85 100644 --- a/src/ydb_write.cc +++ b/src/ydb_write.cc @@ -52,7 +52,8 @@ static YDB_WRITE_LAYER_STATUS_S ydb_write_layer_status; #undef STATUS_VALUE #endif #define STATUS_VALUE(x) ydb_write_layer_status.status[x].value.num - +#define STATUS_VALUE_ADD(x,y) TOKUFT_STATUS_ADD(STATUS_VALUE(x),y) +#define STATUS_VALUE_INC(x) TOKUFT_STATUS_INC(STATUS_VALUE(x)) #define STATUS_INIT(k,c,t,l,inc) TOKUFT_STATUS_INIT(ydb_write_layer_status, k, c, t, l, inc) static void @@ -192,10 +193,10 @@ toku_db_del(DB *db, DB_TXN *txn, DBT *key, uint32_t flags, bool holds_mo_lock) { } if (r == 0) { - STATUS_VALUE(YDB_LAYER_NUM_DELETES)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_DELETES); // accountability } else { - STATUS_VALUE(YDB_LAYER_NUM_DELETES_FAIL)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_DELETES_FAIL); // accountability } return r; } @@ -251,11 +252,11 @@ toku_db_put(DB *db, DB_TXN *txn, DBT *key, DBT *val, uint32_t flags, bool holds_ if (r == 0) { // helgrind flags a race on this status update. we increment it atomically to satisfy helgrind. - // STATUS_VALUE(YDB_LAYER_NUM_INSERTS)++; // accountability - (void) toku_sync_fetch_and_add(&STATUS_VALUE(YDB_LAYER_NUM_INSERTS), 1); + STATUS_VALUE_INC(YDB_LAYER_NUM_INSERTS); // accountability + // (void) toku_sync_fetch_and_add(&STATUS_VALUE(YDB_LAYER_NUM_INSERTS), 1); } else { - // STATUS_VALUE(YDB_LAYER_NUM_INSERTS_FAIL)++; // accountability - (void) toku_sync_fetch_and_add(&STATUS_VALUE(YDB_LAYER_NUM_INSERTS_FAIL), 1); + STATUS_VALUE_INC(YDB_LAYER_NUM_INSERTS_FAIL); // accountability + // (void) toku_sync_fetch_and_add(&STATUS_VALUE(YDB_LAYER_NUM_INSERTS_FAIL), 1); } return r; @@ -293,9 +294,9 @@ toku_db_update(DB *db, DB_TXN *txn, cleanup: if (r == 0) - STATUS_VALUE(YDB_LAYER_NUM_UPDATES)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_UPDATES); // accountability else - STATUS_VALUE(YDB_LAYER_NUM_UPDATES_FAIL)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_UPDATES_FAIL); // accountability return r; } @@ -351,9 +352,9 @@ toku_db_update_broadcast(DB *db, DB_TXN *txn, cleanup: if (r == 0) - STATUS_VALUE(YDB_LAYER_NUM_UPDATES_BROADCAST)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_UPDATES_BROADCAST); // accountability else - STATUS_VALUE(YDB_LAYER_NUM_UPDATES_BROADCAST_FAIL)++; // accountability + STATUS_VALUE_INC(YDB_LAYER_NUM_UPDATES_BROADCAST_FAIL); // accountability return r; } @@ -582,9 +583,9 @@ env_del_multiple( cleanup: if (r == 0) - STATUS_VALUE(YDB_LAYER_NUM_MULTI_DELETES) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_DELETES, (uint64_t)num_dbs); // accountability else - STATUS_VALUE(YDB_LAYER_NUM_MULTI_DELETES_FAIL) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_DELETES_FAIL, (uint64_t)num_dbs); // accountability return r; } @@ -764,9 +765,9 @@ env_put_multiple_internal( cleanup: if (r == 0) - STATUS_VALUE(YDB_LAYER_NUM_MULTI_INSERTS) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_INSERTS, (uint64_t)num_dbs); // accountability else - STATUS_VALUE(YDB_LAYER_NUM_MULTI_INSERTS_FAIL) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_INSERTS_FAIL, (uint64_t)num_dbs); // accountability return r; } @@ -1050,9 +1051,9 @@ env_update_multiple(DB_ENV *env, DB *src_db, DB_TXN *txn, cleanup: if (r == 0) - STATUS_VALUE(YDB_LAYER_NUM_MULTI_UPDATES) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_UPDATES, (uint64_t)num_dbs); // accountability else - STATUS_VALUE(YDB_LAYER_NUM_MULTI_UPDATES_FAIL) += num_dbs; // accountability + STATUS_VALUE_ADD(YDB_LAYER_NUM_MULTI_UPDATES_FAIL, (uint64_t)num_dbs); // accountability return r; } diff --git a/tsan.suppressions b/tsan.suppressions new file mode 100644 index 000000000..3d274f65c --- /dev/null +++ b/tsan.suppressions @@ -0,0 +1,58 @@ +# suppress operations already marked unsafe +race:toku_unsafe_fetch +race:toku_unsafe_set +race:toku_unsafe_inc +race:toku_unsafe_add + +# ignore racy omt functions marked as ignored by valgrind. +race:toku::omt_internal::subtree_templated::enable_bit +race:toku::omt_internal::subtree_templated::get_bit +race:toku::omt_internal::subtree_templated::get_index + +# races on the partitioned counter sum are ignored by valgrind, should be ignored +# by the thread sanitizer. alternatively, the sum could be an c++ atomic variable. +race:sumit(local_counter*, unsigned long*) +race:^increment_partitioned_counter$ + +# ignore racy util status functions +race:toku_kibbutz_get_status +race:toku_context_get_status + +# ignore racy locktree status function +race:locktree_manager::get_status + +# ignore deadlocks in concurrent tree due to tree rebalancing. deadlocks can not +# happen in concurrent tree algorithms, but thread sanitizer does not know that. +deadlock:toku::treenode::child_ptr::get_locked + +# ignore racy ft functions +race:evictor::fill_engine_status +race:evictor::get_state + +# ignore racy in ft tests +race:evictor_unit_test:: +race:evictor_test_helpers:: +race:cleaner::get_iterations +race:cleaner::set_iterations + +# ignore racy ft status functions +race:update_flush_status(ftnode*, int) +race:toku_ft_flusher_get_status +race:toku_ft_hot_get_status +race:toku_ft_upgrade_get_status +race:toku_ft_get_status +race:toku_le_get_status +race:toku_cachetable_get_status +race:toku_cachetable_get_state +race:update_le_status +race:toku_checkpoint_get_status + +# the adjust logical row count is racy. could use an algorithm that does +# an unsafe read followed by a compare and exchange instead. +race:toku_ft_adjust_logical_row_count + +# stat64 is racy on many variables +race:toku_ft_stat64 + +# memory statistics are racy. probably they should be removed. +race:portability/memory.cc diff --git a/util/dbt.cc b/util/dbt.cc index 5bc1cb744..d0c105dcd 100644 --- a/util/dbt.cc +++ b/util/dbt.cc @@ -199,7 +199,7 @@ int toku_dbt_set(uint32_t len, const void *val, DBT *d, struct simple_dbt *sdbt) case (DB_DBT_MALLOC): d->data = NULL; d->ulen = 0; - //Fall through to DB_DBT_REALLOC + /* fall through */ case (DB_DBT_REALLOC): if (d->ulen < len) { d->ulen = len*2; diff --git a/util/minicron.cc b/util/minicron.cc index c1412015b..241e498c7 100644 --- a/util/minicron.cc +++ b/util/minicron.cc @@ -85,8 +85,9 @@ minicron_do (void *pv) toku_cond_wait(&p->condvar, &p->mutex); } else if (p->period_in_ms <= 1000) { + uint32_t period_in_ms = p->period_in_ms; toku_mutex_unlock(&p->mutex); - usleep(p->period_in_ms * 1000); + usleep(period_in_ms * 1000); toku_mutex_lock(&p->mutex); } else { diff --git a/util/status.h b/util/status.h index ddf43c72d..ecc2fac4a 100644 --- a/util/status.h +++ b/util/status.h @@ -40,6 +40,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include +#include #define TOKUFT_STATUS_INIT(array,k,c,t,l,inc) do { \ array.status[k].keyname = #k; \ @@ -59,3 +60,13 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. } \ } while (0) +#define TOKUFT_STATUS_ADD(v,n) (v) += (n) +#define TOKUFT_STATUS_INC(v) (v)++ +#if defined(__has_feature) +#if __has_feature(thread_sanitizer) +#undef TOKUFT_STATUS_ADD +#define TOKUFT_STATUS_ADD(v,n) toku_unsafe_add(&(v),(n)) +#undef TOKUFT_STATUS_INC +#define TOKUFT_STATUS_INC(v) toku_unsafe_inc(&(v)) +#endif +#endif diff --git a/util/tests/CMakeLists.txt b/util/tests/CMakeLists.txt index 8d53dd89a..b71681c0b 100644 --- a/util/tests/CMakeLists.txt +++ b/util/tests/CMakeLists.txt @@ -16,9 +16,4 @@ if(BUILD_TESTING) foreach(test ${tests}) add_test(util/${test} ${test}) endforeach(test) - - set(long_tests - util/helgrind_test_partitioned_counter - ) - set_tests_properties(${long_tests} PROPERTIES TIMEOUT 3600) endif(BUILD_TESTING) diff --git a/util/tests/marked-omt-test.cc b/util/tests/marked-omt-test.cc index 7e60c711c..10d2dabc1 100644 --- a/util/tests/marked-omt-test.cc +++ b/util/tests/marked-omt-test.cc @@ -45,7 +45,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include - +#include #include #include @@ -155,7 +155,7 @@ int int_heaviside(const uint32_t &v, const uint32_t &target) { struct stress_shared { stress_omt *omt; - volatile bool running; + std::atomic_bool running; struct st_rwlock lock; toku_mutex_t mutex; int num_marker_threads; @@ -385,7 +385,8 @@ static void *stress_delete_worker(void *extrav) { rwlock_write_unlock(&shared.lock); toku_mutex_unlock(&mutex); } - toku_sync_bool_compare_and_swap(&shared.running, true, false); + bool expected = true; + atomic_compare_exchange_strong(&shared.running, &expected, false); return nullptr; } diff --git a/util/tests/sm-crash-double-free.cc b/util/tests/sm-crash-double-free.cc index 5aa356551..7611c89b2 100644 --- a/util/tests/sm-crash-double-free.cc +++ b/util/tests/sm-crash-double-free.cc @@ -44,8 +44,9 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include +#include -volatile int state = 0; +std::atomic_int state(0); static void sm_test(void) { toku::scoped_malloc a(1); diff --git a/util/tests/test-frwlock-fair-writers.cc b/util/tests/test-frwlock-fair-writers.cc index 9a625c32a..85096e505 100644 --- a/util/tests/test-frwlock-fair-writers.cc +++ b/util/tests/test-frwlock-fair-writers.cc @@ -43,10 +43,11 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include +#include toku_mutex_t rwlock_mutex; toku::frwlock rwlock; -volatile int killed = 0; +std::atomic_int killed(0); static void *t1_func(void *arg) { int i; diff --git a/util/tests/test-rwlock-unfair-writers.cc b/util/tests/test-rwlock-unfair-writers.cc index 0d1fc8551..676ae1a85 100644 --- a/util/tests/test-rwlock-unfair-writers.cc +++ b/util/tests/test-rwlock-unfair-writers.cc @@ -42,9 +42,10 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include +#include pthread_rwlock_t rwlock; -volatile int killed = 0; +std::atomic_int killed(0); static void *t1_func(void *arg) { int i; diff --git a/util/tests/test_partitioned_counter.cc b/util/tests/test_partitioned_counter.cc index a4e6f8422..a96e782a2 100644 --- a/util/tests/test_partitioned_counter.cc +++ b/util/tests/test_partitioned_counter.cc @@ -80,6 +80,7 @@ Copyright (c) 2006, 2015, Percona and/or its affiliates. All rights reserved. #include #include #include "test.h" +#include // The test code includes the fastest version I could figure out to make, implemented below. @@ -311,7 +312,7 @@ struct test_arguments { PARTITIONED_COUNTER pc; uint64_t limit; uint64_t total_increment_per_writer; - volatile uint64_t unfinished_count; + std::atomic_uint64_t unfinished_count; }; static void *reader_test_fun (void *ta_v) { @@ -322,7 +323,7 @@ static void *reader_test_fun (void *ta_v) { assert(lastval <= thisval); assert(thisval <= ta->limit+2); lastval = thisval; - if (verboseness_cmdarg && (0==(thisval & (thisval-1)))) printf("ufc=%" PRIu64 " Thisval=%" PRIu64 "\n", ta->unfinished_count,thisval); + if (verboseness_cmdarg && (0==(thisval & (thisval-1)))) printf("ufc=%" PRIu64 " Thisval=%" PRIu64 "\n", (uint64_t) ta->unfinished_count, thisval); } uint64_t thisval = read_partitioned_counter(ta->pc); assert(thisval==ta->limit+2); // we incremented two extra times in the test @@ -335,7 +336,7 @@ static void *writer_test_fun (void *ta_v) { if (i%1000 == 0) sched_yield(); increment_partitioned_counter(ta->pc, 1); } - uint64_t c __attribute__((__unused__)) = toku_sync_fetch_and_sub(&ta->unfinished_count, 1); + uint64_t c __attribute__((__unused__)) = ta->unfinished_count.fetch_sub(1); return ta_v; } @@ -375,7 +376,8 @@ static void do_testit (void) { } } -volatile int spinwait=0; +#include +std::atomic_int spinwait (0); static void* test2_fun (void* mypc_v) { PARTITIONED_COUNTER mypc = (PARTITIONED_COUNTER)mypc_v; increment_partitioned_counter(mypc, 3);