Browse Source

MDEV-21452: Replace all direct use of os_event_t

Let us replace os_event_t with mysql_cond_t, and replace the
necessary ib_mutex_t with mysql_mutex_t so that they can be
used with condition variables.

Also, let us replace polling (os_thread_sleep() or timed waits)
with plain mysql_cond_wait() wherever possible.

Furthermore, we will use the lightweight srw_mutex for trx_t::mutex,
to hopefully reduce contention on lock_sys.mutex.

FIXME: Add test coverage of
mariabackup --backup --kill-long-queries-timeout
bb-10.6-MDEV-24424
Marko Mäkelä 5 years ago
parent
commit
38fd7b7d91
  1. 112
      extra/mariabackup/backup_mysql.cc
  2. 237
      extra/mariabackup/xtrabackup.cc
  3. 1
      mysql-test/suite/sys_vars/disabled.def
  4. 12
      storage/innobase/btr/btr0btr.cc
  5. 4
      storage/innobase/btr/btr0cur.cc
  6. 332
      storage/innobase/btr/btr0defragment.cc
  7. 4
      storage/innobase/buf/buf0buf.cc
  8. 545
      storage/innobase/fil/fil0crypt.cc
  9. 28
      storage/innobase/fil/fil0fil.cc
  10. 100
      storage/innobase/fts/fts0fts.cc
  11. 187
      storage/innobase/fts/fts0opt.cc
  12. 8
      storage/innobase/gis/gis0sea.cc
  13. 192
      storage/innobase/handler/ha_innodb.cc
  14. 2
      storage/innobase/handler/ha_innodb.h
  15. 4
      storage/innobase/ibuf/ibuf0ibuf.cc
  16. 17
      storage/innobase/include/btr0defragment.h
  17. 2
      storage/innobase/include/buf0buf.h
  18. 10
      storage/innobase/include/fil0crypt.h
  19. 3
      storage/innobase/include/fts0priv.h
  20. 26
      storage/innobase/include/fts0types.h
  21. 56
      storage/innobase/include/lock0lock.h
  22. 5
      storage/innobase/include/lock0priv.h
  23. 4
      storage/innobase/include/lock0priv.ic
  24. 37
      storage/innobase/include/log0recv.h
  25. 7
      storage/innobase/include/row0ftsort.h
  26. 2
      storage/innobase/include/row0vers.h
  27. 17
      storage/innobase/include/srv0srv.h
  28. 1
      storage/innobase/include/sync0sync.h
  29. 24
      storage/innobase/include/sync0types.h
  30. 4
      storage/innobase/include/trx0sys.h
  31. 20
      storage/innobase/include/trx0trx.h
  32. 3
      storage/innobase/include/trx0types.h
  33. 13
      storage/innobase/include/ut0wqueue.h
  34. 5
      storage/innobase/lock/lock0iter.cc
  35. 417
      storage/innobase/lock/lock0lock.cc
  36. 88
      storage/innobase/lock/lock0prdt.cc
  37. 70
      storage/innobase/lock/lock0wait.cc
  38. 3
      storage/innobase/log/log0log.cc
  39. 246
      storage/innobase/log/log0recv.cc
  40. 8
      storage/innobase/os/os0file.cc
  41. 29
      storage/innobase/que/que0que.cc
  42. 23
      storage/innobase/row/row0ftsort.cc
  43. 4
      storage/innobase/row/row0ins.cc
  44. 32
      storage/innobase/row/row0merge.cc
  45. 4
      storage/innobase/row/row0mysql.cc
  46. 10
      storage/innobase/row/row0vers.cc
  47. 81
      storage/innobase/srv/srv0srv.cc
  48. 11
      storage/innobase/srv/srv0start.cc
  49. 52
      storage/innobase/sync/sync0debug.cc
  50. 1
      storage/innobase/sync/sync0sync.cc
  51. 10
      storage/innobase/trx/trx0i_s.cc
  52. 22
      storage/innobase/trx/trx0roll.cc
  53. 31
      storage/innobase/trx/trx0trx.cc
  54. 23
      storage/innobase/ut/ut0wqueue.cc

112
extra/mariabackup/backup_mysql.cc

@ -75,9 +75,10 @@ bool have_multi_threaded_slave = false;
bool have_gtid_slave = false;
/* Kill long selects */
os_event_t kill_query_thread_started;
os_event_t kill_query_thread_stopped;
os_event_t kill_query_thread_stop;
static mysql_mutex_t kill_query_thread_mutex;
static bool kill_query_thread_running, kill_query_thread_stopping;
static mysql_cond_t kill_query_thread_stopped;
static mysql_cond_t kill_query_thread_stop;
bool sql_thread_started = false;
char *mysql_slave_position = NULL;
@ -798,74 +799,73 @@ wait_for_no_updates(MYSQL *connection, uint timeout, uint threshold)
return(false);
}
static
os_thread_ret_t
DECLARE_THREAD(kill_query_thread)(
/*===============*/
void *arg __attribute__((unused)))
static os_thread_ret_t DECLARE_THREAD(kill_query_thread)(void*)
{
MYSQL *mysql;
time_t start_time;
start_time = time(NULL);
mysql_mutex_lock(&kill_query_thread_mutex);
os_event_set(kill_query_thread_started);
msg("Kill query timeout %d seconds.", opt_kill_long_queries_timeout);
msg("Kill query timeout %d seconds.",
opt_kill_long_queries_timeout);
time_t start_time= time(nullptr);
timespec abstime;
set_timespec(abstime, opt_kill_long_queries_timeout);
while (time(NULL) - start_time <
(time_t)opt_kill_long_queries_timeout) {
if (os_event_wait_time(kill_query_thread_stop, 1000) !=
OS_SYNC_TIME_EXCEEDED) {
goto stop_thread;
}
}
if ((mysql = xb_mysql_connect()) == NULL) {
msg("Error: kill query thread failed");
goto stop_thread;
}
while (true) {
kill_long_queries(mysql, time(NULL) - start_time);
if (os_event_wait_time(kill_query_thread_stop, 1000) !=
OS_SYNC_TIME_EXCEEDED) {
break;
}
}
while (!kill_query_thread_stopping)
if (!mysql_cond_timedwait(&kill_query_thread_stop,
&kill_query_thread_mutex, &abstime))
goto func_exit;
mysql_close(mysql);
if (MYSQL *mysql= xb_mysql_connect())
{
do
{
kill_long_queries(mysql, time(nullptr) - start_time);
set_timespec(abstime, 1);
}
while (mysql_cond_timedwait(&kill_query_thread_stop,
&kill_query_thread_mutex, &abstime) &&
!kill_query_thread_stopping);
mysql_close(mysql);
}
else
msg("Error: kill query thread failed");
stop_thread:
msg("Kill query thread stopped");
func_exit:
msg("Kill query thread stopped");
os_event_set(kill_query_thread_stopped);
kill_query_thread_running= false;
mysql_cond_signal(&kill_query_thread_stopped);
mysql_mutex_unlock(&kill_query_thread_mutex);
os_thread_exit();
OS_THREAD_DUMMY_RETURN;
os_thread_exit();
OS_THREAD_DUMMY_RETURN;
}
static
void
start_query_killer()
static void start_query_killer()
{
kill_query_thread_stop = os_event_create(0);
kill_query_thread_started = os_event_create(0);
kill_query_thread_stopped = os_event_create(0);
os_thread_create(kill_query_thread);
os_event_wait(kill_query_thread_started);
ut_ad(!kill_query_thread_running);
kill_query_thread_running= true;
kill_query_thread_stopping= false;
mysql_mutex_init(0, &kill_query_thread_mutex, nullptr);
mysql_cond_init(0, &kill_query_thread_stop, nullptr);
mysql_cond_init(0, &kill_query_thread_stopped, nullptr);
os_thread_create(kill_query_thread);
}
static
void
stop_query_killer()
static void stop_query_killer()
{
os_event_set(kill_query_thread_stop);
os_event_wait_time(kill_query_thread_stopped, 60000);
mysql_mutex_lock(&kill_query_thread_mutex);
kill_query_thread_stopping= true;
mysql_cond_signal(&kill_query_thread_stop);
do
mysql_cond_wait(&kill_query_thread_stopped, &kill_query_thread_mutex);
while (kill_query_thread_running);
mysql_cond_destroy(&kill_query_thread_stop);
mysql_cond_destroy(&kill_query_thread_stopped);
mysql_mutex_unlock(&kill_query_thread_mutex);
mysql_mutex_destroy(&kill_query_thread_mutex);
}

237
extra/mariabackup/xtrabackup.cc

@ -132,8 +132,8 @@ longlong xtrabackup_use_memory;
uint opt_protocol;
long xtrabackup_throttle; /* 0:unlimited */
static lint io_ticket;
static os_event_t wait_throttle;
static os_event_t log_copying_stop;
static mysql_cond_t wait_throttle;
static mysql_cond_t log_copying_stop;
char *xtrabackup_incremental;
lsn_t incremental_lsn;
@ -184,7 +184,6 @@ lsn_t checkpoint_lsn_start;
lsn_t checkpoint_no_start;
static lsn_t log_copy_scanned_lsn;
static bool log_copying_running;
static bool io_watching_thread_running;
int xtrabackup_parallel;
@ -657,17 +656,14 @@ struct dbug_thread_param_t
const char *query;
int expect_err;
int expect_errno;
os_event_t done_event;
};
/* Thread procedure used in dbug_start_query_thread. */
extern "C"
os_thread_ret_t
DECLARE_THREAD(dbug_execute_in_new_connection)(void *arg)
static void *dbug_execute_in_new_connection(void *arg)
{
mysql_thread_init();
dbug_thread_param_t *par= (dbug_thread_param_t *)arg;
dbug_thread_param_t *par= static_cast<dbug_thread_param_t*>(arg);
int err = mysql_query(par->con, par->query);
int err_no = mysql_errno(par->con);
if(par->expect_err != err)
@ -684,13 +680,12 @@ DECLARE_THREAD(dbug_execute_in_new_connection)(void *arg)
}
mysql_close(par->con);
mysql_thread_end();
os_event_t done = par->done_event;
delete par;
os_event_set(done);
os_thread_exit();
return os_thread_ret_t(0);
return nullptr;
}
static pthread_t dbug_alter_thread;
/*
Execute query from a new connection, in own thread.
@ -702,7 +697,7 @@ Execute query from a new connection, in own thread.
@param expected_errno - if not 0, and query finished with error,
expected mysql_errno()
*/
static os_event_t dbug_start_query_thread(
static void dbug_start_query_thread(
const char *query,
const char *wait_state,
int expected_err,
@ -713,12 +708,13 @@ static os_event_t dbug_start_query_thread(
par->query = query;
par->expect_err = expected_err;
par->expect_errno = expected_errno;
par->done_event = os_event_create(0);
par->con = xb_mysql_connect();
os_thread_create(dbug_execute_in_new_connection, par);
mysql_thread_create(0, &dbug_alter_thread, nullptr,
dbug_execute_in_new_connection, par);
if (!wait_state)
return par->done_event;
return;
char q[256];
snprintf(q, sizeof(q),
@ -740,10 +736,7 @@ static os_event_t dbug_start_query_thread(
end:
msg("query '%s' on connection %lu reached state '%s'", query,
mysql_thread_id(par->con), wait_state);
return par->done_event;
}
os_event_t dbug_alter_thread_done;
#endif
void mdl_lock_all()
@ -2414,13 +2407,15 @@ xb_write_delta_metadata(const char *filename, const xb_delta_info_t *info)
}
/* ================= backup ================= */
void
xtrabackup_io_throttling(void)
void xtrabackup_io_throttling()
{
if (xtrabackup_backup && xtrabackup_throttle && (io_ticket--) < 0) {
os_event_reset(wait_throttle);
os_event_wait(wait_throttle);
}
if (!xtrabackup_backup)
return;
mysql_mutex_lock(&log_sys.mutex);
if (xtrabackup_throttle && (io_ticket--) < 0)
mysql_cond_wait(&wait_throttle, &log_sys.mutex);
mysql_mutex_unlock(&log_sys.mutex);
}
static
@ -2916,6 +2911,8 @@ static lsn_t xtrabackup_copy_log(lsn_t start_lsn, lsn_t end_lsn, bool last)
@return whether the operation failed */
static bool xtrabackup_copy_logfile(bool last = false)
{
mysql_mutex_assert_owner(&log_sys.mutex);
ut_a(dst_log_file != NULL);
ut_ad(recv_sys.is_initialised());
@ -2931,9 +2928,10 @@ static bool xtrabackup_copy_logfile(bool last = false)
do {
end_lsn = start_lsn + RECV_SCAN_SIZE;
xtrabackup_io_throttling();
if (xtrabackup_throttle && (io_ticket--) < 0) {
mysql_cond_wait(&wait_throttle, &log_sys.mutex);
}
mysql_mutex_lock(&log_sys.mutex);
lsn_t lsn= start_lsn;
for (int retries= 0; retries < 100; retries++) {
if (log_sys.log.read_log_seg(&lsn, end_lsn)
@ -2945,22 +2943,20 @@ static bool xtrabackup_copy_logfile(bool last = false)
}
if (lsn == start_lsn) {
overwritten_block= !recv_sys.found_corrupt_log
overwritten_block= !recv_sys.is_corrupt_log()
&& log_block_calc_checksum_crc32(log_sys.buf) ==
log_block_get_checksum(log_sys.buf)
&& log_block_get_hdr_no(log_sys.buf) >
log_block_convert_lsn_to_no(start_lsn);
start_lsn = 0;
} else {
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
start_lsn = xtrabackup_copy_log(start_lsn, lsn, last);
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
}
mysql_mutex_unlock(&log_sys.mutex);
if (!start_lsn) {
const char *reason = recv_sys.found_corrupt_log
const char *reason = recv_sys.is_corrupt_log()
? "corrupt log."
: (overwritten_block
? "redo log block is overwritten, please increase redo log size with innodb_log_file_size parameter."
@ -3002,58 +2998,47 @@ extern lsn_t server_lsn_after_lock;
static os_thread_ret_t DECLARE_THREAD(log_copying_thread)(void*)
{
/*
Initialize mysys thread-specific memory so we can
use mysys functions in this thread.
*/
my_thread_init();
for (;;) {
os_event_reset(log_copying_stop);
os_event_wait_time_low(log_copying_stop,
xtrabackup_log_copy_interval * 1000U,
0);
if (xtrabackup_copy_logfile()) {
break;
}
mysql_mutex_lock(&log_sys.mutex);
bool completed = metadata_to_lsn
&& metadata_to_lsn <= log_copy_scanned_lsn;
mysql_mutex_unlock(&log_sys.mutex);
if (completed) {
break;
}
}
log_copying_running = false;
my_thread_end();
os_thread_exit();
return(0);
my_thread_init();
mysql_mutex_lock(&log_sys.mutex);
while (!xtrabackup_copy_logfile() &&
(!metadata_to_lsn || metadata_to_lsn > log_copy_scanned_lsn))
{
timespec abstime;
set_timespec_nsec(abstime, 1000ULL * xtrabackup_log_copy_interval);
mysql_cond_timedwait(&log_copying_stop, &log_sys.mutex, &abstime);
}
log_copying_running= false;
mysql_mutex_unlock(&log_sys.mutex);
my_thread_end();
os_thread_exit();
return 0;
}
static bool have_io_watching_thread;
static pthread_t io_watching_thread_id;
/* io throttle watching (rough) */
static os_thread_ret_t DECLARE_THREAD(io_watching_thread)(void*)
static void *io_watching_thread(void*)
{
/* currently, for --backup only */
ut_a(xtrabackup_backup);
/* currently, for --backup only */
ut_a(xtrabackup_backup);
while (log_copying_running && !metadata_to_lsn) {
os_thread_sleep(1000000); /*1 sec*/
io_ticket = xtrabackup_throttle;
os_event_set(wait_throttle);
}
/* stop io throttle */
xtrabackup_throttle = 0;
os_event_set(wait_throttle);
mysql_mutex_lock(&log_sys.mutex);
io_watching_thread_running = false;
os_thread_exit();
while (log_copying_running && !metadata_to_lsn)
{
timespec abstime;
set_timespec(abstime, 1);
mysql_cond_timedwait(&log_copying_stop, &log_sys.mutex, &abstime);
io_ticket= xtrabackup_throttle;
mysql_cond_broadcast(&wait_throttle);
}
return(0);
/* stop io throttle */
xtrabackup_throttle= 0;
mysql_cond_broadcast(&wait_throttle);
mysql_mutex_unlock(&log_sys.mutex);
return nullptr;
}
#ifndef DBUG_OFF
@ -4072,42 +4057,40 @@ end:
# define xb_set_max_open_files(x) 0UL
#endif
static void stop_backup_threads()
static void stop_backup_threads(bool running)
{
if (log_copying_stop && log_copying_running) {
os_event_set(log_copying_stop);
fputs("mariabackup: Stopping log copying thread", stderr);
fflush(stderr);
while (log_copying_running) {
putc('.', stderr);
fflush(stderr);
os_thread_sleep(200000); /*0.2 sec*/
}
putc('\n', stderr);
os_event_destroy(log_copying_stop);
}
if (running)
{
fputs("mariabackup: Stopping log copying thread", stderr);
fflush(stderr);
while (log_copying_running)
{
putc('.', stderr);
fflush(stderr);
os_thread_sleep(200000); /*0.2 sec*/
}
putc('\n', stderr);
mysql_cond_destroy(&log_copying_stop);
}
if (wait_throttle) {
/* wait for io_watching_thread completion */
while (io_watching_thread_running) {
os_thread_sleep(1000000);
}
os_event_destroy(wait_throttle);
}
if (have_io_watching_thread)
{
pthread_join(io_watching_thread_id, nullptr);
mysql_cond_destroy(&wait_throttle);
}
}
/** Implement the core of --backup
@return whether the operation succeeded */
static bool xtrabackup_backup_low()
{
mysql_mutex_lock(&log_sys.mutex);
ut_ad(!metadata_to_lsn);
/* read the latest checkpoint lsn */
{
ulint max_cp_field;
mysql_mutex_lock(&log_sys.mutex);
if (recv_find_max_checkpoint(&max_cp_field) == DB_SUCCESS
&& log_sys.log.format != 0) {
if (max_cp_field == LOG_CHECKPOINT_1) {
@ -4123,16 +4106,21 @@ static bool xtrabackup_backup_low()
} else {
msg("Error: recv_find_max_checkpoint() failed.");
}
mysql_cond_broadcast(&log_copying_stop);
const bool running= log_copying_running;
mysql_mutex_unlock(&log_sys.mutex);
stop_backup_threads(running);
mysql_mutex_lock(&log_sys.mutex);
}
stop_backup_threads();
if (metadata_to_lsn && xtrabackup_copy_logfile(true)) {
mysql_mutex_unlock(&log_sys.mutex);
ds_close(dst_log_file);
dst_log_file = NULL;
return false;
}
mysql_mutex_unlock(&log_sys.mutex);
if (ds_close(dst_log_file) || !metadata_to_lsn) {
dst_log_file = NULL;
@ -4220,8 +4208,14 @@ static bool xtrabackup_backup_func()
/* initialize components */
if(innodb_init_param()) {
fail:
metadata_to_lsn = log_copying_running;
stop_backup_threads();
if (log_copying_running) {
mysql_mutex_lock(&log_sys.mutex);
metadata_to_lsn = 1;
mysql_cond_broadcast(&log_copying_stop);
mysql_mutex_unlock(&log_sys.mutex);
stop_backup_threads(true);
}
log_file_op = NULL;
if (dst_log_file) {
ds_close(dst_log_file);
@ -4255,11 +4249,6 @@ fail:
trx_pool_init();
recv_sys.create();
#ifdef WITH_INNODB_DISALLOW_WRITES
srv_allow_writes_event = os_event_create(0);
os_event_set(srv_allow_writes_event);
#endif
xb_filters_init();
xb_fil_io_init();
@ -4385,10 +4374,10 @@ reread_log_header:
/* start io throttle */
if(xtrabackup_throttle) {
io_ticket = xtrabackup_throttle;
wait_throttle = os_event_create(0);
io_watching_thread_running = true;
os_thread_create(io_watching_thread);
have_io_watching_thread = true;
mysql_cond_init(0, &wait_throttle, nullptr);
mysql_thread_create(0, &io_watching_thread_id, nullptr,
io_watching_thread, nullptr);
}
/* Populate fil_system with tablespaces to copy */
@ -4405,12 +4394,18 @@ fail_before_log_copying_thread_start:
log_copy_scanned_lsn = checkpoint_lsn_start;
recv_sys.recovered_lsn = log_copy_scanned_lsn;
if (xtrabackup_copy_logfile())
mysql_mutex_lock(&log_sys.mutex);
const bool log_copy_failed = xtrabackup_copy_logfile();
mysql_mutex_unlock(&log_sys.mutex);
if (log_copy_failed)
goto fail_before_log_copying_thread_start;
DBUG_MARIABACKUP_EVENT("before_innodb_log_copy_thread_started",0);
log_copying_stop = os_event_create(0);
mysql_cond_init(0, &log_copying_stop, nullptr);
os_thread_create(log_copying_thread);
/* FLUSH CHANGED_PAGE_BITMAPS call */
@ -4431,7 +4426,6 @@ fail_before_log_copying_thread_start:
mdl_lock_all();
DBUG_EXECUTE_IF("check_mdl_lock_works",
dbug_alter_thread_done =
dbug_start_query_thread("ALTER TABLE test.t ADD COLUMN mdl_lock_column int",
"Waiting for table metadata lock", 0, 0););
}
@ -4482,9 +4476,7 @@ fail_before_log_copying_thread_start:
backup_release();
DBUG_EXECUTE_IF("check_mdl_lock_works",
os_event_wait(dbug_alter_thread_done);
os_event_destroy(dbug_alter_thread_done);
);
pthread_join(dbug_alter_thread, nullptr););
if (ok) {
backup_finish();
@ -5692,10 +5684,6 @@ static bool xtrabackup_prepare_func(char** argv)
log_sys.create();
recv_sys.recovery_on = true;
#ifdef WITH_INNODB_DISALLOW_WRITES
srv_allow_writes_event = os_event_create(0);
os_event_set(srv_allow_writes_event);
#endif
xb_fil_io_init();
if (dberr_t err = xb_load_tablespaces()) {
msg("mariabackup: error: xb_data_files_init() failed "
@ -5718,9 +5706,6 @@ static bool xtrabackup_prepare_func(char** argv)
xb_filter_hash_free(&inc_dir_tables_hash);
fil_system.close();
#ifdef WITH_INNODB_DISALLOW_WRITES
os_event_destroy(srv_allow_writes_event);
#endif
innodb_free_param();
log_sys.close();
sync_check_close();

1
mysql-test/suite/sys_vars/disabled.def

@ -12,3 +12,4 @@
all_vars: obsolete, see sysvars_* tests
innodb_buffer_pool_dump_pct_function: MDEV-11454 follow-up needed (unstable)
innodb_fatal_semaphore_wait_threshold : MDEV-21452 TODO: lock_sys.mutex

12
storage/innobase/btr/btr0btr.cc

@ -3293,10 +3293,10 @@ btr_lift_page_up(
if (!dict_table_is_locking_disabled(index->table)) {
/* Free predicate page locks on the block */
if (dict_index_is_spatial(index)) {
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
lock_update_copy_and_discard(father_block, block);
}
@ -3546,11 +3546,11 @@ retry:
}
/* No GAP lock needs to be worrying about */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(block);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
} else {
btr_cur_node_ptr_delete(&father_cursor, mtr);
if (!dict_table_is_locking_disabled(index->table)) {
@ -3699,11 +3699,11 @@ retry:
offsets2, offsets,
merge_page, mtr);
}
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_prdt_page_free_from_discard(
block, &lock_sys.prdt_page_hash);
lock_rec_free_all_from_discard_page(block);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
} else {
compressed = btr_cur_pessimistic_delete(&err, TRUE,

4
storage/innobase/btr/btr0cur.cc

@ -1997,11 +1997,11 @@ retry_page_get:
trx_t* trx = thr_get_trx(cursor->thr);
lock_prdt_t prdt;
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_init_prdt_from_mbr(
&prdt, &cursor->rtr_info->mbr, mode,
trx->lock.lock_heap);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
if (rw_latch == RW_NO_LATCH && height != 0) {
block->lock.s_lock();

332
storage/innobase/btr/btr0defragment.cc

@ -53,16 +53,15 @@ time will make sure the page is compressible within a couple of iterations. */
/** Item in the work queue for btr_degrament_thread. */
struct btr_defragment_item_t
{
btr_pcur_t* pcur; /* persistent cursor where
btr_defragment_n_pages should start */
os_event_t event; /* if not null, signal after work
is done */
bool removed; /* Mark an item as removed */
ulonglong last_processed; /* timestamp of last time this index
is processed by defragment thread */
btr_defragment_item_t(btr_pcur_t* pcur, os_event_t event);
~btr_defragment_item_t();
/** persistent cursor where btr_defragment_n_pages should start */
btr_pcur_t * const pcur;
/** completion signal */
mysql_cond_t *cond;
/** timestamp of last time this index is processed by defragment thread */
ulonglong last_processed= 0;
btr_defragment_item_t(btr_pcur_t *pcur, mysql_cond_t *cond)
: pcur(pcur), cond(cond) {}
};
/* Work queue for defragmentation. */
@ -70,9 +69,9 @@ typedef std::list<btr_defragment_item_t*> btr_defragment_wq_t;
static btr_defragment_wq_t btr_defragment_wq;
/* Mutex protecting the defragmentation work queue.*/
ib_mutex_t btr_defragment_mutex;
static mysql_mutex_t btr_defragment_mutex;
#ifdef UNIV_PFS_MUTEX
UNIV_INTERN mysql_pfs_key_t btr_defragment_mutex_key;
mysql_pfs_key_t btr_defragment_mutex_key;
#endif /* UNIV_PFS_MUTEX */
/* Number of compression failures caused by defragmentation since server
@ -87,13 +86,6 @@ the amount of effort wasted. */
Atomic_counter<ulint> btr_defragment_count;
bool btr_defragment_active;
struct defragment_chunk_state_t
{
btr_defragment_item_t* m_item;
};
static defragment_chunk_state_t defragment_chunk_state;
static void btr_defragment_chunk(void*);
static tpool::timer* btr_defragment_timer;
@ -101,29 +93,6 @@ static tpool::task_group task_group(1);
static tpool::task btr_defragment_task(btr_defragment_chunk, 0, &task_group);
static void btr_defragment_start();
/******************************************************************//**
Constructor for btr_defragment_item_t. */
btr_defragment_item_t::btr_defragment_item_t(
btr_pcur_t* pcur,
os_event_t event)
{
this->pcur = pcur;
this->event = event;
this->removed = false;
this->last_processed = 0;
}
/******************************************************************//**
Destructor for btr_defragment_item_t. */
btr_defragment_item_t::~btr_defragment_item_t() {
if (this->pcur) {
btr_pcur_free_for_mysql(this->pcur);
}
if (this->event) {
os_event_set(this->event);
}
}
static void submit_defragment_task(void*arg=0)
{
srv_thread_pool->submit_task(&btr_defragment_task);
@ -135,8 +104,8 @@ void
btr_defragment_init()
{
srv_defragment_interval = 1000000000ULL / srv_defragment_frequency;
mutex_create(LATCH_ID_BTR_DEFRAGMENT_MUTEX, &btr_defragment_mutex);
defragment_chunk_state.m_item = 0;
mysql_mutex_init(btr_defragment_mutex_key, &btr_defragment_mutex,
nullptr);
btr_defragment_timer = srv_thread_pool->create_timer(submit_defragment_task);
btr_defragment_active = true;
}
@ -151,15 +120,17 @@ btr_defragment_shutdown()
delete btr_defragment_timer;
btr_defragment_timer = 0;
task_group.cancel_pending(&btr_defragment_task);
mutex_enter(&btr_defragment_mutex);
mysql_mutex_lock(&btr_defragment_mutex);
std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
while(iter != btr_defragment_wq.end()) {
btr_defragment_item_t* item = *iter;
iter = btr_defragment_wq.erase(iter);
delete item;
if (item->cond) {
mysql_cond_signal(item->cond);
}
}
mutex_exit(&btr_defragment_mutex);
mutex_free(&btr_defragment_mutex);
mysql_mutex_unlock(&btr_defragment_mutex);
mysql_mutex_destroy(&btr_defragment_mutex);
btr_defragment_active = false;
}
@ -174,7 +145,7 @@ bool
btr_defragment_find_index(
dict_index_t* index) /*!< Index to find. */
{
mutex_enter(&btr_defragment_mutex);
mysql_mutex_lock(&btr_defragment_mutex);
for (std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
iter != btr_defragment_wq.end();
++iter) {
@ -183,65 +154,47 @@ btr_defragment_find_index(
btr_cur_t* cursor = btr_pcur_get_btr_cur(pcur);
dict_index_t* idx = btr_cur_get_index(cursor);
if (index->id == idx->id) {
mutex_exit(&btr_defragment_mutex);
mysql_mutex_unlock(&btr_defragment_mutex);
return true;
}
}
mutex_exit(&btr_defragment_mutex);
mysql_mutex_unlock(&btr_defragment_mutex);
return false;
}
/******************************************************************//**
Query thread uses this function to add an index to btr_defragment_wq.
Return a pointer to os_event for the query thread to wait on if this is a
synchronized defragmentation. */
os_event_t
btr_defragment_add_index(
dict_index_t* index, /*!< index to be added */
dberr_t* err) /*!< out: error code */
/** Defragment an index.
@param pcur persistent cursor
@param thd current session, for checking thd_killed()
@return whether the operation was interrupted */
bool btr_defragment_add_index(btr_pcur_t *pcur, THD *thd)
{
mtr_t mtr;
*err = DB_SUCCESS;
mtr_start(&mtr);
buf_block_t* block = btr_root_block_get(index, RW_NO_LATCH, &mtr);
page_t* page = NULL;
if (block) {
page = buf_block_get_frame(block);
}
if (page == NULL && !index->is_readable()) {
mtr_commit(&mtr);
*err = DB_DECRYPTION_FAILED;
return NULL;
}
ut_ad(fil_page_index_page_check(page));
ut_ad(!page_has_siblings(page));
if (page_is_leaf(page)) {
// Index root is a leaf page, no need to defragment.
mtr_commit(&mtr);
return NULL;
}
btr_pcur_t* pcur = btr_pcur_create_for_mysql();
os_event_t event = os_event_create(0);
btr_pcur_open_at_index_side(true, index, BTR_SEARCH_LEAF, pcur,
true, 0, &mtr);
btr_pcur_move_to_next(pcur, &mtr);
btr_pcur_store_position(pcur, &mtr);
mtr_commit(&mtr);
dict_stats_empty_defrag_summary(index);
btr_defragment_item_t* item = new btr_defragment_item_t(pcur, event);
mutex_enter(&btr_defragment_mutex);
btr_defragment_wq.push_back(item);
if(btr_defragment_wq.size() == 1){
/* Kick off defragmentation work */
btr_defragment_start();
}
mutex_exit(&btr_defragment_mutex);
return event;
dict_stats_empty_defrag_summary(pcur->btr_cur.index);
mysql_cond_t cond;
mysql_cond_init(0, &cond, nullptr);
btr_defragment_item_t item(pcur, &cond);
mysql_mutex_lock(&btr_defragment_mutex);
btr_defragment_wq.push_back(&item);
if (btr_defragment_wq.size() == 1)
/* Kick off defragmentation work */
btr_defragment_start();
bool interrupted= false;
for (;;)
{
timespec abstime;
set_timespec(abstime, 1);
if (!mysql_cond_timedwait(&cond, &btr_defragment_mutex, &abstime))
break;
if (thd_killed(thd))
{
item.cond= nullptr;
interrupted= true;
break;
}
}
mysql_cond_destroy(&cond);
mysql_mutex_unlock(&btr_defragment_mutex);
return interrupted;
}
/******************************************************************//**
@ -252,95 +205,16 @@ void
btr_defragment_remove_table(
dict_table_t* table) /*!< Index to be removed. */
{
mutex_enter(&btr_defragment_mutex);
for (std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
iter != btr_defragment_wq.end();
++iter) {
btr_defragment_item_t* item = *iter;
btr_pcur_t* pcur = item->pcur;
btr_cur_t* cursor = btr_pcur_get_btr_cur(pcur);
dict_index_t* idx = btr_cur_get_index(cursor);
if (table->id == idx->table->id) {
item->removed = true;
}
}
mutex_exit(&btr_defragment_mutex);
}
/******************************************************************//**
Query thread uses this function to mark an index as removed in
btr_efragment_wq. */
void
btr_defragment_remove_index(
dict_index_t* index) /*!< Index to be removed. */
{
mutex_enter(&btr_defragment_mutex);
for (std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
iter != btr_defragment_wq.end();
++iter) {
btr_defragment_item_t* item = *iter;
btr_pcur_t* pcur = item->pcur;
btr_cur_t* cursor = btr_pcur_get_btr_cur(pcur);
dict_index_t* idx = btr_cur_get_index(cursor);
if (index->id == idx->id) {
item->removed = true;
item->event = NULL;
break;
}
}
mutex_exit(&btr_defragment_mutex);
}
/******************************************************************//**
Functions used by defragmentation thread: btr_defragment_xxx_item.
Defragmentation thread operates on the work *item*. It gets/removes
item from the work queue. */
/******************************************************************//**
Defragment thread uses this to remove an item from btr_defragment_wq.
When an item is removed from the work queue, all resources associated with it
are free as well. */
void
btr_defragment_remove_item(
btr_defragment_item_t* item) /*!< Item to be removed. */
{
mutex_enter(&btr_defragment_mutex);
for (std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
iter != btr_defragment_wq.end();
++iter) {
if (item == *iter) {
btr_defragment_wq.erase(iter);
delete item;
break;
}
}
mutex_exit(&btr_defragment_mutex);
}
/******************************************************************//**
Defragment thread uses this to get an item from btr_defragment_wq to work on.
The item is not removed from the work queue so query threads can still access
this item. We keep it this way so query threads can find and kill a
defragmentation even if that index is being worked on. Be aware that while you
work on this item you have no lock protection on it whatsoever. This is OK as
long as the query threads and defragment thread won't modify the same fields
without lock protection.
*/
btr_defragment_item_t*
btr_defragment_get_item()
{
if (btr_defragment_wq.empty()) {
return NULL;
//return nullptr;
}
mutex_enter(&btr_defragment_mutex);
std::list< btr_defragment_item_t* >::iterator iter = btr_defragment_wq.begin();
if (iter == btr_defragment_wq.end()) {
iter = btr_defragment_wq.begin();
}
btr_defragment_item_t* item = *iter;
iter++;
mutex_exit(&btr_defragment_mutex);
return item;
mysql_mutex_lock(&btr_defragment_mutex);
for (auto item : btr_defragment_wq)
{
if (item->cond && table == item->pcur->btr_cur.index->table)
{
mysql_cond_signal(item->cond);
item->cond= nullptr;
}
}
mysql_mutex_unlock(&btr_defragment_mutex);
}
/*********************************************************************//**
@ -572,7 +446,7 @@ the process, if any page becomes empty, that page will be removed from
the level list. Record locks, hash, and node pointers are updated after
page reorganization.
@return pointer to the last block processed, or NULL if reaching end of index */
UNIV_INTERN
static
buf_block_t*
btr_defragment_n_pages(
buf_block_t* block, /*!< in: starting block for defragmentation */
@ -739,34 +613,32 @@ The state (current item) is stored in function parameter.
*/
static void btr_defragment_chunk(void*)
{
defragment_chunk_state_t* state = &defragment_chunk_state;
btr_pcur_t* pcur;
btr_cur_t* cursor;
dict_index_t* index;
btr_defragment_item_t* item = nullptr;
mtr_t mtr;
buf_block_t* first_block;
buf_block_t* last_block;
mysql_mutex_lock(&btr_defragment_mutex);
while (srv_shutdown_state == SRV_SHUTDOWN_NONE) {
if (!state->m_item) {
state->m_item = btr_defragment_get_item();
}
/* If an index is marked as removed, we remove it from the work
queue. No other thread could be using this item at this point so
it's safe to remove now. */
while (state->m_item && state->m_item->removed) {
btr_defragment_remove_item(state->m_item);
state->m_item = btr_defragment_get_item();
if (!item) {
if (btr_defragment_wq.empty()) {
mysql_mutex_unlock(&btr_defragment_mutex);
return;
}
item = *btr_defragment_wq.begin();
ut_ad(item);
}
if (!state->m_item) {
/* Queue empty */
return;
if (!item->cond) {
processed:
btr_defragment_wq.remove(item);
item = nullptr;
continue;
}
pcur = state->m_item->pcur;
mysql_mutex_unlock(&btr_defragment_mutex);
ulonglong now = my_interval_timer();
ulonglong elapsed = now - state->m_item->last_processed;
ulonglong elapsed = now - item->last_processed;
if (elapsed < srv_defragment_interval) {
/* If we see an index again before the interval
@ -783,21 +655,19 @@ static void btr_defragment_chunk(void*)
}
log_free_check();
mtr_start(&mtr);
cursor = btr_pcur_get_btr_cur(pcur);
index = btr_cur_get_index(cursor);
dict_index_t *index = item->pcur->btr_cur.index;
index->set_modified(mtr);
/* To follow the latching order defined in WL#6326, acquire index->lock X-latch.
This entitles us to acquire page latches in any order for the index. */
mtr_x_lock_index(index, &mtr);
/* This will acquire index->lock SX-latch, which per WL#6363 is allowed
when we are already holding the X-latch. */
btr_pcur_restore_position(BTR_MODIFY_TREE, pcur, &mtr);
first_block = btr_cur_get_block(cursor);
last_block = btr_defragment_n_pages(first_block, index,
srv_defragment_n_pages,
&mtr);
if (last_block) {
btr_pcur_restore_position(BTR_MODIFY_TREE, item->pcur, &mtr);
buf_block_t* first_block = btr_pcur_get_block(item->pcur);
if (buf_block_t *last_block =
btr_defragment_n_pages(first_block, index,
srv_defragment_n_pages,
&mtr)) {
/* If we haven't reached the end of the index,
place the cursor on the last record of last page,
store the cursor position, and put back in queue. */
@ -806,18 +676,17 @@ static void btr_defragment_chunk(void*)
page_get_supremum_rec(last_page));
ut_a(page_rec_is_user_rec(rec));
page_cur_position(rec, last_block,
btr_cur_get_page_cur(cursor));
btr_pcur_store_position(pcur, &mtr);
btr_pcur_get_page_cur(item->pcur));
btr_pcur_store_position(item->pcur, &mtr);
mtr_commit(&mtr);
/* Update the last_processed time of this index. */
state->m_item->last_processed = now;
item->last_processed = now;
mysql_mutex_lock(&btr_defragment_mutex);
} else {
dberr_t err = DB_SUCCESS;
mtr_commit(&mtr);
/* Reaching the end of the index. */
dict_stats_empty_defrag_stats(index);
err = dict_stats_save_defrag_stats(index);
if (err != DB_SUCCESS) {
if (dberr_t err= dict_stats_save_defrag_stats(index)) {
ib::error() << "Saving defragmentation stats for table "
<< index->table->name
<< " index " << index->name()
@ -833,8 +702,13 @@ static void btr_defragment_chunk(void*)
}
}
btr_defragment_remove_item(state->m_item);
state->m_item = NULL;
mysql_mutex_lock(&btr_defragment_mutex);
if (item->cond) {
mysql_cond_signal(item->cond);
}
goto processed;
}
}
mysql_mutex_unlock(&btr_defragment_mutex);
}

4
storage/innobase/buf/buf0buf.cc

@ -2035,11 +2035,11 @@ withdraw_retry:
message_interval *= 2;
}
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
bool found = false;
trx_sys.trx_list.for_each(find_interesting_trx{
found, withdraw_started, current_time});
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
withdraw_started = current_time;
}

545
storage/innobase/fil/fil0crypt.cc

@ -55,17 +55,18 @@ UNIV_INTERN uint srv_n_fil_crypt_threads_started = 0;
/** At this age or older a space/page will be rotated */
UNIV_INTERN uint srv_fil_crypt_rotate_key_age;
/** Event to signal FROM the key rotation threads. */
static os_event_t fil_crypt_event;
/** Condition variable for srv_n_fil_crypt_threads_started */
static mysql_cond_t fil_crypt_cond;
/** Event to signal TO the key rotation threads. */
UNIV_INTERN os_event_t fil_crypt_threads_event;
/** Condition variable to to signal the key rotation threads */
static mysql_cond_t fil_crypt_threads_cond;
/** Event for waking up threads throttle. */
static os_event_t fil_crypt_throttle_sleep_event;
/** Condition variable for interrupting sleeptime_ms sleep at the end
of fil_crypt_rotate_page() */
static mysql_cond_t fil_crypt_throttle_sleep_cond;
/** Mutex for key rotation threads. */
UNIV_INTERN ib_mutex_t fil_crypt_threads_mutex;
/** Mutex for key rotation threads. Acquired before fil_system.mutex! */
static mysql_mutex_t fil_crypt_threads_mutex;
/** Variable ensuring only 1 thread at time does initial conversion */
static bool fil_crypt_start_converting = false;
@ -79,7 +80,18 @@ static uint n_fil_crypt_iops_allocated = 0;
/** Statistics variables */
static fil_crypt_stat_t crypt_stat;
static ib_mutex_t crypt_stat_mutex;
static mysql_mutex_t crypt_stat_mutex;
/** Wake up the encryption threads */
void fil_crypt_threads_signal(bool broadcast)
{
mysql_mutex_lock(&fil_crypt_threads_mutex);
if (broadcast)
mysql_cond_broadcast(&fil_crypt_threads_cond);
else
mysql_cond_signal(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
/***********************************************************************
Check if a key needs rotation given a key_state
@ -98,24 +110,19 @@ fil_crypt_needs_rotation(
/*********************************************************************
Init space crypt */
UNIV_INTERN
void
fil_space_crypt_init()
void fil_space_crypt_init()
{
fil_crypt_throttle_sleep_event = os_event_create(0);
mutex_create(LATCH_ID_FIL_CRYPT_STAT_MUTEX, &crypt_stat_mutex);
memset(&crypt_stat, 0, sizeof(crypt_stat));
mysql_cond_init(0, &fil_crypt_throttle_sleep_cond, nullptr);
mysql_mutex_init(0, &crypt_stat_mutex, nullptr);
memset(&crypt_stat, 0, sizeof crypt_stat);
}
/*********************************************************************
Cleanup space crypt */
UNIV_INTERN
void
fil_space_crypt_cleanup()
void fil_space_crypt_cleanup()
{
os_event_destroy(fil_crypt_throttle_sleep_event);
mutex_free(&crypt_stat_mutex);
mysql_cond_destroy(&fil_crypt_throttle_sleep_cond);
mysql_mutex_destroy(&crypt_stat_mutex);
}
/**
@ -154,11 +161,8 @@ fil_crypt_get_latest_key_version(
crypt_data->min_key_version,
key_version,
srv_fil_crypt_rotate_key_age)) {
/* Below event seen as NULL-pointer at startup
when new database was created and we create a
checkpoint. Only seen when debugging. */
if (fil_crypt_threads_inited) {
os_event_set(fil_crypt_threads_event);
fil_crypt_threads_signal();
}
}
}
@ -178,9 +182,9 @@ crypt_data_scheme_locker(
static_cast<fil_space_crypt_t*>(scheme);
if (exit) {
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
} else {
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&crypt_data->mutex);
}
}
@ -242,7 +246,7 @@ fil_space_merge_crypt_data(
fil_space_crypt_t* dst,
const fil_space_crypt_t* src)
{
mutex_enter(&dst->mutex);
mysql_mutex_lock(&dst->mutex);
/* validate that they are mergeable */
ut_a(src->type == CRYPT_SCHEME_UNENCRYPTED ||
@ -256,7 +260,7 @@ fil_space_merge_crypt_data(
dst->min_key_version = src->min_key_version;
dst->keyserver_requests += src->keyserver_requests;
mutex_exit(&dst->mutex);
mysql_mutex_unlock(&dst->mutex);
}
/** Initialize encryption parameters from a tablespace header page.
@ -318,10 +322,10 @@ fil_space_destroy_crypt_data(
if (crypt_data != NULL && (*crypt_data) != NULL) {
fil_space_crypt_t* c;
if (UNIV_LIKELY(fil_crypt_threads_inited)) {
mutex_enter(&fil_crypt_threads_mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
c = *crypt_data;
*crypt_data = NULL;
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
} else {
ut_ad(srv_read_only_mode || !srv_was_started);
c = *crypt_data;
@ -1008,21 +1012,22 @@ fil_crypt_read_crypt_data(fil_space_t* space)
@return true if a recheck of tablespace is needed by encryption thread. */
static bool fil_crypt_start_encrypting_space(fil_space_t* space)
{
mutex_enter(&fil_crypt_threads_mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
fil_space_crypt_t *crypt_data = space->crypt_data;
/* If space is not encrypted and encryption is not enabled, then
do not continue encrypting the space. */
if (!crypt_data && !srv_encrypt_tables) {
mutex_exit(&fil_crypt_threads_mutex);
func_exit:
mysql_mutex_unlock(&fil_crypt_threads_mutex);
return false;
}
const bool recheck = fil_crypt_start_converting;
if (recheck || crypt_data || space->is_stopping()) {
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
return recheck;
}
@ -1035,13 +1040,12 @@ static bool fil_crypt_start_encrypting_space(fil_space_t* space)
crypt_data = fil_space_create_crypt_data(
FIL_ENCRYPTION_DEFAULT, FIL_DEFAULT_ENCRYPTION_KEY);
if (crypt_data == NULL) {
mutex_exit(&fil_crypt_threads_mutex);
return false;
if (!crypt_data) {
goto func_exit;
}
fil_crypt_start_converting = true;
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
mtr_t mtr;
mtr.start();
@ -1080,25 +1084,25 @@ static bool fil_crypt_start_encrypting_space(fil_space_t* space)
while (buf_flush_dirty_pages(space->id));
/* 5 - publish crypt data */
mutex_enter(&fil_crypt_threads_mutex);
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
mysql_mutex_lock(&crypt_data->mutex);
crypt_data->type = CRYPT_SCHEME_1;
ut_a(crypt_data->rotate_state.active_threads == 1);
crypt_data->rotate_state.active_threads = 0;
crypt_data->rotate_state.starting = false;
fil_crypt_start_converting = false;
mutex_exit(&crypt_data->mutex);
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&crypt_data->mutex);
return false;
}
abort:
mtr.commit();
mutex_enter(&fil_crypt_threads_mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
fil_crypt_start_converting = false;
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
crypt_data->~fil_space_crypt_t();
ut_free(crypt_data);
@ -1131,6 +1135,7 @@ struct rotate_thread_t {
/** @return whether this thread should terminate */
bool should_shutdown() const {
mysql_mutex_assert_owner(&fil_crypt_threads_mutex);
switch (srv_shutdown_state) {
case SRV_SHUTDOWN_NONE:
return thread_no >= srv_n_fil_crypt_threads;
@ -1160,14 +1165,12 @@ fil_crypt_space_needs_rotation(
key_state_t* key_state,
bool* recheck)
{
fil_space_t* space = state->space;
mysql_mutex_assert_not_owner(&fil_crypt_threads_mutex);
/* Make sure that tablespace is normal tablespace */
if (space->purpose != FIL_TYPE_TABLESPACE) {
return false;
}
fil_space_t* space = state->space;
ut_ad(space->referenced());
ut_ad(space->purpose == FIL_TYPE_TABLESPACE);
fil_space_crypt_t *crypt_data = space->crypt_data;
@ -1192,7 +1195,9 @@ fil_crypt_space_needs_rotation(
return false;
}
mutex_enter(&crypt_data->mutex);
bool need_key_rotation = false;
mysql_mutex_lock(&crypt_data->mutex);
do {
/* prevent threads from starting to rotate space */
@ -1221,25 +1226,15 @@ fil_crypt_space_needs_rotation(
fil_crypt_get_key_state(key_state, crypt_data);
}
bool need_key_rotation = fil_crypt_needs_rotation(
need_key_rotation = fil_crypt_needs_rotation(
crypt_data,
crypt_data->min_key_version,
key_state->key_version,
key_state->rotate_key_age);
if (need_key_rotation == false) {
break;
}
mutex_exit(&crypt_data->mutex);
return true;
} while (0);
mutex_exit(&crypt_data->mutex);
return false;
mysql_mutex_unlock(&crypt_data->mutex);
return need_key_rotation;
}
/***********************************************************************
@ -1249,7 +1244,7 @@ static void
fil_crypt_update_total_stat(
rotate_thread_t *state)
{
mutex_enter(&crypt_stat_mutex);
mysql_mutex_lock(&crypt_stat_mutex);
crypt_stat.pages_read_from_cache +=
state->crypt_stat.pages_read_from_cache;
crypt_stat.pages_read_from_disk +=
@ -1260,7 +1255,7 @@ fil_crypt_update_total_stat(
crypt_stat.estimated_iops -= state->crypt_stat.estimated_iops;
// add new estimate
crypt_stat.estimated_iops += state->estimated_max_iops;
mutex_exit(&crypt_stat_mutex);
mysql_mutex_unlock(&crypt_stat_mutex);
// make new estimate "current" estimate
memset(&state->crypt_stat, 0, sizeof(state->crypt_stat));
@ -1273,11 +1268,9 @@ Allocate iops to thread from global setting,
used before starting to rotate a space.
@param[in,out] state Rotation state
@return true if allocation succeeded, false if failed */
static
bool
fil_crypt_alloc_iops(
rotate_thread_t *state)
static bool fil_crypt_alloc_iops(rotate_thread_t *state)
{
mysql_mutex_assert_owner(&fil_crypt_threads_mutex);
ut_ad(state->allocated_iops == 0);
/* We have not yet selected the space to rotate, thus
@ -1285,11 +1278,11 @@ fil_crypt_alloc_iops(
its status yet. */
uint max_iops = state->estimated_max_iops;
mutex_enter(&fil_crypt_threads_mutex);
if (n_fil_crypt_iops_allocated >= srv_n_fil_crypt_iops) {
/* this can happen when user decreases srv_fil_crypt_iops */
mutex_exit(&fil_crypt_threads_mutex);
wait:
mysql_cond_wait(&fil_crypt_threads_cond,
&fil_crypt_threads_mutex);
return false;
}
@ -1299,22 +1292,21 @@ fil_crypt_alloc_iops(
alloc = max_iops;
}
if (!alloc) {
goto wait;
}
n_fil_crypt_iops_allocated += alloc;
mutex_exit(&fil_crypt_threads_mutex);
state->allocated_iops = alloc;
return alloc > 0;
return true;
}
/***********************************************************************
Reallocate iops to thread,
used when inside a space
@param[in,out] state Rotation state */
static
void
fil_crypt_realloc_iops(
rotate_thread_t *state)
/**
Reallocate iops to thread when processing a tablespace
@param[in,out] state Rotation state
@return whether the thread should continue running */
static bool fil_crypt_realloc_iops(rotate_thread_t *state)
{
ut_a(state->allocated_iops > 0);
@ -1334,7 +1326,8 @@ fil_crypt_realloc_iops(
state->estimated_max_iops,
1000000 / avg_wait_time_us));
state->estimated_max_iops = uint(1000000 / avg_wait_time_us);
state->estimated_max_iops = std::max(
1U, uint(1000000 / avg_wait_time_us));
state->cnt_waited = 0;
state->sum_waited_us = 0;
} else {
@ -1346,89 +1339,59 @@ fil_crypt_realloc_iops(
/ (state->batch ? state->batch : 1)));
}
if (state->estimated_max_iops <= state->allocated_iops) {
/* return extra iops */
uint extra = state->allocated_iops - state->estimated_max_iops;
if (extra > 0) {
mutex_enter(&fil_crypt_threads_mutex);
if (n_fil_crypt_iops_allocated < extra) {
/* unknown bug!
* crash in debug
* keep n_fil_crypt_iops_allocated unchanged
* in release */
ut_ad(0);
extra = 0;
}
n_fil_crypt_iops_allocated -= extra;
state->allocated_iops -= extra;
if (state->allocated_iops == 0) {
/* no matter how slow io system seems to be
* never decrease allocated_iops to 0... */
state->allocated_iops ++;
n_fil_crypt_iops_allocated ++;
}
ut_ad(state->estimated_max_iops);
os_event_set(fil_crypt_threads_event);
mutex_exit(&fil_crypt_threads_mutex);
}
} else {
/* see if there are more to get */
mutex_enter(&fil_crypt_threads_mutex);
if (n_fil_crypt_iops_allocated < srv_n_fil_crypt_iops) {
/* there are extra iops free */
uint extra = srv_n_fil_crypt_iops -
n_fil_crypt_iops_allocated;
if (state->allocated_iops + extra >
state->estimated_max_iops) {
/* but don't alloc more than our max */
extra = state->estimated_max_iops -
state->allocated_iops;
}
n_fil_crypt_iops_allocated += extra;
state->allocated_iops += extra;
mysql_mutex_lock(&fil_crypt_threads_mutex);
DBUG_PRINT("ib_crypt",
("thr_no: %u increased iops from %u to %u.",
state->thread_no,
state->allocated_iops - extra,
state->allocated_iops));
if (state->should_shutdown()) {
mysql_mutex_unlock(&fil_crypt_threads_mutex);
return false;
}
if (state->allocated_iops > state->estimated_max_iops) {
/* release iops */
uint extra = state->allocated_iops - state->estimated_max_iops;
state->allocated_iops = state->estimated_max_iops;
ut_ad(n_fil_crypt_iops_allocated >= extra);
n_fil_crypt_iops_allocated -= extra;
mysql_cond_broadcast(&fil_crypt_threads_cond);
} else if (srv_n_fil_crypt_iops > n_fil_crypt_iops_allocated) {
/* there are extra iops free */
uint add = srv_n_fil_crypt_iops - n_fil_crypt_iops_allocated;
if (state->allocated_iops + add > state->estimated_max_iops) {
/* but don't alloc more than our max */
add= state->estimated_max_iops - state->allocated_iops;
}
mutex_exit(&fil_crypt_threads_mutex);
n_fil_crypt_iops_allocated += add;
state->allocated_iops += add;
DBUG_PRINT("ib_crypt",
("thr_no: %u increased iops from %u to %u.",
state->thread_no,
state->allocated_iops - add,
state->allocated_iops));
}
fil_crypt_update_total_stat(state);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
return true;
}
/***********************************************************************
Return allocated iops to global
/** Release allocated iops.
@param[in,out] state Rotation state */
static
void
fil_crypt_return_iops(
rotate_thread_t *state)
static void fil_crypt_return_iops(rotate_thread_t *state)
{
if (state->allocated_iops > 0) {
uint iops = state->allocated_iops;
mutex_enter(&fil_crypt_threads_mutex);
if (n_fil_crypt_iops_allocated < iops) {
/* unknown bug!
* crash in debug
* keep n_fil_crypt_iops_allocated unchanged
* in release */
ut_ad(0);
iops = 0;
}
mysql_mutex_assert_owner(&fil_crypt_threads_mutex);
n_fil_crypt_iops_allocated -= iops;
state->allocated_iops = 0;
os_event_set(fil_crypt_threads_event);
mutex_exit(&fil_crypt_threads_mutex);
}
if (uint iops = state->allocated_iops)
{
ut_ad(n_fil_crypt_iops_allocated >= iops);
n_fil_crypt_iops_allocated-= iops;
state->allocated_iops= 0;
mysql_cond_broadcast(&fil_crypt_threads_cond);
}
fil_crypt_update_total_stat(state);
fil_crypt_update_total_stat(state);
}
/** Return the next tablespace from rotation_list.
@ -1528,30 +1491,23 @@ inline fil_space_t *fil_space_t::next(fil_space_t *space, bool recheck,
@param[in,out] key_state Key state
@param[in,out] state Rotation state
@param[in,out] recheck recheck of the tablespace is needed or
still encryption thread does write page 0 */
still encryption thread does write page 0
@return whether the thread should keep running */
static bool fil_crypt_find_space_to_rotate(
key_state_t* key_state,
rotate_thread_t* state,
bool* recheck)
{
/* we need iops to start rotating */
while (!state->should_shutdown() && !fil_crypt_alloc_iops(state)) {
if (state->space && state->space->is_stopping()) {
state->space->release();
state->space = NULL;
}
os_event_reset(fil_crypt_threads_event);
os_event_wait_time(fil_crypt_threads_event, 100000);
}
if (state->should_shutdown()) {
if (state->space) {
state->space->release();
state->space = NULL;
do {
if (state->should_shutdown()) {
if (state->space) {
state->space->release();
state->space = NULL;
}
return false;
}
return false;
}
} while (!fil_crypt_alloc_iops(state));
if (state->first) {
state->first = false;
@ -1565,6 +1521,7 @@ static bool fil_crypt_find_space_to_rotate(
key_state->key_version != 0);
while (!state->should_shutdown() && state->space) {
mysql_mutex_unlock(&fil_crypt_threads_mutex);
/* If there is no crypt data and we have not yet read
page 0 for this tablespace, we need to read it before
we can continue. */
@ -1577,11 +1534,13 @@ static bool fil_crypt_find_space_to_rotate(
/* init state->min_key_version_found before
* starting on a space */
state->min_key_version_found = key_state->key_version;
mysql_mutex_lock(&fil_crypt_threads_mutex);
return true;
}
state->space = fil_space_t::next(state->space, *recheck,
key_state->key_version != 0);
key_state->key_version != 0);
mysql_mutex_lock(&fil_crypt_threads_mutex);
}
if (state->space) {
@ -1591,9 +1550,7 @@ static bool fil_crypt_find_space_to_rotate(
/* no work to do; release our allocation of I/O capacity */
fil_crypt_return_iops(state);
return false;
return true;
}
/***********************************************************************
@ -1609,7 +1566,7 @@ fil_crypt_start_rotate_space(
fil_space_crypt_t *crypt_data = state->space->crypt_data;
ut_ad(crypt_data);
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&crypt_data->mutex);
ut_ad(key_state->key_id == crypt_data->key_id);
if (crypt_data->rotate_state.active_threads == 0) {
@ -1642,7 +1599,7 @@ fil_crypt_start_rotate_space(
state->min_key_version_found =
crypt_data->rotate_state.min_key_version_found;
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
}
/***********************************************************************
@ -1668,7 +1625,7 @@ fil_crypt_find_page_to_rotate(
fil_space_crypt_t *crypt_data = space->crypt_data;
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&crypt_data->mutex);
ut_ad(key_state->key_id == crypt_data->key_id);
bool found = crypt_data->rotate_state.max_offset >=
@ -1687,7 +1644,7 @@ fil_crypt_find_page_to_rotate(
}
crypt_data->rotate_state.next_offset += uint32_t(batch);
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
return found;
}
@ -1881,9 +1838,12 @@ fil_crypt_rotate_page(
}
if (sleeptime_ms) {
os_event_reset(fil_crypt_throttle_sleep_event);
os_event_wait_time(fil_crypt_throttle_sleep_event,
1000 * sleeptime_ms);
mysql_mutex_lock(&fil_crypt_threads_mutex);
timespec abstime;
set_timespec_nsec(abstime, 1000000ULL * sleeptime_ms);
mysql_cond_timedwait(&fil_crypt_throttle_sleep_cond,
&fil_crypt_threads_mutex, &abstime);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
}
@ -1998,10 +1958,10 @@ static void fil_crypt_complete_rotate_space(rotate_thread_t* state)
ut_ad(crypt_data);
ut_ad(state->space->referenced());
mysql_mutex_lock(&crypt_data->mutex);
/* Space might already be dropped */
if (!state->space->is_stopping()) {
mutex_enter(&crypt_data->mutex);
/**
* Update crypt data state with state from thread
*/
@ -2038,95 +1998,76 @@ static void fil_crypt_complete_rotate_space(rotate_thread_t* state)
crypt_data->rotate_state.flushing = true;
crypt_data->min_key_version =
crypt_data->rotate_state.min_key_version_found;
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
fil_crypt_flush_space(state);
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&crypt_data->mutex);
crypt_data->rotate_state.flushing = false;
mutex_exit(&crypt_data->mutex);
} else {
mutex_exit(&crypt_data->mutex);
}
} else {
mutex_enter(&crypt_data->mutex);
ut_a(crypt_data->rotate_state.active_threads > 0);
crypt_data->rotate_state.active_threads--;
mutex_exit(&crypt_data->mutex);
}
mysql_mutex_unlock(&crypt_data->mutex);
}
/*********************************************************************//**
A thread which monitors global key state and rotates tablespaces accordingly
@return a dummy parameter */
extern "C" UNIV_INTERN
os_thread_ret_t
DECLARE_THREAD(fil_crypt_thread)(void*)
static os_thread_ret_t DECLARE_THREAD(fil_crypt_thread)(void*)
{
mutex_enter(&fil_crypt_threads_mutex);
uint thread_no = srv_n_fil_crypt_threads_started;
srv_n_fil_crypt_threads_started++;
os_event_set(fil_crypt_event); /* signal that we started */
mutex_exit(&fil_crypt_threads_mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
rotate_thread_t thr(srv_n_fil_crypt_threads_started++);
mysql_cond_signal(&fil_crypt_cond); /* signal that we started */
/* state of this thread */
rotate_thread_t thr(thread_no);
/* if we find a space that is starting, skip over it and recheck it later */
bool recheck = false;
while (!thr.should_shutdown()) {
key_state_t new_state;
while (!thr.should_shutdown()) {
if (!thr.should_shutdown()) {
/* if we find a tablespace that is starting, skip over it
and recheck it later */
bool recheck = false;
wait_for_work:
if (!recheck && !thr.should_shutdown()) {
/* wait for key state changes
* i.e either new key version of change or
* new rotate_key_age */
os_event_reset(fil_crypt_threads_event);
if (os_event_wait_time(fil_crypt_threads_event, 1000000) == 0) {
break;
}
if (recheck) {
/* check recheck here, after sleep, so
* that we don't busy loop while when one thread is starting
* a space*/
break;
}
mysql_cond_wait(&fil_crypt_threads_cond,
&fil_crypt_threads_mutex);
}
recheck = false;
thr.first = true; // restart from first tablespace
key_state_t new_state;
/* iterate all spaces searching for those needing rotation */
while (!thr.should_shutdown() &&
fil_crypt_find_space_to_rotate(&new_state, &thr, &recheck)) {
while (fil_crypt_find_space_to_rotate(&new_state, &thr,
&recheck)) {
if (!thr.space) {
goto wait_for_work;
}
/* we found a space to rotate */
mysql_mutex_unlock(&fil_crypt_threads_mutex);
fil_crypt_start_rotate_space(&new_state, &thr);
/* iterate all pages (cooperativly with other threads) */
while (!thr.should_shutdown() &&
fil_crypt_find_page_to_rotate(&new_state, &thr)) {
if (!thr.space->is_stopping()) {
/* rotate a (set) of pages */
fil_crypt_rotate_pages(&new_state, &thr);
}
while (fil_crypt_find_page_to_rotate(&new_state, &thr)) {
/* If space is marked as stopping, release
space and stop rotation. */
if (thr.space->is_stopping()) {
fil_crypt_complete_rotate_space(&thr);
thr.space->release();
thr.space = NULL;
thr.space = nullptr;
break;
}
fil_crypt_rotate_pages(&new_state, &thr);
/* realloc iops */
fil_crypt_realloc_iops(&thr);
if (!fil_crypt_realloc_iops(&thr)) {
break;
}
}
/* complete rotation */
@ -2137,24 +2078,21 @@ DECLARE_THREAD(fil_crypt_thread)(void*)
/* force key state refresh */
new_state.key_id = 0;
/* return iops */
mysql_mutex_lock(&fil_crypt_threads_mutex);
/* release iops */
fil_crypt_return_iops(&thr);
}
}
/* return iops if shutting down */
fil_crypt_return_iops(&thr);
/* release current space if shutting down */
if (thr.space) {
thr.space->release();
thr.space = NULL;
if (thr.space) {
thr.space->release();
thr.space = nullptr;
}
}
mutex_enter(&fil_crypt_threads_mutex);
fil_crypt_return_iops(&thr);
srv_n_fil_crypt_threads_started--;
os_event_set(fil_crypt_event); /* signal that we stopped */
mutex_exit(&fil_crypt_threads_mutex);
mysql_cond_signal(&fil_crypt_cond); /* signal that we stopped */
mysql_mutex_unlock(&fil_crypt_threads_mutex);
/* We count the number of threads in os_thread_exit(). A created
thread should always use that to exit and not use return() to exit. */
@ -2176,7 +2114,7 @@ fil_crypt_set_thread_cnt(
fil_crypt_threads_init();
}
mutex_enter(&fil_crypt_threads_mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
if (new_cnt > srv_n_fil_crypt_threads) {
uint add = new_cnt - srv_n_fil_crypt_threads;
@ -2189,21 +2127,16 @@ fil_crypt_set_thread_cnt(
}
} else if (new_cnt < srv_n_fil_crypt_threads) {
srv_n_fil_crypt_threads = new_cnt;
os_event_set(fil_crypt_threads_event);
}
mutex_exit(&fil_crypt_threads_mutex);
mysql_cond_broadcast(&fil_crypt_threads_cond);
while(srv_n_fil_crypt_threads_started != srv_n_fil_crypt_threads) {
os_event_reset(fil_crypt_event);
os_event_wait_time(fil_crypt_event, 100000);
while (srv_n_fil_crypt_threads_started != srv_n_fil_crypt_threads) {
mysql_cond_wait(&fil_crypt_cond, &fil_crypt_threads_mutex);
}
/* Send a message to encryption threads that there could be
something to do. */
if (srv_n_fil_crypt_threads) {
os_event_set(fil_crypt_threads_event);
}
mysql_cond_broadcast(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
/** Initialize the tablespace rotation_list
@ -2257,30 +2190,27 @@ next:
/*********************************************************************
Adjust max key age
@param[in] val New max key age */
UNIV_INTERN
void
fil_crypt_set_rotate_key_age(
uint val)
void fil_crypt_set_rotate_key_age(uint val)
{
mutex_enter(&fil_system.mutex);
srv_fil_crypt_rotate_key_age = val;
if (val == 0) {
fil_crypt_rotation_list_fill();
}
mutex_exit(&fil_system.mutex);
os_event_set(fil_crypt_threads_event);
mysql_mutex_lock(&fil_crypt_threads_mutex);
mutex_enter(&fil_system.mutex);
srv_fil_crypt_rotate_key_age= val;
if (val == 0)
fil_crypt_rotation_list_fill();
mutex_exit(&fil_system.mutex);
mysql_cond_broadcast(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
/*********************************************************************
Adjust rotation iops
@param[in] val New max roation iops */
UNIV_INTERN
void
fil_crypt_set_rotation_iops(
uint val)
void fil_crypt_set_rotation_iops(uint val)
{
srv_n_fil_crypt_iops = val;
os_event_set(fil_crypt_threads_event);
mysql_mutex_lock(&fil_crypt_threads_mutex);
srv_n_fil_crypt_iops= val;
mysql_cond_broadcast(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
/*********************************************************************
@ -2288,17 +2218,18 @@ Adjust encrypt tables
@param[in] val New setting for innodb-encrypt-tables */
void fil_crypt_set_encrypt_tables(ulong val)
{
mutex_enter(&fil_system.mutex);
mysql_mutex_lock(&fil_crypt_threads_mutex);
srv_encrypt_tables = val;
mutex_enter(&fil_system.mutex);
srv_encrypt_tables= val;
if (srv_fil_crypt_rotate_key_age == 0) {
fil_crypt_rotation_list_fill();
}
if (srv_fil_crypt_rotate_key_age == 0)
fil_crypt_rotation_list_fill();
mutex_exit(&fil_system.mutex);
mutex_exit(&fil_system.mutex);
os_event_set(fil_crypt_threads_event);
mysql_cond_broadcast(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
}
/*********************************************************************
@ -2308,11 +2239,9 @@ void
fil_crypt_threads_init()
{
if (!fil_crypt_threads_inited) {
fil_crypt_event = os_event_create(0);
fil_crypt_threads_event = os_event_create(0);
mutex_create(LATCH_ID_FIL_CRYPT_THREADS_MUTEX,
&fil_crypt_threads_mutex);
mysql_cond_init(0, &fil_crypt_cond, nullptr);
mysql_cond_init(0, &fil_crypt_threads_cond, nullptr);
mysql_mutex_init(0, &fil_crypt_threads_mutex, nullptr);
uint cnt = srv_n_fil_crypt_threads;
srv_n_fil_crypt_threads = 0;
fil_crypt_threads_inited = true;
@ -2330,9 +2259,9 @@ fil_crypt_threads_cleanup()
return;
}
ut_a(!srv_n_fil_crypt_threads_started);
os_event_destroy(fil_crypt_event);
os_event_destroy(fil_crypt_threads_event);
mutex_free(&fil_crypt_threads_mutex);
mysql_cond_destroy(&fil_crypt_cond);
mysql_cond_destroy(&fil_crypt_threads_cond);
mysql_mutex_destroy(&fil_crypt_threads_mutex);
fil_crypt_threads_inited = false;
}
@ -2351,48 +2280,44 @@ fil_space_crypt_close_tablespace(
return;
}
mutex_enter(&fil_crypt_threads_mutex);
time_t start = time(0);
time_t last = start;
mutex_enter(&crypt_data->mutex);
mutex_exit(&fil_crypt_threads_mutex);
ulint cnt = crypt_data->rotate_state.active_threads;
bool flushing = crypt_data->rotate_state.flushing;
mysql_mutex_lock(&crypt_data->mutex);
while (cnt > 0 || flushing) {
mutex_exit(&crypt_data->mutex);
while (crypt_data->rotate_state.active_threads
|| crypt_data->rotate_state.flushing) {
mysql_mutex_unlock(&crypt_data->mutex);
/* release dict mutex so that scrub threads can release their
* table references */
dict_mutex_exit_for_mysql();
/* wakeup throttle (all) sleepers */
os_event_set(fil_crypt_throttle_sleep_event);
os_event_set(fil_crypt_threads_event);
mysql_mutex_lock(&fil_crypt_threads_mutex);
mysql_cond_broadcast(&fil_crypt_throttle_sleep_cond);
mysql_cond_broadcast(&fil_crypt_threads_cond);
mysql_mutex_unlock(&fil_crypt_threads_mutex);
os_thread_sleep(20000);
dict_mutex_enter_for_mysql();
mutex_enter(&crypt_data->mutex);
cnt = crypt_data->rotate_state.active_threads;
flushing = crypt_data->rotate_state.flushing;
mysql_mutex_lock(&crypt_data->mutex);
time_t now = time(0);
if (now >= last + 30) {
if (UNIV_UNLIKELY(now >= last + 30)) {
ib::warn() << "Waited "
<< now - start
<< " seconds to drop space: "
<< space->name << " ("
<< space->id << ") active threads "
<< cnt << "flushing="
<< flushing << ".";
<< crypt_data->rotate_state.active_threads
<< "flushing="
<< crypt_data->rotate_state.flushing << ".";
last = now;
}
}
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
}
/*********************************************************************
@ -2420,7 +2345,7 @@ fil_space_crypt_get_status(
if (fil_space_crypt_t* crypt_data = space->crypt_data) {
status->space = space->id;
mutex_enter(&crypt_data->mutex);
mysql_mutex_lock(&crypt_data->mutex);
status->scheme = crypt_data->type;
status->keyserver_requests = crypt_data->keyserver_requests;
status->min_key_version = crypt_data->min_key_version;
@ -2437,7 +2362,7 @@ fil_space_crypt_get_status(
crypt_data->rotate_state.max_offset;
}
mutex_exit(&crypt_data->mutex);
mysql_mutex_unlock(&crypt_data->mutex);
if (srv_encrypt_tables || crypt_data->min_key_version) {
status->current_key_version =
@ -2454,9 +2379,9 @@ void
fil_crypt_total_stat(
fil_crypt_stat_t *stat)
{
mutex_enter(&crypt_stat_mutex);
mysql_mutex_lock(&crypt_stat_mutex);
*stat = crypt_stat;
mutex_exit(&crypt_stat_mutex);
mysql_mutex_unlock(&crypt_stat_mutex);
}
#endif /* UNIV_INNOCHECKSUM */

28
storage/innobase/fil/fil0fil.cc

@ -46,7 +46,6 @@ Created 10/25/1995 Heikki Tuuri
#include "trx0purge.h"
#include "buf0lru.h"
#include "ibuf0ibuf.h"
#include "os0event.h"
#include "sync0sync.h"
#include "buf0flu.h"
#ifdef UNIV_LINUX
@ -1009,20 +1008,21 @@ fil_space_t *fil_space_t::create(const char *name, ulint id, ulint flags,
fil_system.max_assigned_id = id;
}
/* Inform key rotation that there could be something
to do */
if (purpose == FIL_TYPE_TABLESPACE
&& !srv_fil_crypt_rotate_key_age && fil_crypt_threads_event &&
(mode == FIL_ENCRYPTION_ON || mode == FIL_ENCRYPTION_OFF
|| srv_encrypt_tables)) {
/* Key rotation is not enabled, need to inform background
encryption threads. */
const bool rotate= purpose == FIL_TYPE_TABLESPACE
&& (mode == FIL_ENCRYPTION_ON || mode == FIL_ENCRYPTION_OFF
|| srv_encrypt_tables)
&& !srv_fil_crypt_rotate_key_age
&& srv_n_fil_crypt_threads_started;
if (rotate) {
fil_system.rotation_list.push_back(*space);
space->is_in_rotation_list = true;
mutex_exit(&fil_system.mutex);
os_event_set(fil_crypt_threads_event);
} else {
mutex_exit(&fil_system.mutex);
}
mutex_exit(&fil_system.mutex);
if (rotate) {
fil_crypt_threads_signal();
}
return(space);
@ -3365,7 +3365,7 @@ write_completed:
if (dberr_t err= buf_page_read_complete(request.bpage, *request.node))
{
if (recv_recovery_is_on() && !srv_force_recovery)
recv_sys.found_corrupt_fs= true;
recv_sys.set_corrupt_fs();
ib::error() << "Failed to read page " << id.page_no()
<< " from file '" << request.node->name << "': " << err;

100
storage/innobase/fts/fts0fts.cc

@ -286,9 +286,9 @@ fts_cache_destroy(fts_cache_t* cache)
{
mysql_mutex_destroy(&cache->lock);
mysql_mutex_destroy(&cache->init_lock);
mutex_free(&cache->deleted_lock);
mutex_free(&cache->doc_id_lock);
os_event_destroy(cache->sync->event);
mysql_mutex_destroy(&cache->deleted_lock);
mysql_mutex_destroy(&cache->doc_id_lock);
mysql_cond_destroy(&cache->sync->cond);
if (cache->stopword_info.cached_stopword) {
rbt_free(cache->stopword_info.cached_stopword);
@ -580,10 +580,10 @@ fts_cache_init(
cache->total_size = 0;
mutex_enter((ib_mutex_t*) &cache->deleted_lock);
mysql_mutex_lock(&cache->deleted_lock);
cache->deleted_doc_ids = ib_vector_create(
cache->sync_heap, sizeof(doc_id_t), 4);
mutex_exit((ib_mutex_t*) &cache->deleted_lock);
mysql_mutex_unlock(&cache->deleted_lock);
/* Reset the cache data for all the FTS indexes. */
for (i = 0; i < ib_vector_size(cache->indexes); ++i) {
@ -615,10 +615,8 @@ fts_cache_create(
mysql_mutex_init(fts_cache_mutex_key, &cache->lock, nullptr);
mysql_mutex_init(fts_cache_init_mutex_key, &cache->init_lock, nullptr);
mutex_create(LATCH_ID_FTS_DELETE, &cache->deleted_lock);
mutex_create(LATCH_ID_FTS_DOC_ID, &cache->doc_id_lock);
mysql_mutex_init(fts_delete_mutex_key, &cache->deleted_lock, nullptr);
mysql_mutex_init(fts_doc_id_mutex_key, &cache->doc_id_lock, nullptr);
/* This is the heap used to create the cache itself. */
cache->self_heap = ib_heap_allocator_create(heap);
@ -631,7 +629,7 @@ fts_cache_create(
mem_heap_zalloc(heap, sizeof(fts_sync_t)));
cache->sync->table = table;
cache->sync->event = os_event_create(0);
mysql_cond_init(0, &cache->sync->cond, nullptr);
/* Create the index cache vector that will hold the inverted indexes. */
cache->indexes = ib_vector_create(
@ -1090,9 +1088,9 @@ fts_cache_clear(
cache->total_size = 0;
mutex_enter((ib_mutex_t*) &cache->deleted_lock);
mysql_mutex_lock(&cache->deleted_lock);
cache->deleted_doc_ids = NULL;
mutex_exit((ib_mutex_t*) &cache->deleted_lock);
mysql_mutex_unlock(&cache->deleted_lock);
mem_heap_free(static_cast<mem_heap_t*>(cache->sync_heap->arg));
cache->sync_heap->arg = NULL;
@ -1232,11 +1230,11 @@ fts_cache_node_add_positions(
byte* ptr_start;
ulint doc_id_delta;
#ifdef UNIV_DEBUG
#ifdef SAFE_MUTEX
if (cache) {
mysql_mutex_assert_owner(&cache->lock);
}
#endif /* UNIV_DEBUG */
#endif /* SAFE_MUTEX */
ut_ad(doc_id >= node->last_doc_id);
@ -2556,9 +2554,9 @@ fts_get_next_doc_id(
}
DEBUG_SYNC_C("get_next_FTS_DOC_ID");
mutex_enter(&cache->doc_id_lock);
mysql_mutex_lock(&cache->doc_id_lock);
*doc_id = cache->next_doc_id++;
mutex_exit(&cache->doc_id_lock);
mysql_mutex_unlock(&cache->doc_id_lock);
return(DB_SUCCESS);
}
@ -2653,13 +2651,13 @@ retry:
cache->synced_doc_id = ut_max(cmp_doc_id, *doc_id);
}
mutex_enter(&cache->doc_id_lock);
mysql_mutex_lock(&cache->doc_id_lock);
/* For each sync operation, we will add next_doc_id by 1,
so to mark a sync operation */
if (cache->next_doc_id < cache->synced_doc_id + 1) {
cache->next_doc_id = cache->synced_doc_id + 1;
}
mutex_exit(&cache->doc_id_lock);
mysql_mutex_unlock(&cache->doc_id_lock);
if (cmp_doc_id > *doc_id) {
error = fts_update_sync_doc_id(
@ -2801,9 +2799,9 @@ fts_add(
fts_add_doc_by_id(ftt, doc_id, row->fts_indexes);
mutex_enter(&table->fts->cache->deleted_lock);
mysql_mutex_lock(&table->fts->cache->deleted_lock);
++table->fts->cache->added;
mutex_exit(&table->fts->cache->deleted_lock);
mysql_mutex_unlock(&table->fts->cache->deleted_lock);
if (!DICT_TF2_FLAG_IS_SET(table, DICT_TF2_FTS_HAS_DOC_ID)
&& doc_id >= table->fts->cache->next_doc_id) {
@ -2851,7 +2849,7 @@ fts_delete(
is re-established and sync-ed */
if (table->fts->added_synced
&& doc_id > cache->synced_doc_id) {
mutex_enter(&table->fts->cache->deleted_lock);
mysql_mutex_lock(&table->fts->cache->deleted_lock);
/* The Doc ID could belong to those left in
ADDED table from last crash. So need to check
@ -2862,7 +2860,7 @@ fts_delete(
--table->fts->cache->added;
}
mutex_exit(&table->fts->cache->deleted_lock);
mysql_mutex_unlock(&table->fts->cache->deleted_lock);
/* Only if the row was really deleted. */
ut_a(row->state == FTS_DELETE || row->state == FTS_MODIFY);
@ -2896,11 +2894,11 @@ fts_delete(
/* Increment the total deleted count, this is used to calculate the
number of documents indexed. */
if (error == DB_SUCCESS) {
mutex_enter(&table->fts->cache->deleted_lock);
mysql_mutex_lock(&table->fts->cache->deleted_lock);
++table->fts->cache->deleted;
mutex_exit(&table->fts->cache->deleted_lock);
mysql_mutex_unlock(&table->fts->cache->deleted_lock);
}
return(error);
@ -3515,7 +3513,10 @@ fts_add_doc_by_id(
DBUG_EXECUTE_IF(
"fts_instrument_sync",
fts_optimize_request_sync_table(table);
os_event_wait(cache->sync->event);
mysql_mutex_lock(&cache->lock);
mysql_cond_wait(&cache->sync->cond,
&cache->lock);
mysql_mutex_unlock(&cache->lock);
);
DBUG_EXECUTE_IF(
@ -4233,16 +4234,14 @@ fts_sync(
/* Check if cache is being synced.
Note: we release cache lock in fts_sync_write_words() to
avoid long wait for the lock by other threads. */
while (sync->in_progress) {
mysql_mutex_unlock(&cache->lock);
if (wait) {
os_event_wait(sync->event);
} else {
if (sync->in_progress) {
if (!wait) {
mysql_mutex_unlock(&cache->lock);
return(DB_SUCCESS);
}
mysql_mutex_lock(&cache->lock);
do {
mysql_cond_wait(&sync->cond, &cache->lock);
} while (sync->in_progress);
}
sync->unlock_cache = unlock_cache;
@ -4309,22 +4308,22 @@ end_sync:
}
mysql_mutex_lock(&cache->lock);
ut_ad(sync->in_progress);
sync->interrupted = false;
sync->in_progress = false;
os_event_set(sync->event);
mysql_cond_signal(&sync->cond);
mysql_mutex_unlock(&cache->lock);
/* We need to check whether an optimize is required, for that
we make copies of the two variables that control the trigger. These
variables can change behind our back and we don't want to hold the
lock for longer than is needed. */
mutex_enter(&cache->deleted_lock);
mysql_mutex_lock(&cache->deleted_lock);
cache->added = 0;
cache->deleted = 0;
mutex_exit(&cache->deleted_lock);
mysql_mutex_unlock(&cache->deleted_lock);
return(error);
}
@ -5151,27 +5150,20 @@ Append deleted doc ids to vector. */
void
fts_cache_append_deleted_doc_ids(
/*=============================*/
const fts_cache_t* cache, /*!< in: cache to use */
fts_cache_t* cache, /*!< in: cache to use */
ib_vector_t* vector) /*!< in: append to this vector */
{
mutex_enter(const_cast<ib_mutex_t*>(&cache->deleted_lock));
if (cache->deleted_doc_ids == NULL) {
mutex_exit((ib_mutex_t*) &cache->deleted_lock);
return;
}
mysql_mutex_lock(&cache->deleted_lock);
for (ulint i = 0; i < ib_vector_size(cache->deleted_doc_ids); ++i) {
doc_id_t* update;
update = static_cast<doc_id_t*>(
ib_vector_get(cache->deleted_doc_ids, i));
ib_vector_push(vector, &update);
}
if (cache->deleted_doc_ids)
for (ulint i= 0; i < ib_vector_size(cache->deleted_doc_ids); ++i)
{
doc_id_t *update= static_cast<doc_id_t*>(
ib_vector_get(cache->deleted_doc_ids, i));
ib_vector_push(vector, &update);
}
mutex_exit((ib_mutex_t*) &cache->deleted_lock);
mysql_mutex_unlock(&cache->deleted_lock);
}
/*********************************************************************//**

187
storage/innobase/fts/fts0opt.cc

@ -58,8 +58,9 @@ static const ulint FTS_OPTIMIZE_INTERVAL_IN_SECS = 300;
/** Server is shutting down, so does we exiting the optimize thread */
static bool fts_opt_start_shutdown = false;
/** Event to wait for shutdown of the optimize thread */
static os_event_t fts_opt_shutdown_event = NULL;
/** Condition variable for shutting down the optimize thread.
Protected by fts_optimize_wq->mutex. */
static mysql_cond_t fts_opt_shutdown_cond;
/** Initial size of nodes in fts_word_t. */
static const ulint FTS_WORD_NODES_INIT_SIZE = 64;
@ -195,12 +196,12 @@ struct fts_slot_t {
};
/** A table remove message for the FTS optimize thread. */
struct fts_msg_del_t {
dict_table_t* table; /*!< The table to remove */
os_event_t event; /*!< Event to synchronize acknowledgement
of receipt and processing of the
this message by the consumer */
struct fts_msg_del_t
{
/** the table to remove */
dict_table_t *table;
/** condition variable to signal message consumption */
mysql_cond_t *cond;
};
/** The FTS optimize message work queue message type. */
@ -2530,9 +2531,9 @@ fts_optimize_create_msg(
}
/** Add message to wqueue, signal thread pool*/
static void add_msg(fts_msg_t *msg, bool wq_locked= false)
static void add_msg(fts_msg_t *msg)
{
ib_wqueue_add(fts_optimize_wq, msg, msg->heap, wq_locked);
ib_wqueue_add(fts_optimize_wq, msg, msg->heap, true);
srv_thread_pool->submit_task(&task);
}
@ -2560,13 +2561,13 @@ void fts_optimize_add_table(dict_table_t* table)
msg = fts_optimize_create_msg(FTS_MSG_ADD_TABLE, table);
mutex_enter(&fts_optimize_wq->mutex);
mysql_mutex_lock(&fts_optimize_wq->mutex);
add_msg(msg, true);
add_msg(msg);
table->fts->in_queue = true;
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
}
/**********************************************************************//**
@ -2577,63 +2578,35 @@ fts_optimize_remove_table(
/*======================*/
dict_table_t* table) /*!< in: table to remove */
{
fts_msg_t* msg;
os_event_t event;
fts_msg_del_t* remove;
/* if the optimize system not yet initialized, return */
if (!fts_optimize_wq) {
return;
}
/* FTS optimizer thread is already exited */
if (fts_opt_start_shutdown) {
ib::info() << "Try to remove table " << table->name
<< " after FTS optimize thread exiting.";
/* If the table can't be removed then wait till
fts optimize thread shuts down */
while (fts_optimize_wq) {
os_thread_sleep(10000);
}
return;
}
mutex_enter(&fts_optimize_wq->mutex);
if (!table->fts->in_queue) {
mutex_exit(&fts_optimize_wq->mutex);
return;
}
msg = fts_optimize_create_msg(FTS_MSG_DEL_TABLE, NULL);
/* We will wait on this event until signalled by the consumer. */
event = os_event_create(0);
remove = static_cast<fts_msg_del_t*>(
mem_heap_alloc(msg->heap, sizeof(*remove)));
remove->table = table;
remove->event = event;
msg->ptr = remove;
ut_ad(!mutex_own(&dict_sys.mutex));
add_msg(msg, true);
if (!fts_optimize_wq)
return;
mutex_exit(&fts_optimize_wq->mutex);
if (fts_opt_start_shutdown)
{
ib::info() << "Try to remove table " << table->name
<< " after FTS optimize thread exiting.";
while (fts_optimize_wq)
os_thread_sleep(10000);
return;
}
os_event_wait(event);
mysql_mutex_lock(&fts_optimize_wq->mutex);
os_event_destroy(event);
if (table->fts->in_queue)
{
ut_ad(!mutex_own(&dict_sys.mutex));
fts_msg_t *msg= fts_optimize_create_msg(FTS_MSG_DEL_TABLE, nullptr);
mysql_cond_t cond;
mysql_cond_init(0, &cond, nullptr);
msg->ptr= new(mem_heap_alloc(msg->heap, sizeof(fts_msg_del_t)))
fts_msg_del_t{table, &cond};
add_msg(msg);
mysql_cond_wait(&cond, &fts_optimize_wq->mutex);
mysql_cond_destroy(&cond);
ut_ad(!table->fts->in_queue);
}
#ifdef UNIV_DEBUG
if (!fts_opt_start_shutdown) {
mutex_enter(&fts_optimize_wq->mutex);
ut_ad(!table->fts->in_queue);
mutex_exit(&fts_optimize_wq->mutex);
}
#endif /* UNIV_DEBUG */
mysql_mutex_unlock(&fts_optimize_wq->mutex);
}
/** Send sync fts cache for the table.
@ -2647,22 +2620,21 @@ fts_optimize_request_sync_table(
return;
}
mysql_mutex_lock(&fts_optimize_wq->mutex);
/* FTS optimizer thread is already exited */
if (fts_opt_start_shutdown) {
ib::info() << "Try to sync table " << table->name
<< " after FTS optimize thread exiting.";
mysql_mutex_unlock(&fts_optimize_wq->mutex);
return;
}
fts_msg_t* msg = fts_optimize_create_msg(FTS_MSG_SYNC_TABLE, table);
mutex_enter(&fts_optimize_wq->mutex);
add_msg(msg, true);
add_msg(fts_optimize_create_msg(FTS_MSG_SYNC_TABLE, table));
table->fts->sync_message = true;
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
}
/** Add a table to fts_slots if it doesn't already exist. */
@ -2697,9 +2669,10 @@ static bool fts_optimize_new_table(dict_table_t* table)
}
/** Remove a table from fts_slots if it exists.
@param[in,out] table table to be removed from fts_slots */
static bool fts_optimize_del_table(const dict_table_t* table)
@param remove table to be removed from fts_slots */
static bool fts_optimize_del_table(fts_msg_del_t *remove)
{
const dict_table_t* table = remove->table;
ut_ad(table);
for (ulint i = 0; i < ib_vector_size(fts_slots); ++i) {
fts_slot_t* slot;
@ -2712,14 +2685,18 @@ static bool fts_optimize_del_table(const dict_table_t* table)
<< table->name;
}
mutex_enter(&fts_optimize_wq->mutex);
slot->table->fts->in_queue = false;
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_lock(&fts_optimize_wq->mutex);
table->fts->in_queue = false;
mysql_cond_signal(remove->cond);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
slot->table = NULL;
return true;
}
}
mysql_mutex_lock(&fts_optimize_wq->mutex);
mysql_cond_signal(remove->cond);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
return false;
}
@ -2804,9 +2781,9 @@ static void fts_optimize_sync_table(dict_table_t *table,
fts_sync_table(sync_table, false);
if (process_message)
{
mutex_enter(&fts_optimize_wq->mutex);
mysql_mutex_lock(&fts_optimize_wq->mutex);
sync_table->fts->sync_message = false;
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
}
}
@ -2887,14 +2864,9 @@ static void fts_optimize_callback(void *)
case FTS_MSG_DEL_TABLE:
if (fts_optimize_del_table(
static_cast<fts_msg_del_t*>(
msg->ptr)->table)) {
msg->ptr))) {
--n_tables;
}
/* Signal the producer that we have
removed the table. */
os_event_set(
((fts_msg_del_t*) msg->ptr)->event);
break;
case FTS_MSG_SYNC_TABLE:
@ -2930,15 +2902,12 @@ static void fts_optimize_callback(void *)
}
ib_vector_free(fts_slots);
mysql_mutex_lock(&fts_optimize_wq->mutex);
fts_slots = NULL;
mysql_cond_broadcast(&fts_opt_shutdown_cond);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
ib_wqueue_free(fts_optimize_wq);
fts_optimize_wq = NULL;
innobase_destroy_background_thd(fts_opt_thd);
ib::info() << "FTS optimize thread exiting.";
os_event_set(fts_opt_shutdown_event);
}
/**********************************************************************//**
@ -2986,7 +2955,7 @@ fts_optimize_init(void)
}
mutex_exit(&dict_sys.mutex);
fts_opt_shutdown_event = os_event_create(0);
mysql_cond_init(0, &fts_opt_shutdown_cond, nullptr);
last_check_sync_time = time(NULL);
}
@ -2996,12 +2965,10 @@ fts_optimize_shutdown()
{
ut_ad(!srv_read_only_mode);
fts_msg_t* msg;
/* If there is an ongoing activity on dictionary, such as
srv_master_evict_from_table_cache(), wait for it */
dict_mutex_enter_for_mysql();
mysql_mutex_lock(&fts_optimize_wq->mutex);
/* Tells FTS optimizer system that we are exiting from
optimizer thread, message send their after will not be
processed */
@ -3014,14 +2981,20 @@ fts_optimize_shutdown()
timer->disarm();
task_group.cancel_pending(&task);
msg = fts_optimize_create_msg(FTS_MSG_STOP, NULL);
add_msg(fts_optimize_create_msg(FTS_MSG_STOP, nullptr));
add_msg(msg);
os_event_wait(fts_opt_shutdown_event);
while (fts_slots) {
mysql_cond_wait(&fts_opt_shutdown_cond, &fts_optimize_wq->mutex);
}
os_event_destroy(fts_opt_shutdown_event);
innobase_destroy_background_thd(fts_opt_thd);
fts_opt_thd = NULL;
mysql_cond_destroy(&fts_opt_shutdown_cond);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
ib_wqueue_free(fts_optimize_wq);
fts_optimize_wq = NULL;
delete timer;
timer = NULL;
}
@ -3030,17 +3003,15 @@ fts_optimize_shutdown()
@param[in] table table to be synced */
void fts_sync_during_ddl(dict_table_t* table)
{
mutex_enter(&fts_optimize_wq->mutex);
if (!table->fts->sync_message)
{
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_lock(&fts_optimize_wq->mutex);
const auto sync_message= table->fts->sync_message;
mysql_mutex_unlock(&fts_optimize_wq->mutex);
if (!sync_message)
return;
}
mutex_exit(&fts_optimize_wq->mutex);
fts_sync_table(table, false);
mutex_enter(&fts_optimize_wq->mutex);
mysql_mutex_lock(&fts_optimize_wq->mutex);
table->fts->sync_message = false;
mutex_exit(&fts_optimize_wq->mutex);
mysql_mutex_unlock(&fts_optimize_wq->mutex);
}

8
storage/innobase/gis/gis0sea.cc

@ -386,11 +386,11 @@ rtr_pcur_getnext_from_path(
trx_t* trx = thr_get_trx(
btr_cur->rtr_info->thr);
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_init_prdt_from_mbr(
&prdt, &btr_cur->rtr_info->mbr,
mode, trx->lock.lock_heap);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
if (rw_latch == RW_NO_LATCH) {
block->lock.s_lock();
@ -1198,10 +1198,10 @@ rtr_check_discard_page(
mutex_exit(&index->rtr_track->rtr_active_mutex);
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_prdt_page_free_from_discard(block, &lock_sys.prdt_hash);
lock_prdt_page_free_from_discard(block, &lock_sys.prdt_page_hash);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
/** Structure acts as functor to get the optimistic access of the page.

192
storage/innobase/handler/ha_innodb.cc

@ -542,7 +542,6 @@ static PSI_mutex_info all_innodb_mutexes[] = {
PSI_KEY(trx_pool_manager_mutex),
PSI_KEY(lock_mutex),
PSI_KEY(lock_wait_mutex),
PSI_KEY(trx_mutex),
PSI_KEY(srv_threads_mutex),
PSI_KEY(rtr_active_mutex),
PSI_KEY(rtr_match_mutex),
@ -3058,6 +3057,11 @@ ha_innobase::init_table_handle_for_HANDLER(void)
reset_template();
}
#ifdef WITH_INNODB_DISALLOW_WRITES
/** Condition variable for innodb_disallow_writes */
static mysql_cond_t allow_writes_cond;
#endif /* WITH_INNODB_DISALLOW_WRITES */
/** Free tablespace resources allocated. */
void innobase_space_shutdown()
{
@ -3074,7 +3078,7 @@ void innobase_space_shutdown()
srv_tmp_space.shutdown();
#ifdef WITH_INNODB_DISALLOW_WRITES
os_event_destroy(srv_allow_writes_event);
mysql_cond_destroy(&allow_writes_cond);
#endif /* WITH_INNODB_DISALLOW_WRITES */
DBUG_VOID_RETURN;
@ -3606,6 +3610,10 @@ static int innodb_init(void* p)
/* After this point, error handling has to use
innodb_init_abort(). */
#ifdef WITH_INNODB_DISALLOW_WRITES
mysql_cond_init(0, &allow_writes_cond, nullptr);
#endif /* WITH_INNODB_DISALLOW_WRITES */
#ifdef HAVE_PSI_INTERFACE
/* Register keys with MySQL performance schema */
int count;
@ -4411,9 +4419,9 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
Also, BF thread should own trx mutex for the victim. */
DBUG_VOID_RETURN;
#endif /* WITH_WSREP */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
trx_sys.trx_list.freeze();
trx_mutex_enter(trx);
trx->mutex.wr_lock();
/* It is possible that innobase_close_connection() is concurrently
being executed on our victim. Even if the trx object is later
reused for another client connection or a background transaction,
@ -4432,8 +4440,8 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
if (!cancel);
else if (lock_t *lock= trx->lock.wait_lock)
lock_cancel_waiting_and_release(lock);
lock_mutex_exit();
trx_mutex_exit(trx);
mysql_mutex_unlock(&lock_sys.mutex);
trx->mutex.wr_unlock();
}
DBUG_VOID_RETURN;
@ -14266,90 +14274,64 @@ ha_innobase::analyze(THD*, HA_CHECK_OPT*)
/*****************************************************************//**
Defragment table.
@return error number */
inline int ha_innobase::defragment_table(const char *name)
inline int ha_innobase::defragment_table()
{
char norm_name[FN_REFLEN];
dict_table_t* table = NULL;
dict_index_t* index = NULL;
int ret = 0;
dberr_t err = DB_SUCCESS;
normalize_table_name(norm_name, name);
table = dict_table_open_on_name(norm_name, FALSE,
FALSE, DICT_ERR_IGNORE_FK_NOKEY);
for (index = dict_table_get_first_index(table); index;
index = dict_table_get_next_index(index)) {
if (index->is_corrupted()) {
continue;
}
if (dict_index_is_spatial(index)) {
/* Do not try to defragment spatial indexes,
because doing it properly would require
appropriate logic around the SSN (split
sequence number). */
continue;
}
if (index->page == FIL_NULL) {
/* Do not defragment auxiliary tables related
to FULLTEXT INDEX. */
ut_ad(index->type & DICT_FTS);
continue;
}
if (btr_defragment_find_index(index)) {
// We borrow this error code. When the same index is
// already in the defragmentation queue, issue another
// defragmentation only introduces overhead. We return
// an error here to let the user know this is not
// necessary. Note that this will fail a query that's
// trying to defragment a full table if one of the
// indicies in that table is already in defragmentation.
// We choose this behavior so user is aware of this
// rather than silently defragment other indicies of
// that table.
ret = ER_SP_ALREADY_EXISTS;
break;
}
for (dict_index_t *index= dict_table_get_first_index(m_prebuilt->table);
index; index= dict_table_get_next_index(index))
{
if (index->is_corrupted() || index->is_spatial())
continue;
os_event_t event = btr_defragment_add_index(index, &err);
if (index->page == FIL_NULL)
{
/* Do not defragment auxiliary tables related to FULLTEXT INDEX. */
ut_ad(index->type & DICT_FTS);
continue;
}
if (err != DB_SUCCESS) {
push_warning_printf(
current_thd,
Sql_condition::WARN_LEVEL_WARN,
ER_NO_SUCH_TABLE,
"Table %s is encrypted but encryption service or"
" used key_id is not available. "
" Can't continue checking table.",
index->table->name.m_name);
if (btr_defragment_find_index(index))
{
// We borrow this error code. When the same index is already in
// the defragmentation queue, issuing another defragmentation
// only introduces overhead. We return an error here to let the
// user know this is not necessary. Note that this will fail a
// query that's trying to defragment a full table if one of the
// indicies in that table is already in defragmentation. We
// choose this behavior so user is aware of this rather than
// silently defragment other indicies of that table.
return ER_SP_ALREADY_EXISTS;
}
ret = convert_error_code_to_mysql(err, 0, current_thd);
break;
}
btr_pcur_t pcur;
pcur.btr_cur.index = nullptr;
btr_pcur_init(&pcur);
if (event) {
while(os_event_wait_time(event, 1000000)) {
if (thd_killed(current_thd)) {
btr_defragment_remove_index(index);
ret = ER_QUERY_INTERRUPTED;
break;
}
}
os_event_destroy(event);
}
mtr_t mtr;
mtr.start();
if (dberr_t err= btr_pcur_open_at_index_side(true, index,
BTR_SEARCH_LEAF, &pcur,
true, 0, &mtr))
{
mtr.commit();
return convert_error_code_to_mysql(err, 0, m_user_thd);
}
else if (btr_pcur_get_block(&pcur)->page.id().page_no() == index->page)
{
mtr.commit();
continue;
}
if (ret) {
break;
}
}
btr_pcur_move_to_next(&pcur, &mtr);
btr_pcur_store_position(&pcur, &mtr);
mtr.commit();
ut_ad(pcur.btr_cur.index == index);
const bool interrupted= btr_defragment_add_index(&pcur, m_user_thd);
btr_pcur_free(&pcur);
if (interrupted)
return ER_QUERY_INTERRUPTED;
}
dict_table_close(table, FALSE, FALSE);
return ret;
return 0;
}
/**********************************************************************//**
@ -14373,8 +14355,10 @@ ha_innobase::optimize(
calls to OPTIMIZE, which is undesirable. */
bool try_alter = true;
if (!m_prebuilt->table->is_temporary() && srv_defragment) {
int err = defragment_table(m_prebuilt->table->name.m_name);
if (!m_prebuilt->table->is_temporary()
&& m_prebuilt->table->is_readable()
&& srv_defragment) {
int err = defragment_table();
if (err == 0) {
try_alter = false;
@ -18190,8 +18174,7 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
{
ut_ad(bf_thd);
ut_ad(victim_trx);
ut_ad(lock_mutex_own());
ut_ad(trx_mutex_own(victim_trx));
mysql_mutex_assert_owner(&lock_sys.mutex);
DBUG_ENTER("wsrep_innobase_kill_one_trx");
@ -18293,12 +18276,12 @@ wsrep_abort_transaction(
wsrep_thd_transaction_state_str(victim_thd));
if (victim_trx) {
lock_mutex_enter();
trx_mutex_enter(victim_trx);
mysql_mutex_lock(&lock_sys.mutex);
victim_trx->mutex.wr_lock();
int rcode= wsrep_innobase_kill_one_trx(bf_thd,
victim_trx, signal);
trx_mutex_exit(victim_trx);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
victim_trx->mutex.wr_unlock();
DBUG_RETURN(rcode);
} else {
wsrep_thd_bf_abort(bf_thd, victim_thd, signal);
@ -19066,12 +19049,18 @@ static MYSQL_SYSVAR_ULONG(buf_dump_status_frequency, srv_buf_dump_status_frequen
NULL, NULL, 0, 0, 100, 0);
#ifdef WITH_INNODB_DISALLOW_WRITES
/*******************************************************
* innobase_disallow_writes variable definition *
*******************************************************/
/* Must always init to FALSE. */
static my_bool innobase_disallow_writes = FALSE;
my_bool innodb_disallow_writes;
void innodb_wait_allow_writes()
{
if (UNIV_UNLIKELY(innodb_disallow_writes))
{
mysql_mutex_lock(&LOCK_global_system_variables);
while (innodb_disallow_writes)
mysql_cond_wait(&allow_writes_cond, &LOCK_global_system_variables);
mysql_mutex_unlock(&LOCK_global_system_variables);
}
}
/**************************************************************************
An "update" method for innobase_disallow_writes variable. */
@ -19082,17 +19071,14 @@ innobase_disallow_writes_update(THD*, st_mysql_sys_var*,
{
const my_bool val = *static_cast<const my_bool*>(save);
*static_cast<my_bool*>(var_ptr) = val;
ut_a(srv_allow_writes_event);
mysql_mutex_unlock(&LOCK_global_system_variables);
if (val) {
os_event_reset(srv_allow_writes_event);
} else {
os_event_set(srv_allow_writes_event);
if (!val) {
mysql_cond_broadcast(&allow_writes_cond);
}
mysql_mutex_lock(&LOCK_global_system_variables);
}
static MYSQL_SYSVAR_BOOL(disallow_writes, innobase_disallow_writes,
static MYSQL_SYSVAR_BOOL(disallow_writes, innodb_disallow_writes,
PLUGIN_VAR_NOCMDOPT,
"Tell InnoDB to stop any writes to disk",
NULL, innobase_disallow_writes_update, FALSE);

2
storage/innobase/handler/ha_innodb.h

@ -207,7 +207,7 @@ public:
int delete_table(const char *name) override;
int rename_table(const char* from, const char* to) override;
inline int defragment_table(const char* name);
inline int defragment_table();
int check(THD* thd, HA_CHECK_OPT* check_opt) override;
char* update_table_comment(const char* comment) override;

4
storage/innobase/ibuf/ibuf0ibuf.cc

@ -3284,9 +3284,9 @@ commit_exit:
ibuf_mtr_commit(&bitmap_mtr);
goto fail_exit;
} else {
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
const auto lock_exists = lock_sys.get_first(page_id);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
if (lock_exists) {
goto commit_exit;
}

17
storage/innobase/include/btr0defragment.h

@ -43,13 +43,11 @@ Check whether the given index is in btr_defragment_wq. */
bool
btr_defragment_find_index(
dict_index_t* index); /*!< Index to find. */
/******************************************************************//**
Add an index to btr_defragment_wq. Return a pointer to os_event if this
is a synchronized defragmentation. */
os_event_t
btr_defragment_add_index(
dict_index_t* index, /*!< index to be added */
dberr_t* err); /*!< out: error code */
/** Defragment an index.
@param pcur persistent cursor
@param thd current session, for checking thd_killed()
@return whether the operation was interrupted */
bool btr_defragment_add_index(btr_pcur_t *pcur, THD *thd);
/******************************************************************//**
When table is dropped, this function is called to mark a table as removed in
btr_efragment_wq. The difference between this function and the remove_index
@ -57,11 +55,6 @@ function is this will not NULL the event. */
void
btr_defragment_remove_table(
dict_table_t* table); /*!< Index to be removed. */
/******************************************************************//**
Mark an index as removed from btr_defragment_wq. */
void
btr_defragment_remove_index(
dict_index_t* index); /*!< Index to be removed. */
/*********************************************************************//**
Check whether we should save defragmentation statistics to persistent storage.*/
UNIV_INTERN

2
storage/innobase/include/buf0buf.h

@ -32,9 +32,9 @@ Created 11/5/1995 Heikki Tuuri
#include "fil0fil.h"
#include "mtr0types.h"
#include "buf0types.h"
#include "span.h"
#include "assume_aligned.h"
#include "buf0types.h"
#ifndef UNIV_INNOCHECKSUM
#include "hash0hash.h"
#include "ut0byte.h"

10
storage/innobase/include/fil0crypt.h

@ -26,7 +26,6 @@ Created 04/01/2015 Jan Lindström
#ifndef fil0crypt_h
#define fil0crypt_h
#include "os0event.h"
#include "my_crypt.h"
#include "fil0fil.h"
@ -41,7 +40,8 @@ static const unsigned char CRYPT_MAGIC[MAGIC_SZ] = {
/* This key will be used if nothing else is given */
#define FIL_DEFAULT_ENCRYPTION_KEY ENCRYPTION_KEY_SYSTEM_DATA
extern os_event_t fil_crypt_threads_event;
/** Wake up the encryption threads */
void fil_crypt_threads_signal(bool broadcast= false);
/**
* CRYPT_SCHEME_UNENCRYPTED
@ -116,7 +116,7 @@ struct fil_space_crypt_t : st_encryption_scheme
{
key_id = new_key_id;
my_random_bytes(iv, sizeof(iv));
mutex_create(LATCH_ID_FIL_CRYPT_DATA_MUTEX, &mutex);
mysql_mutex_init(0, &mutex, nullptr);
locker = crypt_data_scheme_locker;
type = new_type;
@ -135,7 +135,7 @@ struct fil_space_crypt_t : st_encryption_scheme
/** Destructor */
~fil_space_crypt_t()
{
mutex_free(&mutex);
mysql_mutex_destroy(&mutex);
}
/** Get latest key version from encryption plugin
@ -186,7 +186,7 @@ struct fil_space_crypt_t : st_encryption_scheme
uint min_key_version; // min key version for this space
fil_encryption_t encryption; // Encryption setup
ib_mutex_t mutex; // mutex protecting following variables
mysql_mutex_t mutex; // mutex protecting following variables
/** Return code from encryption_key_get_latest_version.
If ENCRYPTION_KEY_VERSION_INVALID encryption plugin

3
storage/innobase/include/fts0priv.h

@ -424,8 +424,7 @@ Append deleted doc ids to vector and sort the vector. */
void
fts_cache_append_deleted_doc_ids(
/*=============================*/
const fts_cache_t*
cache, /*!< in: cache to use */
fts_cache_t* cache, /*!< in: cache to use */
ib_vector_t* vector); /*!< in: append to this vector */
/******************************************************************//**
Search the index specific cache for a particular FTS index.

26
storage/innobase/include/fts0types.h

@ -114,23 +114,25 @@ struct fts_sync_t {
bool in_progress; /*!< flag whether sync is in progress.*/
bool unlock_cache; /*!< flag whether unlock cache when
write fts node */
os_event_t event; /*!< sync finish event;
only os_event_set() and os_event_wait()
are used */
/** condition variable for in_progress; used with table->fts->cache->lock */
mysql_cond_t cond;
};
/** The cache for the FTS system. It is a memory-based inverted index
that new entries are added to, until it grows over the configured maximum
size, at which time its contents are written to the INDEX table. */
struct fts_cache_t {
mysql_mutex_t lock; /*!< lock protecting all access to the
memory buffer */
mysql_mutex_t init_lock; /*!< lock used for the cache
intialization */
ib_mutex_t deleted_lock; /*!< Lock covering deleted_doc_ids */
ib_mutex_t doc_id_lock; /*!< Lock covering Doc ID */
struct fts_cache_t
{
/** lock protecting all access to the memory buffer */
mysql_mutex_t lock;
/** cache initialization */
mysql_mutex_t init_lock;
/** protection for deleted_doc_ids */
mysql_mutex_t deleted_lock;
/** protection for DOC_ID */
mysql_mutex_t doc_id_lock;
ib_vector_t* deleted_doc_ids;/*!< Array of deleted doc ids, each
element is of type fts_update_t */

56
storage/innobase/include/lock0lock.h

@ -701,17 +701,14 @@ struct lock_op_t{
lock_mode mode; /*!< lock mode */
};
typedef ib_mutex_t LockMutex;
/** The lock system struct */
class lock_sys_t
{
bool m_initialised;
public:
MY_ALIGNED(CACHE_LINE_SIZE)
LockMutex mutex; /*!< Mutex protecting the
locks */
/** mutex proteting the locks */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t mutex;
/** record locks */
hash_table_t rec_hash;
/** predicate locks for SPATIAL INDEX */
@ -719,22 +716,13 @@ public:
/** page locks for SPATIAL INDEX */
hash_table_t prdt_page_hash;
MY_ALIGNED(CACHE_LINE_SIZE)
LockMutex wait_mutex; /*!< Mutex protecting the
next two fields */
/** mutex protecting waiting_threads, last_slot */
MY_ALIGNED(CACHE_LINE_SIZE) mysql_mutex_t wait_mutex;
srv_slot_t* waiting_threads; /*!< Array of user threads
suspended while waiting for
locks within InnoDB, protected
by the lock_sys.wait_mutex;
os_event_set() and
os_event_reset() on
waiting_threads[]->event
are protected by
trx_t::mutex */
locks within InnoDB */
srv_slot_t* last_slot; /*!< highest slot ever used
in the waiting_threads array,
protected by
lock_sys.wait_mutex */
in the waiting_threads array */
ulint n_lock_max_wait_time; /*!< Max wait time */
@ -775,7 +763,7 @@ public:
/** @return the hash value for a page address */
ulint hash(const page_id_t id) const
{ ut_ad(mutex_own(&mutex)); return rec_hash.calc_hash(id.fold()); }
{ mysql_mutex_assert_owner(&mutex); return rec_hash.calc_hash(id.fold()); }
/** Get the first lock on a page.
@param lock_hash hash table to look at
@ -925,36 +913,6 @@ lock_rec_free_all_from_discard_page(
/** The lock system */
extern lock_sys_t lock_sys;
/** Test if lock_sys.mutex can be acquired without waiting. */
#define lock_mutex_enter_nowait() \
(lock_sys.mutex.trylock(__FILE__, __LINE__))
/** Test if lock_sys.mutex is owned. */
#define lock_mutex_own() (lock_sys.mutex.is_owned())
/** Acquire the lock_sys.mutex. */
#define lock_mutex_enter() do { \
mutex_enter(&lock_sys.mutex); \
} while (0)
/** Release the lock_sys.mutex. */
#define lock_mutex_exit() do { \
lock_sys.mutex.exit(); \
} while (0)
/** Test if lock_sys.wait_mutex is owned. */
#define lock_wait_mutex_own() (lock_sys.wait_mutex.is_owned())
/** Acquire the lock_sys.wait_mutex. */
#define lock_wait_mutex_enter() do { \
mutex_enter(&lock_sys.wait_mutex); \
} while (0)
/** Release the lock_sys.wait_mutex. */
#define lock_wait_mutex_exit() do { \
lock_sys.wait_mutex.exit(); \
} while (0)
#ifdef WITH_WSREP
/*********************************************************************//**
Cancels a waiting lock request and releases possible other transactions

5
storage/innobase/include/lock0priv.h

@ -629,8 +629,7 @@ inline void lock_set_lock_and_trx_wait(lock_t* lock, trx_t* trx)
ut_ad(lock);
ut_ad(lock->trx == trx);
ut_ad(trx->lock.wait_lock == NULL);
ut_ad(lock_mutex_own());
ut_ad(trx_mutex_own(trx));
mysql_mutex_assert_owner(&lock_sys.mutex);
trx->lock.wait_lock = lock;
lock->type_mode |= LOCK_WAIT;
@ -641,7 +640,7 @@ inline void lock_set_lock_and_trx_wait(lock_t* lock, trx_t* trx)
inline void lock_reset_lock_and_trx_wait(lock_t* lock)
{
ut_ad(lock_get_wait(lock));
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
ut_ad(lock->trx->lock.wait_lock == NULL
|| lock->trx->lock.wait_lock == lock);
lock->trx->lock.wait_lock = NULL;

4
storage/innobase/include/lock0priv.ic

@ -131,7 +131,7 @@ lock_rec_get_next(
ulint heap_no,/*!< in: heap number of the record */
lock_t* lock) /*!< in: lock */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
do {
ut_ad(lock_get_type_low(lock) == LOCK_REC);
@ -206,7 +206,7 @@ lock_rec_get_next_on_page_const(
/*============================*/
const lock_t* lock) /*!< in: a record lock */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
ut_ad(lock_get_type_low(lock) == LOCK_REC);
const page_id_t page_id(lock->un_member.rec_lock.page_id);

37
storage/innobase/include/log0recv.h

@ -209,14 +209,25 @@ struct page_recv_t
struct recv_sys_t
{
/** mutex protecting apply_log_recs and page_recv_t::state */
ib_mutex_t mutex;
mysql_mutex_t mutex;
private:
/** condition variable for
!apply_batch_on || pages.empty() || found_corrupt_log || found_corrupt_fs */
mysql_cond_t cond;
/** whether recv_apply_hashed_log_recs() is running */
bool apply_batch_on;
/** set when finding a corrupt log block or record, or there is a
log parsing buffer overflow */
bool found_corrupt_log;
/** set when an inconsistency with the file system contents is detected
during log scan or apply */
bool found_corrupt_fs;
public:
/** whether we are applying redo log records during crash recovery */
bool recovery_on;
/** whether recv_recover_page(), invoked from buf_page_read_complete(),
should apply log records*/
bool apply_log_recs;
/** whether recv_apply_hashed_log_recs() is running */
bool apply_batch_on;
byte* buf; /*!< buffer for parsing log records */
ulint len; /*!< amount of data in buf */
lsn_t parse_start_lsn;
@ -236,14 +247,6 @@ struct recv_sys_t
lsn_t recovered_lsn;
/*!< the log records have been parsed up to
this lsn */
bool found_corrupt_log;
/*!< set when finding a corrupt log
block or record, or there is a log
parsing buffer overflow */
bool found_corrupt_fs;
/*!< set when an inconsistency with
the file system contents is detected
during log scan or apply */
lsn_t mlog_checkpoint_lsn;
/*!< the LSN of a FILE_CHECKPOINT
record, or 0 if none was parsed */
@ -382,6 +385,18 @@ public:
@param page_id corrupted page identifier */
ATTRIBUTE_COLD void free_corrupted_page(page_id_t page_id);
/** Flag data file corruption during recovery. */
ATTRIBUTE_COLD void set_corrupt_fs();
/** Flag log file corruption during recovery. */
ATTRIBUTE_COLD void set_corrupt_log();
/** Possibly finish a recovery batch. */
inline void maybe_finish_batch();
/** @return whether data file corruption was found */
bool is_corrupt_fs() const { return UNIV_UNLIKELY(found_corrupt_fs); }
/** @return whether log file corruption was found */
bool is_corrupt_log() const { return UNIV_UNLIKELY(found_corrupt_log); }
/** Attempt to initialize a page based on redo log records.
@param page_id page identifier
@return the recovered block

7
storage/innobase/include/row0ftsort.h

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 2010, 2016, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2015, 2019, MariaDB Corporation.
Copyright (c) 2015, 2020, MariaDB Corporation.
This program is free software; you can redistribute it and/or modify it under
the terms of the GNU General Public License as published by the Free Software
@ -65,7 +65,7 @@ struct fts_psort_common_t {
ulint old_zip_size;
trx_t* trx; /*!< transaction */
fts_psort_t* all_info; /*!< all parallel sort info */
os_event_t sort_event; /*!< sort event */
mysql_cond_t sort_cond; /*!< sort completion */
ibool opt_doc_id_size;/*!< whether to use 4 bytes
instead of 8 bytes integer to
store Doc ID during sort, if
@ -90,7 +90,7 @@ struct fts_psort_t {
tpool::waitable_task* task; /*!< threadpool task */
dberr_t error; /*!< db error during psort */
ulint memory_used; /*!< memory used by fts_doc_list */
ib_mutex_t mutex; /*!< mutex for fts_doc_list */
mysql_mutex_t mutex; /*!< mutex for fts_doc_list */
};
/** Row fts token for plugin parser */
@ -152,7 +152,6 @@ typedef struct fts_psort_insert fts_psort_insert_t;
#define FTS_PARENT_COMPLETE 1
#define FTS_PARENT_EXITING 2
#define FTS_CHILD_COMPLETE 1
#define FTS_CHILD_EXITING 2
/** Print some debug information */
#define FTSORT_PRINT

2
storage/innobase/include/row0vers.h

@ -45,7 +45,7 @@ index record.
@param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index)
@return the active transaction; state must be rechecked after
trx_mutex_enter(), and trx->release_reference() must be invoked
acquiring trx->mutex, and trx->release_reference() must be invoked
@retval NULL if the record was committed */
trx_t*
row_vers_impl_x_locked(

17
storage/innobase/include/srv0srv.h

@ -301,8 +301,10 @@ extern my_bool srv_adaptive_flushing;
extern my_bool srv_flush_sync;
#ifdef WITH_INNODB_DISALLOW_WRITES
/* When this event is reset we do not allow any file writes to take place. */
extern os_event_t srv_allow_writes_event;
extern my_bool innodb_disallow_writes;
void innodb_wait_allow_writes();
#else
# define innodb_wait_allow_writes() do {} while (0)
#endif /* WITH_INNODB_DISALLOW_WRITES */
/* If this flag is TRUE, then we will load the indexes' (and tables') metadata
@ -715,6 +717,9 @@ void srv_init_purge_tasks();
void
srv_master_thread_disabled_debug_update(THD*, st_mysql_sys_var*, void*,
const void* save);
/** Enable the master thread on shutdown. */
void srv_master_thread_enable();
#endif /* UNIV_DEBUG */
/** Status variables to be passed to MySQL */
@ -862,7 +867,7 @@ struct export_var_t{
/** Thread slot in the thread table. */
struct srv_slot_t{
ibool in_use; /*!< TRUE if this slot
bool in_use; /*!< true if this slot
is in use */
/** time(NULL) when the thread was suspended.
FIXME: Use my_interval_timer() or similar, to avoid bogus
@ -877,9 +882,9 @@ struct srv_slot_t{
Initialized by
lock_wait_table_reserve_slot()
for lock wait */
os_event_t event; /*!< event used in suspending
the thread when it has nothing
to do */
mysql_cond_t cond; /*!< condition variable for
waking up suspended thread,
under lock_sys.mutex */
que_thr_t* thr; /*!< suspended query thread
(only used for user threads) */
};

1
storage/innobase/include/sync0sync.h

@ -68,7 +68,6 @@ extern mysql_pfs_key_t srv_innodb_monitor_mutex_key;
extern mysql_pfs_key_t srv_misc_tmpfile_mutex_key;
extern mysql_pfs_key_t srv_monitor_file_mutex_key;
extern mysql_pfs_key_t buf_dblwr_mutex_key;
extern mysql_pfs_key_t trx_mutex_key;
extern mysql_pfs_key_t trx_pool_mutex_key;
extern mysql_pfs_key_t trx_pool_manager_mutex_key;
extern mysql_pfs_key_t lock_mutex_key;

24
storage/innobase/include/sync0types.h

@ -141,10 +141,10 @@ V
File system pages
|
V
lock_sys_wait_mutex Mutex protecting lock timeout data
lock_sys.wait_mutex Mutex protecting lock timeout data
|
V
lock_sys_mutex Mutex protecting lock_sys_t
lock_sys.mutex Mutex protecting lock_sys_t
|
V
trx_sys.mutex Mutex protecting trx_sys.trx_list
@ -191,18 +191,10 @@ enum latch_level_t {
SYNC_POOL,
SYNC_POOL_MANAGER,
SYNC_WORK_QUEUE,
SYNC_FTS_TOKENIZE,
SYNC_FTS_OPTIMIZE,
SYNC_RECV,
SYNC_PURGE_QUEUE,
SYNC_TRX,
SYNC_RW_TRX_HASH_ELEMENT,
SYNC_READ_VIEW,
SYNC_TRX_SYS,
SYNC_LOCK_SYS,
SYNC_LOCK_WAIT_SYS,
SYNC_INDEX_ONLINE_LOG,
@ -231,15 +223,11 @@ enum latch_id_t {
LATCH_ID_DICT_FOREIGN_ERR,
LATCH_ID_DICT_SYS,
LATCH_ID_FIL_SYSTEM,
LATCH_ID_FTS_DELETE,
LATCH_ID_FTS_DOC_ID,
LATCH_ID_FTS_PLL_TOKENIZE,
LATCH_ID_IBUF_BITMAP,
LATCH_ID_IBUF,
LATCH_ID_IBUF_PESSIMISTIC_INSERT,
LATCH_ID_PURGE_SYS_PQ,
LATCH_ID_RECALC_POOL,
LATCH_ID_RECV_SYS,
LATCH_ID_REDO_RSEG,
LATCH_ID_NOREDO_RSEG,
LATCH_ID_RTR_ACTIVE_MUTEX,
@ -250,22 +238,14 @@ enum latch_id_t {
LATCH_ID_SRV_MONITOR_FILE,
LATCH_ID_TRX_POOL,
LATCH_ID_TRX_POOL_MANAGER,
LATCH_ID_TRX,
LATCH_ID_LOCK_SYS,
LATCH_ID_LOCK_SYS_WAIT,
LATCH_ID_TRX_SYS,
LATCH_ID_SRV_SYS_TASKS,
LATCH_ID_PAGE_ZIP_STAT_PER_INDEX,
LATCH_ID_SYNC_ARRAY_MUTEX,
LATCH_ID_ROW_DROP_LIST,
LATCH_ID_INDEX_ONLINE_LOG,
LATCH_ID_WORK_QUEUE,
LATCH_ID_DICT_TABLE_STATS,
LATCH_ID_DEFRAGMENT_MUTEX,
LATCH_ID_BTR_DEFRAGMENT_MUTEX,
LATCH_ID_FIL_CRYPT_STAT_MUTEX,
LATCH_ID_FIL_CRYPT_DATA_MUTEX,
LATCH_ID_FIL_CRYPT_THREADS_MUTEX,
LATCH_ID_RW_TRX_HASH_ELEMENT,
LATCH_ID_READ_VIEW,
LATCH_ID_MAX = LATCH_ID_READ_VIEW

4
storage/innobase/include/trx0sys.h

@ -514,12 +514,12 @@ class rw_trx_hash_t
ut_ad(!trx->read_only || !trx->rsegs.m_redo.rseg);
ut_ad(!trx_is_autocommit_non_locking(trx));
/* trx->state can be anything except TRX_STATE_NOT_STARTED */
mutex_enter(&trx->mutex);
ut_d(trx->mutex.wr_lock());
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE) ||
trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY) ||
trx_state_eq(trx, TRX_STATE_PREPARED_RECOVERED) ||
trx_state_eq(trx, TRX_STATE_PREPARED));
mutex_exit(&trx->mutex);
ut_d(trx->mutex.wr_unlock());
}

20
storage/innobase/include/trx0trx.h

@ -719,10 +719,9 @@ private:
public:
TrxMutex mutex; /*!< Mutex protecting the fields
state and lock (except some fields
of lock, which are protected by
lock_sys.mutex) */
/** mutex protecting state and some of lock
(some are protected by lock_sys.mutex) */
srw_mutex mutex;
trx_id_t id; /*!< transaction id */
@ -1169,19 +1168,6 @@ struct commit_node_t{
};
/** Test if trx->mutex is owned. */
#define trx_mutex_own(t) mutex_own(&t->mutex)
/** Acquire the trx->mutex. */
#define trx_mutex_enter(t) do { \
mutex_enter(&t->mutex); \
} while (0)
/** Release the trx->mutex. */
#define trx_mutex_exit(t) do { \
mutex_exit(&t->mutex); \
} while (0)
#include "trx0trx.ic"
#endif

3
storage/innobase/include/trx0types.h

@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2014, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2017, 2019, MariaDB Corporation.
Copyright (c) 2017, 2020, MariaDB Corporation.
This program is free software; you can redistribute it and/or modify it under
the terms of the GNU General Public License as published by the Free Software
@ -136,7 +136,6 @@ typedef byte trx_undo_rec_t;
/* @} */
typedef ib_mutex_t RsegMutex;
typedef ib_mutex_t TrxMutex;
typedef ib_mutex_t PQMutex;
typedef ib_mutex_t TrxSysMutex;

13
storage/innobase/include/ut0wqueue.h

@ -30,8 +30,7 @@ wait for work items to be available and take them off the queue for
processing.
************************************************************************/
#ifndef IB_WORK_QUEUE_H
#define IB_WORK_QUEUE_H
#pragma once
#include "ut0list.h"
#include "ut0mutex.h"
@ -43,10 +42,10 @@ struct ib_list_t;
/** Work queue */
struct ib_wqueue_t
{
/** Mutex protecting everything */
ib_mutex_t mutex;
/** Work item list */
ib_list_t* items;
/** Mutex protecting everything */
mysql_mutex_t mutex;
/** Work item list */
ib_list_t *items;
};
/****************************************************************//**
@ -91,5 +90,3 @@ ulint
ib_wqueue_len(
/*==========*/
ib_wqueue_t* wq); /*<! in: work queue */
#endif /* IB_WORK_QUEUE_H */

5
storage/innobase/lock/lock0iter.cc

@ -1,6 +1,7 @@
/*****************************************************************************
Copyright (c) 2007, 2014, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2020, MariaDB Corporation.
This program is free software; you can redistribute it and/or modify it under
the terms of the GNU General Public License as published by the Free Software
@ -48,7 +49,7 @@ lock_queue_iterator_reset(
ulint bit_no) /*!< in: record number in the
heap */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
iter->current_lock = lock;
@ -83,7 +84,7 @@ lock_queue_iterator_get_prev(
{
const lock_t* prev_lock;
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
switch (lock_get_type_low(iter->current_lock)) {
case LOCK_REC:

417
storage/innobase/lock/lock0lock.cc
File diff suppressed because it is too large
View File

88
storage/innobase/lock/lock0prdt.cc

@ -237,7 +237,7 @@ lock_prdt_has_lock(
{
lock_t* lock;
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
ut_ad((precise_mode & LOCK_MODE_MASK) == LOCK_S
|| (precise_mode & LOCK_MODE_MASK) == LOCK_X);
ut_ad(!(precise_mode & LOCK_INSERT_INTENTION));
@ -295,7 +295,7 @@ lock_prdt_other_has_conflicting(
the new lock will be on */
const trx_t* trx) /*!< in: our transaction */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
for (lock_t* lock = lock_rec_get_first(
lock_hash_get(mode), block, PRDT_HEAPNO);
@ -392,7 +392,7 @@ lock_prdt_find_on_page(
{
lock_t* lock;
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
for (lock = lock_sys.get_first(*lock_hash_get(type_mode),
block->page.id());
@ -437,8 +437,7 @@ lock_prdt_add_to_queue(
/*!< in: TRUE if caller owns the
transaction mutex */
{
ut_ad(lock_mutex_own());
ut_ad(caller_owns_trx_mutex == trx_mutex_own(trx));
mysql_mutex_assert_owner(&lock_sys.mutex);
ut_ad(!dict_index_is_clust(index) && !dict_index_is_online_ddl(index));
ut_ad(type_mode & (LOCK_PREDICATE | LOCK_PRDT_PAGE));
@ -532,7 +531,7 @@ lock_prdt_insert_check_and_lock(
trx_t* trx = thr_get_trx(thr);
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
/* Because this code is invoked for a running transaction by
the thread that is serving the transaction, it is not necessary
@ -546,7 +545,7 @@ lock_prdt_insert_check_and_lock(
lock = lock_rec_get_first(&lock_sys.prdt_hash, block, PRDT_HEAPNO);
if (lock == NULL) {
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
/* Update the page max trx id field */
page_update_max_trx_id(block, buf_block_get_page_zip(block),
@ -578,7 +577,7 @@ lock_prdt_insert_check_and_lock(
lock_init_prdt_from_mbr(prdt, mbr, 0, trx->lock.lock_heap);
/* Note that we may get DB_SUCCESS also here! */
trx_mutex_enter(trx);
trx->mutex.wr_lock();
err = lock_rec_enqueue_waiting(
#ifdef WITH_WSREP
@ -587,12 +586,12 @@ lock_prdt_insert_check_and_lock(
LOCK_X | LOCK_PREDICATE | LOCK_INSERT_INTENTION,
block, PRDT_HEAPNO, index, thr, prdt);
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
} else {
err = DB_SUCCESS;
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
switch (err) {
case DB_SUCCESS_LOCKED_REC:
@ -623,7 +622,7 @@ lock_prdt_update_parent(
lock_prdt_t* right_prdt, /*!< in: MBR on the new page */
const page_id_t page_id) /*!< in: parent page */
{
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
/* Get all locks in parent */
for (lock_t *lock = lock_sys.get_first_prdt(page_id);
@ -661,7 +660,7 @@ lock_prdt_update_parent(
}
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
/**************************************************************//**
@ -682,16 +681,16 @@ lock_prdt_update_split_low(
for (lock = lock_sys.get_first(*lock_hash_get(type_mode), page_id);
lock;
lock = lock_rec_get_next_on_page(lock)) {
trx_t* trx = lock->trx;
/* First dealing with Page Lock */
if (lock->type_mode & LOCK_PRDT_PAGE) {
/* Duplicate the lock to new page */
trx_mutex_enter(lock->trx);
trx->mutex.wr_lock();
lock_prdt_add_to_queue(lock->type_mode,
new_block,
lock->index,
lock->trx, NULL, TRUE);
trx_mutex_exit(lock->trx);
trx, NULL, TRUE);
trx->mutex.wr_unlock();
continue;
}
@ -708,27 +707,13 @@ lock_prdt_update_split_low(
lock_prdt = lock_get_prdt_from_lock(lock);
if (lock_prdt_consistent(lock_prdt, prdt, op)) {
if (!lock_prdt_consistent(lock_prdt, new_prdt, op)) {
/* Move the lock to new page */
trx_mutex_enter(lock->trx);
lock_prdt_add_to_queue(lock->type_mode,
new_block,
lock->index,
lock->trx, lock_prdt,
TRUE);
trx_mutex_exit(lock->trx);
}
} else if (!lock_prdt_consistent(lock_prdt, new_prdt, op)) {
/* Duplicate the lock to new page */
trx_mutex_enter(lock->trx);
lock_prdt_add_to_queue(lock->type_mode,
new_block,
lock->index,
lock->trx, lock_prdt, TRUE);
trx_mutex_exit(lock->trx);
if (!lock_prdt_consistent(lock_prdt, new_prdt, op)) {
/* Move the lock to new page */
trx->mutex.wr_lock();
lock_prdt_add_to_queue(lock->type_mode, new_block,
lock->index, trx, lock_prdt,
TRUE);
trx->mutex.wr_unlock();
}
}
}
@ -743,7 +728,7 @@ lock_prdt_update_split(
lock_prdt_t* new_prdt, /*!< in: MBR on the new page */
const page_id_t page_id) /*!< in: page number */
{
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock_prdt_update_split_low(new_block, prdt, new_prdt,
page_id, LOCK_PREDICATE);
@ -751,7 +736,7 @@ lock_prdt_update_split(
lock_prdt_update_split_low(new_block, NULL, NULL,
page_id, LOCK_PRDT_PAGE);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
/*********************************************************************//**
@ -816,7 +801,7 @@ lock_prdt_lock(
index record, and this would not have been possible if another active
transaction had modified this secondary index record. */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
const unsigned prdt_mode = type_mode | mode;
lock_t* lock = lock_sys.get_first(hash, block->page.id());
@ -831,7 +816,7 @@ lock_prdt_lock(
status = LOCK_REC_SUCCESS_CREATED;
} else {
trx_mutex_enter(trx);
trx->mutex.wr_lock();
if (lock_rec_get_next_on_page(lock)
|| lock->trx != trx
@ -871,10 +856,9 @@ lock_prdt_lock(
}
}
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
} else {
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
if (!lock_rec_get_nth_bit(lock, PRDT_HEAPNO)) {
lock_rec_set_nth_bit(lock, PRDT_HEAPNO);
@ -883,7 +867,7 @@ lock_prdt_lock(
}
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
if (status == LOCK_REC_SUCCESS_CREATED && type_mode == LOCK_PREDICATE) {
/* Append the predicate in the lock record */
@ -913,7 +897,7 @@ lock_place_prdt_page_lock(
index record, and this would not have been possible if another active
transaction had modified this secondary index record. */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
const lock_t* lock = lock_sys.get_first_prdt_page(page_id);
const ulint mode = LOCK_S | LOCK_PRDT_PAGE;
@ -943,7 +927,7 @@ lock_place_prdt_page_lock(
#endif /* PRDT_DIAG */
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
return(DB_SUCCESS);
}
@ -956,11 +940,11 @@ bool lock_test_prdt_page_lock(const trx_t *trx, const page_id_t page_id)
{
lock_t* lock;
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
lock = lock_sys.get_first_prdt_page(page_id);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
return(!lock || trx == lock->trx);
}
@ -976,7 +960,7 @@ lock_prdt_rec_move(
const buf_block_t* donator) /*!< in: buffer block containing
the donating record */
{
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
for (lock_t *lock = lock_rec_get_first(&lock_sys.prdt_hash,
donator, PRDT_HEAPNO);
@ -994,7 +978,7 @@ lock_prdt_rec_move(
lock_prdt, FALSE);
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
/** Removes predicate lock objects set on an index page which is discarded.
@ -1008,7 +992,7 @@ lock_prdt_page_free_from_discard(
lock_t* lock;
lock_t* next_lock;
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
lock = lock_sys.get_first(*lock_hash, block->page.id());

70
storage/innobase/lock/lock0wait.cc

@ -45,7 +45,7 @@ void
lock_wait_table_print(void)
/*=======================*/
{
ut_ad(lock_wait_mutex_own());
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
const srv_slot_t* slot = lock_sys.waiting_threads;
@ -74,7 +74,7 @@ lock_wait_table_release_slot(
srv_slot_t* upper = lock_sys.waiting_threads + srv_max_n_threads;
#endif /* UNIV_DEBUG */
lock_wait_mutex_enter();
mysql_mutex_lock(&lock_sys.wait_mutex);
ut_ad(slot->in_use);
ut_ad(slot->thr != NULL);
@ -92,13 +92,13 @@ lock_wait_table_release_slot(
trx_t::mutex. To reduce contention on the lock mutex when reserving the
slot we avoid acquiring the lock mutex. */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
slot->thr->slot = NULL;
slot->thr = NULL;
slot->in_use = FALSE;
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
/* Scan backwards and adjust the last free slot pointer. */
for (slot = lock_sys.last_slot;
@ -119,7 +119,7 @@ lock_wait_table_release_slot(
ut_ad(lock_sys.last_slot >= lock_sys.waiting_threads);
ut_ad(lock_sys.last_slot <= upper);
lock_wait_mutex_exit();
mysql_mutex_unlock(&lock_sys.wait_mutex);
}
/*********************************************************************//**
@ -136,23 +136,15 @@ lock_wait_table_reserve_slot(
ulint i;
srv_slot_t* slot;
ut_ad(lock_wait_mutex_own());
ut_ad(trx_mutex_own(thr_get_trx(thr)));
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
slot = lock_sys.waiting_threads;
for (i = srv_max_n_threads; i--; ++slot) {
if (!slot->in_use) {
slot->in_use = TRUE;
slot->in_use = true;
slot->thr = thr;
slot->thr->slot = slot;
if (slot->event == NULL) {
slot->event = os_event_create(0);
ut_a(slot->event);
}
os_event_reset(slot->event);
slot->suspend_time = time(NULL);
slot->wait_timeout = wait_timeout;
@ -198,15 +190,15 @@ wsrep_is_BF_lock_timeout(
ib::info() << "WSREP: BF lock wait long for trx:" << ib::hex(trx->id)
<< " query: " << wsrep_thd_query(trx->mysql_thd);
if (!locked) {
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
}
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
trx_print_latched(stderr, trx, 3000);
if (!locked) {
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
srv_print_innodb_monitor = TRUE;
@ -247,9 +239,8 @@ lock_wait_suspend_thread(
innodb_lock_wait_timeout, because trx->mysql_thd == NULL. */
lock_wait_timeout = trx_lock_wait_timeout_get(trx);
lock_wait_mutex_enter();
trx_mutex_enter(trx);
mysql_mutex_lock(&lock_sys.wait_mutex);
trx->mutex.wr_lock();
trx->error_state = DB_SUCCESS;
@ -266,8 +257,8 @@ lock_wait_suspend_thread(
trx->lock.was_chosen_as_deadlock_victim = false;
}
lock_wait_mutex_exit();
trx_mutex_exit(trx);
mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock();
return;
}
@ -275,8 +266,8 @@ lock_wait_suspend_thread(
slot = lock_wait_table_reserve_slot(thr, lock_wait_timeout);
lock_wait_mutex_exit();
trx_mutex_exit(trx);
mysql_mutex_unlock(&lock_sys.wait_mutex);
trx->mutex.wr_unlock();
ulonglong start_time = 0;
@ -293,12 +284,12 @@ lock_wait_suspend_thread(
current thread which owns the transaction. Only acquire the
mutex if the wait_lock is still active. */
if (const lock_t* wait_lock = trx->lock.wait_lock) {
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
wait_lock = trx->lock.wait_lock;
if (wait_lock) {
lock_type = lock_get_type_low(wait_lock);
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
ulint had_dict_lock = trx->dict_operation_lock_mode;
@ -334,7 +325,11 @@ lock_wait_suspend_thread(
thd_wait_begin(trx->mysql_thd, THD_WAIT_TABLE_LOCK);
}
os_event_wait(slot->event);
mysql_mutex_lock(&lock_sys.mutex);
while (trx->lock.wait_lock) {
mysql_cond_wait(&slot->cond, &lock_sys.mutex);
}
mysql_mutex_unlock(&lock_sys.mutex);
thd_wait_end(trx->mysql_thd);
@ -408,8 +403,7 @@ lock_wait_release_thread_if_suspended(
que_thr_t* thr) /*!< in: query thread associated with the
user OS thread */
{
ut_ad(lock_mutex_own());
ut_ad(trx_mutex_own(thr_get_trx(thr)));
mysql_mutex_assert_owner(&lock_sys.mutex);
/* We own both the lock mutex and the trx_t::mutex but not the
lock wait mutex. This is OK because other threads will see the state
@ -425,7 +419,7 @@ lock_wait_release_thread_if_suspended(
trx->lock.was_chosen_as_deadlock_victim = false;
}
os_event_set(thr->slot->event);
mysql_cond_signal(&thr->slot->cond);
}
}
@ -439,7 +433,7 @@ lock_wait_check_and_cancel(
const srv_slot_t* slot) /*!< in: slot reserved by a user
thread when the wait started */
{
ut_ad(lock_wait_mutex_own());
mysql_mutex_assert_owner(&lock_sys.wait_mutex);
ut_ad(slot->in_use);
double wait_time = difftime(time(NULL), slot->suspend_time);
@ -457,7 +451,7 @@ lock_wait_check_and_cancel(
possible that the lock has already been
granted: in that case do nothing */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
if (trx->lock.wait_lock != NULL) {
ut_a(trx->lock.que_state == TRX_QUE_LOCK_WAIT);
@ -465,22 +459,22 @@ lock_wait_check_and_cancel(
#ifdef WITH_WSREP
if (!wsrep_is_BF_lock_timeout(trx)) {
#endif /* WITH_WSREP */
mutex_enter(&trx->mutex);
trx->mutex.wr_lock();
lock_cancel_waiting_and_release(trx->lock.wait_lock);
mutex_exit(&trx->mutex);
trx->mutex.wr_unlock();
#ifdef WITH_WSREP
}
#endif /* WITH_WSREP */
}
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
}
}
/** A task which wakes up threads whose lock wait may have lasted too long */
void lock_wait_timeout_task(void*)
{
lock_wait_mutex_enter();
mysql_mutex_lock(&lock_sys.wait_mutex);
/* Check all slots for user threads that are waiting
on locks, and if they have exceeded the time limit. */
@ -503,5 +497,5 @@ void lock_wait_timeout_task(void*)
else
lock_sys.timeout_timer_active= false;
lock_wait_mutex_exit();
mysql_mutex_unlock(&lock_sys.wait_mutex);
}

3
storage/innobase/log/log0log.cc

@ -1003,6 +1003,7 @@ ATTRIBUTE_COLD void logs_empty_and_mark_files_at_shutdown()
dict_stats_shutdown();
btr_defragment_shutdown();
ut_d(srv_master_thread_enable());
srv_shutdown_state = SRV_SHUTDOWN_CLEANUP;
if (srv_buffer_pool_dump_at_shutdown &&
@ -1077,7 +1078,7 @@ wait_suspend_loop:
ut_ad(!srv_any_background_activity());
if (srv_n_fil_crypt_threads_started) {
os_event_set(fil_crypt_threads_event);
fil_crypt_threads_signal(true);
thread_name = "fil_crypt_thread";
goto wait_suspend_loop;
}

246
storage/innobase/log/log0recv.cc

@ -54,6 +54,7 @@ Created 9/20/1997 Heikki Tuuri
#include "buf0rea.h"
#include "srv0srv.h"
#include "srv0start.h"
#include "sync0sync.h"
#include "fil0pagecompress.h"
/** Read-ahead area in applying log records to file pages */
@ -260,7 +261,7 @@ public:
record_corrupted:
if (!srv_force_recovery)
{
recv_sys.found_corrupt_log= true;
recv_sys.set_corrupt_log();
return applied;
}
next_not_same_page:
@ -312,7 +313,7 @@ public:
{
page_corrupted:
ib::error() << "Set innodb_force_recovery=1 to ignore corruption.";
recv_sys.found_corrupt_log= true;
recv_sys.set_corrupt_log();
return applied;
}
break;
@ -627,7 +628,7 @@ public:
@return whether the state was changed */
bool add(const page_id_t page_id, lsn_t lsn)
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
const init init = { lsn, false };
std::pair<map::iterator, bool> p = inits.insert(
map::value_type(page_id, init));
@ -646,7 +647,7 @@ public:
not valid after releasing recv_sys.mutex. */
init& last(page_id_t page_id)
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
return inits.find(page_id)->second;
}
@ -656,7 +657,7 @@ public:
@return whether page_id will be freed or initialized after lsn */
bool will_avoid_read(page_id_t page_id, lsn_t lsn) const
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
auto i= inits.find(page_id);
return i != inits.end() && i->second.lsn > lsn;
}
@ -664,7 +665,7 @@ public:
/** At the end of each recovery batch, reset the 'created' flags. */
void reset()
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
ut_ad(recv_no_ibuf_operations);
for (map::value_type& i : inits) {
i.second.created = false;
@ -677,7 +678,7 @@ public:
@param[in,out] mtr dummy mini-transaction */
void mark_ibuf_exist(mtr_t& mtr)
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
mtr.start();
for (const map::value_type& i : inits) {
@ -708,12 +709,12 @@ public:
mtr.start();
continue;
}
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
block->page.ibuf_exist = ibuf_page_exists(
block->page.id(), block->zip_size());
mtr.commit();
mtr.start();
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
}
}
@ -736,7 +737,7 @@ inline void recv_sys_t::trim(const page_id_t page_id, lsn_t lsn)
DBUG_LOG("ib_log",
"discarding log beyond end of tablespace "
<< page_id << " before LSN " << lsn);
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
for (recv_sys_t::map::iterator p = pages.lower_bound(page_id);
p != pages.end() && p->first.space() == page_id.space();) {
recv_sys_t::map::iterator r = p++;
@ -854,7 +855,7 @@ same_space:
<< " has been found in two places: '"
<< f.name << "' and '" << name << "'."
" You must delete one of them.";
recv_sys.found_corrupt_fs = true;
recv_sys.set_corrupt_fs();
}
break;
@ -906,7 +907,7 @@ same_space:
" disk is broken, and you cannot"
" remove the .ibd file, you can set"
" --innodb_force_recovery.";
recv_sys.found_corrupt_fs = true;
recv_sys.set_corrupt_fs();
break;
}
@ -927,9 +928,9 @@ void recv_sys_t::close()
if (is_initialised())
{
dblwr.pages.clear();
ut_d(mutex_enter(&mutex));
ut_d(mysql_mutex_lock(&mutex));
clear();
ut_d(mutex_exit(&mutex));
ut_d(mysql_mutex_unlock(&mutex));
if (buf)
{
@ -938,7 +939,8 @@ void recv_sys_t::close()
}
last_stored_lsn= 0;
mutex_free(&mutex);
mysql_mutex_destroy(&mutex);
mysql_cond_destroy(&cond);
}
recv_spaces.clear();
@ -952,7 +954,8 @@ void recv_sys_t::create()
{
ut_ad(this == &recv_sys);
ut_ad(!is_initialised());
mutex_create(LATCH_ID_RECV_SYS, &mutex);
mysql_mutex_init(recv_sys_mutex_key, &mutex, nullptr);
mysql_cond_init(0, &cond, nullptr);
apply_log_recs = false;
apply_batch_on = false;
@ -980,7 +983,7 @@ void recv_sys_t::create()
/** Clear a fully processed set of stored redo log records. */
inline void recv_sys_t::clear()
{
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
apply_log_recs= false;
apply_batch_on= false;
ut_ad(!after_apply || !UT_LIST_GET_LAST(blocks));
@ -995,6 +998,8 @@ inline void recv_sys_t::clear()
buf_block_free(block);
block= prev_block;
}
mysql_cond_broadcast(&cond);
}
/** Free most recovery data structures. */
@ -1002,7 +1007,7 @@ void recv_sys_t::debug_free()
{
ut_ad(this == &recv_sys);
ut_ad(is_initialised());
mutex_enter(&mutex);
mysql_mutex_lock(&mutex);
recovery_on= false;
pages.clear();
@ -1010,12 +1015,12 @@ void recv_sys_t::debug_free()
buf= nullptr;
mutex_exit(&mutex);
mysql_mutex_unlock(&mutex);
}
inline void *recv_sys_t::alloc(size_t len)
{
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
ut_ad(len);
ut_ad(len <= srv_page_size);
@ -1059,7 +1064,7 @@ inline void recv_sys_t::free(const void *data)
{
ut_ad(!ut_align_offset(data, ALIGNMENT));
data= page_align(data);
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
/* MDEV-14481 FIXME: To prevent race condition with buf_pool.resize(),
we must acquire and hold the buffer pool mutex here. */
@ -1176,7 +1181,7 @@ fail:
if (dl < LOG_BLOCK_HDR_SIZE
|| (dl != OS_FILE_LOG_BLOCK_SIZE
&& dl > log_sys.trailer_offset())) {
recv_sys.found_corrupt_log = true;
recv_sys.set_corrupt_log();
goto fail;
}
}
@ -1638,7 +1643,7 @@ inline bool page_recv_t::trim(lsn_t start_lsn)
inline void page_recv_t::recs_t::clear()
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
for (const log_rec_t *l= head; l; )
{
const log_rec_t *next= l->next;
@ -1668,7 +1673,7 @@ inline void recv_sys_t::add(const page_id_t page_id,
lsn_t start_lsn, lsn_t lsn, const byte *l,
size_t len)
{
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
std::pair<map::iterator, bool> p= pages.emplace(map::value_type
(page_id, page_recv_t()));
page_recv_t& recs= p.first->second;
@ -1755,7 +1760,7 @@ or corruption was noticed */
bool recv_sys_t::parse(lsn_t checkpoint_lsn, store_t *store, bool apply)
{
mysql_mutex_assert_owner(&log_sys.mutex);
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
ut_ad(parse_start_lsn);
ut_ad(log_sys.is_physical());
@ -2203,10 +2208,10 @@ same_page:
fn2 ? static_cast<ulint>(fn2end - fn2) : 0);
if (!fn2 || !apply);
else if (!fil_op_replay_rename(space_id, fn, fn2))
else if (UNIV_UNLIKELY(!fil_op_replay_rename(space_id, fn, fn2)))
found_corrupt_fs= true;
const_cast<char&>(fn[rlen])= saved_end;
if (UNIV_UNLIKELY(found_corrupt_fs))
if (is_corrupt_fs())
return true;
}
}
@ -2235,7 +2240,7 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
fil_space_t* space = NULL,
mlog_init_t::init* init = NULL)
{
ut_ad(mutex_own(&recv_sys.mutex));
mysql_mutex_assert_owner(&recv_sys.mutex);
ut_ad(recv_sys.apply_log_recs);
ut_ad(recv_needed_recovery);
ut_ad(!init || init->created);
@ -2255,7 +2260,7 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
p->second.state = page_recv_t::RECV_BEING_PROCESSED;
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
byte *frame = UNIV_LIKELY_NULL(block->page.zip.data)
? block->page.zip.data
@ -2372,7 +2377,7 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
}
set_start_lsn:
if (recv_sys.found_corrupt_log && !srv_force_recovery) {
if (recv_sys.is_corrupt_log() && !srv_force_recovery) {
break;
}
@ -2413,7 +2418,7 @@ set_start_lsn:
time_t now = time(NULL);
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
if (recv_max_page_lsn < page_lsn) {
recv_max_page_lsn = page_lsn;
@ -2435,14 +2440,41 @@ This function should only be called when innodb_force_recovery is set.
@param page_id corrupted page identifier */
ATTRIBUTE_COLD void recv_sys_t::free_corrupted_page(page_id_t page_id)
{
mutex_enter(&mutex);
mysql_mutex_lock(&mutex);
map::iterator p= pages.find(page_id);
if (p != pages.end())
{
p->second.log.clear();
pages.erase(p);
}
mutex_exit(&mutex);
if (pages.empty())
mysql_cond_broadcast(&cond);
mysql_mutex_unlock(&mutex);
}
/** Possibly finish a recovery batch. */
inline void recv_sys_t::maybe_finish_batch()
{
mysql_mutex_assert_owner(&mutex);
ut_ad(recovery_on);
if (!apply_batch_on || pages.empty() || is_corrupt_log() || is_corrupt_fs())
mysql_cond_broadcast(&cond);
}
ATTRIBUTE_COLD void recv_sys_t::set_corrupt_log()
{
mysql_mutex_lock(&mutex);
found_corrupt_log= true;
mysql_cond_broadcast(&cond);
mysql_mutex_unlock(&mutex);
}
ATTRIBUTE_COLD void recv_sys_t::set_corrupt_fs()
{
mysql_mutex_lock(&mutex);
found_corrupt_fs= true;
mysql_cond_broadcast(&cond);
mysql_mutex_unlock(&mutex);
}
/** Apply any buffered redo log to a page that was just read from a data file.
@ -2466,7 +2498,7 @@ void recv_recover_page(fil_space_t* space, buf_page_t* bpage)
buf_block_buf_fix_inc(block);
mtr.memo_push(block, MTR_MEMO_PAGE_X_FIX);
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
if (recv_sys.apply_log_recs) {
recv_sys_t::map::iterator p = recv_sys.pages.find(bpage->id());
if (p != recv_sys.pages.end()
@ -2474,13 +2506,14 @@ void recv_recover_page(fil_space_t* space, buf_page_t* bpage)
recv_recover_page(block, mtr, p, space);
p->second.log.clear();
recv_sys.pages.erase(p);
recv_sys.maybe_finish_batch();
goto func_exit;
}
}
mtr.commit();
func_exit:
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
ut_ad(mtr.has_committed());
}
@ -2508,10 +2541,10 @@ static void recv_read_in_area(page_id_t page_id)
}
if (p != page_nos) {
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
buf_read_recv_pages(page_id.space(), page_nos,
ulint(p - page_nos));
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
}
}
@ -2525,7 +2558,7 @@ inline buf_block_t *recv_sys_t::recover_low(const page_id_t page_id,
map::iterator &p, mtr_t &mtr,
buf_block_t *b)
{
ut_ad(mutex_own(&mutex));
mysql_mutex_assert_owner(&mutex);
ut_ad(p->first == page_id);
page_recv_t &recs= p->second;
ut_ad(recs.state == page_recv_t::RECV_WILL_NOT_READ);
@ -2547,19 +2580,21 @@ inline buf_block_t *recv_sys_t::recover_low(const page_id_t page_id,
being read in. Before buf_page_get_with_no_latch() returned to
buf_page_create(), all changes must have been applied to the
page already. */
ut_ad(recv_sys.pages.find(page_id) == recv_sys.pages.end());
ut_ad(pages.find(page_id) == pages.end());
mtr.commit();
block= nullptr;
}
else
{
ut_ad(&recs == &recv_sys.pages.find(page_id)->second);
ut_ad(&recs == &pages.find(page_id)->second);
i.created= true;
recv_recover_page(block, mtr, p, space, &i);
ut_ad(mtr.has_committed());
recs.log.clear();
map::iterator r= p++;
recv_sys.pages.erase(r);
pages.erase(r);
if (pages.empty())
mysql_cond_signal(&cond);
}
space->release();
}
@ -2575,7 +2610,7 @@ buf_block_t *recv_sys_t::recover_low(const page_id_t page_id)
buf_block_t *free_block= buf_LRU_get_free_block(false);
buf_block_t *block= nullptr;
mutex_enter(&mutex);
mysql_mutex_lock(&mutex);
map::iterator p= pages.find(page_id);
if (p != pages.end() && p->second.state == page_recv_t::RECV_WILL_NOT_READ)
@ -2585,7 +2620,7 @@ buf_block_t *recv_sys_t::recover_low(const page_id_t page_id)
ut_ad(!block || block == free_block);
}
mutex_exit(&mutex);
mysql_mutex_unlock(&mutex);
if (UNIV_UNLIKELY(!block))
buf_pool.free_block(free_block);
return block;
@ -2599,24 +2634,34 @@ void recv_sys_t::apply(bool last_batch)
srv_operation == SRV_OPERATION_RESTORE ||
srv_operation == SRV_OPERATION_RESTORE_EXPORT);
mutex_enter(&mutex);
#ifdef SAFE_MUTEX
DBUG_ASSERT(!last_batch == mysql_mutex_is_owner(&log_sys.mutex));
#endif /* SAFE_MUTEX */
mysql_mutex_lock(&mutex);
timespec abstime;
while (apply_batch_on)
{
bool abort= found_corrupt_log;
mutex_exit(&mutex);
if (abort)
if (is_corrupt_log())
{
mysql_mutex_unlock(&mutex);
return;
os_thread_sleep(500000);
mutex_enter(&mutex);
}
if (last_batch)
{
mysql_mutex_assert_not_owner(&log_sys.mutex);
mysql_cond_wait(&cond, &mutex);
}
else
{
mysql_mutex_unlock(&mutex);
set_timespec_nsec(abstime, 500000000ULL); /* 0.5s */
mysql_cond_timedwait(&cond, &log_sys.mutex, &abstime);
mysql_mutex_lock(&mutex);
}
}
#ifdef SAFE_MUTEX
DBUG_ASSERT(!last_batch == mysql_mutex_is_owner(&log_sys.mutex));
#endif /* SAFE_MUTEX */
recv_no_ibuf_operations = !last_batch ||
srv_operation == SRV_OPERATION_RESTORE ||
srv_operation == SRV_OPERATION_RESTORE_EXPORT;
@ -2660,9 +2705,9 @@ void recv_sys_t::apply(bool last_batch)
case page_recv_t::RECV_WILL_NOT_READ:
if (UNIV_LIKELY(!!recover_low(page_id, p, mtr, free_block)))
{
mutex_exit(&mutex);
mysql_mutex_unlock(&mutex);
free_block= buf_LRU_get_free_block(false);
mutex_enter(&mutex);
mysql_mutex_lock(&mutex);
next_page:
p= pages.lower_bound(page_id);
}
@ -2697,17 +2742,26 @@ next_page:
/* Wait until all the pages have been processed */
while (!pages.empty())
{
const bool abort= found_corrupt_log || found_corrupt_fs;
if (found_corrupt_fs && !srv_force_recovery)
if (!is_corrupt_fs() && !is_corrupt_log())
{
if (last_batch)
{
mysql_mutex_assert_not_owner(&log_sys.mutex);
mysql_cond_wait(&cond, &mutex);
}
else
{
mysql_mutex_unlock(&mutex);
set_timespec_nsec(abstime, 500000000ULL); /* 0.5s */
mysql_cond_timedwait(&cond, &log_sys.mutex, &abstime);
mysql_mutex_lock(&mutex);
}
continue;
}
if (is_corrupt_fs() && !srv_force_recovery)
ib::info() << "Set innodb_force_recovery=1 to ignore corrupted pages.";
mutex_exit(&mutex);
if (abort)
return;
os_thread_sleep(500000);
mutex_enter(&mutex);
mysql_mutex_unlock(&mutex);
return;
}
}
@ -2721,7 +2775,7 @@ next_page:
}
mysql_mutex_assert_not_owner(&log_sys.mutex);
mutex_exit(&mutex);
mysql_mutex_unlock(&mutex);
/* Instead of flushing, last_batch could sort the buf_pool.flush_list
in ascending order of buf_page_t::oldest_modification. */
@ -2733,11 +2787,11 @@ next_page:
mysql_mutex_lock(&log_sys.mutex);
}
mutex_enter(&mutex);
mysql_mutex_lock(&mutex);
ut_d(after_apply= true);
clear();
mutex_exit(&mutex);
mysql_mutex_unlock(&mutex);
}
/** Check whether the number of read redo log blocks exceeds the maximum.
@ -2968,7 +3022,7 @@ static bool recv_scan_log_recs(
ib::error() << "Log parsing buffer overflow."
" Recovery may have failed!";
recv_sys.found_corrupt_log = true;
recv_sys.set_corrupt_log();
if (!srv_force_recovery) {
ib::error()
@ -2976,7 +3030,7 @@ static bool recv_scan_log_recs(
" to ignore this error.";
return(true);
}
} else if (!recv_sys.found_corrupt_log) {
} else if (!recv_sys.is_corrupt_log()) {
more_data = recv_sys_add_to_parsing_buf(
log_block, scanned_lsn);
}
@ -3004,13 +3058,13 @@ static bool recv_scan_log_recs(
*group_scanned_lsn = scanned_lsn;
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
if (more_data && !recv_sys.found_corrupt_log) {
if (more_data && !recv_sys.is_corrupt_log()) {
/* Try to parse more log records */
if (recv_sys.parse(checkpoint_lsn, store, apply)) {
ut_ad(recv_sys.found_corrupt_log
|| recv_sys.found_corrupt_fs
ut_ad(recv_sys.is_corrupt_log()
|| recv_sys.is_corrupt_fs()
|| recv_sys.mlog_checkpoint_lsn
== recv_sys.recovered_lsn);
finished = true;
@ -3035,7 +3089,8 @@ static bool recv_scan_log_recs(
}
func_exit:
mutex_exit(&recv_sys.mutex);
recv_sys.maybe_finish_batch();
mysql_mutex_unlock(&recv_sys.mutex);
return(finished);
}
@ -3057,7 +3112,7 @@ recv_group_scan_log_recs(
DBUG_ENTER("recv_group_scan_log_recs");
DBUG_ASSERT(!last_phase || recv_sys.mlog_checkpoint_lsn > 0);
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
recv_sys.len = 0;
recv_sys.recovered_offset = 0;
recv_sys.clear();
@ -3066,7 +3121,7 @@ recv_group_scan_log_recs(
recv_sys.recovered_lsn = *contiguous_lsn;
recv_sys.scanned_checkpoint_no = 0;
ut_ad(recv_max_page_lsn == 0);
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
lsn_t start_lsn;
lsn_t end_lsn;
@ -3094,7 +3149,7 @@ recv_group_scan_log_recs(
start_lsn, end_lsn, contiguous_lsn,
&log_sys.log.scanned_lsn));
if (recv_sys.found_corrupt_log || recv_sys.found_corrupt_fs) {
if (recv_sys.is_corrupt_log() || recv_sys.is_corrupt_fs()) {
DBUG_RETURN(false);
}
@ -3157,7 +3212,7 @@ recv_validate_tablespace(bool rescan, bool& missing_tablespace)
{
dberr_t err = DB_SUCCESS;
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
for (recv_sys_t::map::iterator p = recv_sys.pages.begin();
p != recv_sys.pages.end();) {
@ -3190,7 +3245,7 @@ next:
if (err != DB_SUCCESS) {
func_exit:
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
return(err);
}
@ -3269,7 +3324,7 @@ recv_init_crash_recovery_spaces(bool rescan, bool& missing_tablespace)
ib::error() << "Missing FILE_CREATE, FILE_DELETE"
" or FILE_MODIFY before FILE_CHECKPOINT"
" for tablespace " << rs.first;
recv_sys.found_corrupt_log = true;
recv_sys.set_corrupt_log();
return(DB_CORRUPTION);
} else {
rs.second.status = file_name_t::MISSING;
@ -3360,7 +3415,7 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
contiguous_lsn = end_lsn;
break;
}
recv_sys.found_corrupt_log = true;
recv_sys.set_corrupt_log();
mysql_mutex_unlock(&log_sys.mutex);
return(DB_ERROR);
}
@ -3376,14 +3431,14 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
recv_group_scan_log_recs(checkpoint_lsn, &contiguous_lsn, false);
/* The first scan should not have stored or applied any records. */
ut_ad(recv_sys.pages.empty());
ut_ad(!recv_sys.found_corrupt_fs);
ut_ad(!recv_sys.is_corrupt_fs());
if (srv_read_only_mode && recv_needed_recovery) {
mysql_mutex_unlock(&log_sys.mutex);
return(DB_READ_ONLY);
}
if (recv_sys.found_corrupt_log && !srv_force_recovery) {
if (recv_sys.is_corrupt_log() && !srv_force_recovery) {
mysql_mutex_unlock(&log_sys.mutex);
ib::warn() << "Log scan aborted at LSN " << contiguous_lsn;
return(DB_ERROR);
@ -3409,8 +3464,8 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
rescan = recv_group_scan_log_recs(
checkpoint_lsn, &contiguous_lsn, false);
if ((recv_sys.found_corrupt_log && !srv_force_recovery)
|| recv_sys.found_corrupt_fs) {
if ((recv_sys.is_corrupt_log() && !srv_force_recovery)
|| recv_sys.is_corrupt_fs()) {
mysql_mutex_unlock(&log_sys.mutex);
return(DB_ERROR);
}
@ -3494,11 +3549,11 @@ completed:
rescan = recv_group_scan_log_recs(
checkpoint_lsn, &recent_stored_lsn, false);
ut_ad(!recv_sys.found_corrupt_fs);
ut_ad(!recv_sys.is_corrupt_fs());
missing_tablespace = false;
err = recv_sys.found_corrupt_log
err = recv_sys.is_corrupt_log()
? DB_ERROR
: recv_validate_tablespace(
rescan, missing_tablespace);
@ -3525,9 +3580,9 @@ completed:
recv_group_scan_log_recs(
checkpoint_lsn, &contiguous_lsn, true);
if ((recv_sys.found_corrupt_log
if ((recv_sys.is_corrupt_log()
&& !srv_force_recovery)
|| recv_sys.found_corrupt_fs) {
|| recv_sys.is_corrupt_fs()) {
mysql_mutex_unlock(&log_sys.mutex);
return(DB_ERROR);
}
@ -3581,12 +3636,9 @@ completed:
log_sys.next_checkpoint_no = ++checkpoint_no;
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
recv_sys.apply_log_recs = true;
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
mysql_mutex_unlock(&log_sys.mutex);
recv_lsn_checks_on = true;

8
storage/innobase/os/os0file.cc

@ -49,7 +49,6 @@ Created 10/21/1995 Heikki Tuuri
#ifdef HAVE_LINUX_UNISTD_H
#include "unistd.h"
#endif
#include "os0event.h"
#include "os0thread.h"
#include "buf0dblwr.h"
@ -152,12 +151,7 @@ static ulint os_innodb_umask = 0;
#endif /* _WIN32 */
#ifdef WITH_INNODB_DISALLOW_WRITES
#define WAIT_ALLOW_WRITES() os_event_wait(srv_allow_writes_event)
#else
#define WAIT_ALLOW_WRITES() do { } while (0)
#endif /* WITH_INNODB_DISALLOW_WRITES */
#define WAIT_ALLOW_WRITES() innodb_wait_allow_writes()
ulint os_n_file_reads;
static ulint os_bytes_read_since_printout;

29
storage/innobase/que/que0que.cc

@ -182,8 +182,7 @@ que_thr_end_lock_wait(
{
que_thr_t* thr;
ut_ad(lock_mutex_own());
ut_ad(trx_mutex_own(trx));
mysql_mutex_assert_owner(&lock_sys.mutex);
thr = trx->lock.wait_thr;
@ -231,7 +230,7 @@ que_fork_scheduler_round_robin(
que_fork_t* fork, /*!< in: a query fork */
que_thr_t* thr) /*!< in: current pos */
{
trx_mutex_enter(fork->trx);
fork->trx->mutex.wr_lock();
/* If no current, start first available. */
if (thr == NULL) {
@ -261,7 +260,7 @@ que_fork_scheduler_round_robin(
}
}
trx_mutex_exit(fork->trx);
fork->trx->mutex.wr_unlock();
return(thr);
}
@ -592,7 +591,7 @@ que_thr_node_step(
auto mutex = &thr->graph->trx->mutex;
mutex_enter(mutex);
mutex->wr_lock();
if (!que_thr_peek_stop(thr)) {
/* Thread execution completed */
@ -600,7 +599,7 @@ que_thr_node_step(
thr = NULL;
}
mutex_exit(mutex);
mutex->wr_unlock();
return(thr);
}
@ -618,8 +617,6 @@ que_thr_stop(
graph = thr->graph;
ut_ad(trx_mutex_own(trx));
if (graph->state == QUE_FORK_COMMAND_WAIT) {
thr->state = QUE_THR_SUSPENDED;
@ -671,7 +668,6 @@ que_thr_dec_refer_count(
trx = thr_get_trx(thr);
ut_a(thr->is_active);
ut_ad(trx_mutex_own(trx));
if (thr->state == QUE_THR_RUNNING) {
@ -721,7 +717,7 @@ que_thr_stop_for_mysql(
trx = thr_get_trx(thr);
trx_mutex_enter(trx);
trx->mutex.wr_lock();
if (thr->state == QUE_THR_RUNNING) {
switch (trx->error_state) {
@ -742,7 +738,7 @@ que_thr_stop_for_mysql(
ut_d(thr->set_active(false));
thr->is_active= false;
func_exit:
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
}
#ifdef UNIV_DEBUG
@ -980,7 +976,6 @@ que_run_threads_low(
ut_ad(thr->state == QUE_THR_RUNNING);
ut_a(thr_get_trx(thr)->error_state == DB_SUCCESS);
ut_ad(!trx_mutex_own(thr_get_trx(thr)));
/* cumul_resource counts how much resources the OS thread (NOT the
query thread) has spent in this function */
@ -1009,9 +1004,9 @@ que_run_threads_low(
/* This can change next_thr to a non-NULL value
if there was a lock wait that already completed. */
mutex_enter(&trx->mutex);
trx->mutex.wr_lock();
que_thr_dec_refer_count(thr, &next_thr);
mutex_exit(&trx->mutex);
trx->mutex.wr_unlock();
if (next_thr != NULL) {
thr = next_thr;
@ -1029,8 +1024,6 @@ que_run_threads(
/*============*/
que_thr_t* thr) /*!< in: query thread */
{
ut_ad(!trx_mutex_own(thr_get_trx(thr)));
loop:
ut_a(thr_get_trx(thr)->error_state == DB_SUCCESS);
@ -1054,7 +1047,7 @@ loop:
lock_wait_suspend_thread(thr);
trx_t* trx = thr->graph->trx;
trx_mutex_enter(trx);
trx->mutex.wr_lock();
ut_ad(trx->id);
const dberr_t err = trx->error_state;
if (err != DB_SUCCESS) {
@ -1062,7 +1055,7 @@ loop:
a lock wait timeout */
que_thr_dec_refer_count(thr, NULL);
}
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
if (err == DB_SUCCESS) {
goto loop;

23
storage/innobase/row/row0ftsort.cc

@ -31,6 +31,7 @@ Created 10/13/2010 Jimmy Yang
#include "btr0cur.h"
#include "fts0plugin.h"
#include "log0crypt.h"
#include "sync0sync.h"
/** Read the next record to buffer N.
@param N index into array of merge info structure */
@ -216,7 +217,7 @@ row_fts_psort_info_init(
common_info->old_zip_size = old_zip_size;
common_info->trx = trx;
common_info->all_info = psort_info;
common_info->sort_event = os_event_create(0);
mysql_cond_init(0, &common_info->sort_cond, nullptr);
common_info->opt_doc_id_size = opt_doc_id_size;
if (log_tmp_is_encrypted()) {
@ -285,7 +286,8 @@ row_fts_psort_info_init(
psort_info[j].psort_common = common_info;
psort_info[j].error = DB_SUCCESS;
psort_info[j].memory_used = 0;
mutex_create(LATCH_ID_FTS_PLL_TOKENIZE, &psort_info[j].mutex);
mysql_mutex_init(fts_pll_tokenize_mutex_key,
&psort_info[j].mutex, nullptr);
}
/* Initialize merge_info structures parallel merge and insert
@ -332,10 +334,10 @@ row_fts_psort_info_destroy(
aligned_free(psort_info[j].crypt_block[i]);
}
mutex_free(&psort_info[j].mutex);
mysql_mutex_destroy(&psort_info[j].mutex);
}
os_event_destroy(merge_info[0].psort_common->sort_event);
mysql_cond_destroy(&merge_info[0].psort_common->sort_cond);
ut_free(merge_info[0].psort_common->dup);
ut_free(merge_info[0].psort_common);
ut_free(psort_info);
@ -721,7 +723,7 @@ row_merge_fts_get_next_doc_item(
ut_free(*doc_item);
}
mutex_enter(&psort_info->mutex);
mysql_mutex_lock(&psort_info->mutex);
*doc_item = UT_LIST_GET_FIRST(psort_info->fts_doc_list);
if (*doc_item != NULL) {
@ -733,7 +735,7 @@ row_merge_fts_get_next_doc_item(
+ (*doc_item)->field->len;
}
mutex_exit(&psort_info->mutex);
mysql_mutex_unlock(&psort_info->mutex);
}
/*********************************************************************//**
@ -1034,9 +1036,9 @@ func_exit:
mem_heap_free(blob_heap);
mutex_enter(&psort_info->mutex);
mysql_mutex_lock(&psort_info->mutex);
psort_info->error = error;
mutex_exit(&psort_info->mutex);
mysql_mutex_unlock(&psort_info->mutex);
if (UT_LIST_GET_LEN(psort_info->fts_doc_list) > 0) {
/* child can exit either with error or told by parent. */
@ -1049,9 +1051,10 @@ func_exit:
row_merge_fts_get_next_doc_item(psort_info, &doc_item);
} while (doc_item != NULL);
mysql_mutex_lock(&psort_info->mutex);
psort_info->child_status = FTS_CHILD_COMPLETE;
os_event_set(psort_info->psort_common->sort_event);
psort_info->child_status = FTS_CHILD_EXITING;
mysql_cond_signal(&psort_info->psort_common->sort_cond);
mysql_mutex_unlock(&psort_info->mutex);
}
/*********************************************************************//**

4
storage/innobase/row/row0ins.cc

@ -704,11 +704,11 @@ row_ins_foreign_trx_print(
ut_ad(!srv_read_only_mode);
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
n_rec_locks = lock_number_of_rows_locked(&trx->lock);
n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
heap_size = mem_heap_get_size(trx->lock.lock_heap);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
mutex_enter(&dict_foreign_err_mutex);
rewind(dict_foreign_err_file);

32
storage/innobase/row/row0merge.cc

@ -655,7 +655,7 @@ row_merge_buf_add(
*doc_id % fts_sort_pll_degree);
/* Add doc item to fts_doc_list */
mutex_enter(&psort_info[bucket].mutex);
mysql_mutex_lock(&psort_info[bucket].mutex);
if (psort_info[bucket].error == DB_SUCCESS) {
UT_LIST_ADD_LAST(
@ -667,7 +667,7 @@ row_merge_buf_add(
ut_free(doc_item);
}
mutex_exit(&psort_info[bucket].mutex);
mysql_mutex_unlock(&psort_info[bucket].mutex);
/* Sleep when memory used exceeds limit*/
while (psort_info[bucket].memory_used
@ -1696,9 +1696,7 @@ row_merge_read_clustered_index(
doc_id_t doc_id = 0;
doc_id_t max_doc_id = 0;
ibool add_doc_id = FALSE;
os_event_t fts_parallel_sort_event = NULL;
ibool fts_pll_sort = FALSE;
int64_t sig_count = 0;
mysql_cond_t* fts_parallel_sort_cond = nullptr;
index_tuple_info_t** sp_tuples = NULL;
mem_heap_t* sp_heap = NULL;
ulint num_spatial = 0;
@ -1774,10 +1772,9 @@ row_merge_read_clustered_index(
ut_ad(doc_id > 0);
}
fts_pll_sort = TRUE;
row_fts_start_psort(psort_info);
fts_parallel_sort_event =
psort_info[0].psort_common->sort_event;
fts_parallel_sort_cond =
&psort_info[0].psort_common->sort_cond;
} else {
if (dict_index_is_spatial(index[i])) {
num_spatial++;
@ -2738,7 +2735,7 @@ all_done:
#ifdef FTS_INTERNAL_DIAG_PRINT
DEBUG_FTS_SORT_PRINT("FTS_SORT: Complete Scan Table\n");
#endif
if (fts_pll_sort) {
if (UNIV_LIKELY_NULL(fts_parallel_sort_cond)) {
wait_again:
/* Check if error occurs in child thread */
for (ulint j = 0; j < fts_sort_pll_degree; j++) {
@ -2759,14 +2756,15 @@ wait_again:
}
/* Now wait all children to report back to be completed */
os_event_wait_time_low(fts_parallel_sort_event,
1000000, sig_count);
timespec abstime;
set_timespec(abstime, 1);
mysql_mutex_lock(&psort_info[0].mutex);
mysql_cond_timedwait(fts_parallel_sort_cond,
&psort_info[0].mutex, &abstime);
mysql_mutex_unlock(&psort_info[0].mutex);
for (ulint i = 0; i < fts_sort_pll_degree; i++) {
if (psort_info[i].child_status != FTS_CHILD_COMPLETE
&& psort_info[i].child_status != FTS_CHILD_EXITING) {
sig_count = os_event_reset(
fts_parallel_sort_event);
if (!psort_info[i].child_status) {
goto wait_again;
}
}
@ -4019,9 +4017,7 @@ pfs_os_file_t
row_merge_file_create_low(
const char* path)
{
#ifdef WITH_INNODB_DISALLOW_WRITES
os_event_wait(srv_allow_writes_event);
#endif /* WITH_INNODB_DISALLOW_WRITES */
innodb_wait_allow_writes();
if (!path) {
path = mysql_tmpdir;
}

4
storage/innobase/row/row0mysql.cc

@ -2610,9 +2610,9 @@ skip:
}
if (!srv_fast_shutdown && !trx_sys.any_active_transactions()) {
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
skip = UT_LIST_GET_LEN(table->locks) != 0;
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
if (skip) {
/* We cannot drop tables that are locked by XA
PREPARE transactions. */

10
storage/innobase/row/row0vers.cc

@ -77,7 +77,7 @@ index record.
@param[in] offsets rec_get_offsets(rec, index)
@param[in,out] mtr mini-transaction
@return the active transaction; state must be rechecked after
trx_mutex_enter(), and trx->release_reference() must be invoked
acquiring trx->mutex, and trx->release_reference() must be invoked
@retval NULL if the record was committed */
UNIV_INLINE
trx_t*
@ -193,10 +193,10 @@ row_vers_impl_x_locked_low(
heap, &prev_version, NULL,
dict_index_has_virtual(index) ? &vrow : NULL, 0);
trx_mutex_enter(trx);
ut_d(trx->mutex.wr_lock());
const bool committed = trx_state_eq(
trx, TRX_STATE_COMMITTED_IN_MEMORY);
trx_mutex_exit(trx);
ut_d(trx->mutex.wr_unlock());
/* The oldest visible clustered index version must not be
delete-marked, because we never start a transaction by
@ -381,7 +381,7 @@ index record.
@param[in] index secondary index
@param[in] offsets rec_get_offsets(rec, index)
@return the active transaction; state must be rechecked after
trx_mutex_enter(), and trx->release_reference() must be invoked
acquiring trx->mutex, and trx->release_reference() must be invoked
@retval NULL if the record was committed */
trx_t*
row_vers_impl_x_locked(
@ -395,7 +395,7 @@ row_vers_impl_x_locked(
const rec_t* clust_rec;
dict_index_t* clust_index;
ut_ad(!lock_mutex_own());
mysql_mutex_assert_not_owner(&lock_sys.mutex);
mtr_start(&mtr);

81
storage/innobase/srv/srv0srv.cc

@ -152,7 +152,7 @@ ulong innodb_compression_algorithm;
/** Used by SET GLOBAL innodb_master_thread_disabled_debug = X. */
my_bool srv_master_thread_disabled_debug;
/** Event used to inform that master thread is disabled. */
static os_event_t srv_master_thread_disabled_event;
static mysql_cond_t srv_master_thread_disabled_cond;
#endif /* UNIV_DEBUG */
/*------------------------- LOG FILES ------------------------ */
@ -194,10 +194,6 @@ srv_printf_innodb_monitor() will request mutex acquisition
with mutex_enter(), which will wait until it gets the mutex. */
#define MUTEX_NOWAIT(mutex_skipped) ((mutex_skipped) < MAX_MUTEX_NOWAIT)
#ifdef WITH_INNODB_DISALLOW_WRITES
UNIV_INTERN os_event_t srv_allow_writes_event;
#endif /* WITH_INNODB_DISALLOW_WRITES */
/** copy of innodb_buffer_pool_size */
ulint srv_buf_pool_size;
const ulint srv_buf_pool_min_size = 5 * 1024 * 1024;
@ -704,7 +700,7 @@ static void srv_init()
}
need_srv_free = true;
ut_d(srv_master_thread_disabled_event = os_event_create(0));
ut_d(mysql_cond_init(0, &srv_master_thread_disabled_cond, nullptr));
/* page_zip_stat_per_index_mutex is acquired from:
1. page_zip_compress()
@ -716,15 +712,6 @@ static void srv_init()
mutex_create(LATCH_ID_PAGE_ZIP_STAT_PER_INDEX,
&page_zip_stat_per_index_mutex);
#ifdef WITH_INNODB_DISALLOW_WRITES
/* Writes have to be enabled on init or else we hang. Thus, we
always set the event here regardless of innobase_disallow_writes.
That flag will always be 0 at this point because it isn't settable
via my.cnf or command line arg. */
srv_allow_writes_event = os_event_create(0);
os_event_set(srv_allow_writes_event);
#endif /* WITH_INNODB_DISALLOW_WRITES */
/* Initialize some INFORMATION SCHEMA internal structures */
trx_i_s_cache_init(trx_i_s_cache);
@ -747,7 +734,7 @@ srv_free(void)
mutex_free(&srv_sys.tasks_mutex);
}
ut_d(os_event_destroy(srv_master_thread_disabled_event));
ut_d(mysql_cond_destroy(&srv_master_thread_disabled_cond));
trx_i_s_cache_free(trx_i_s_cache);
srv_thread_pool_end();
@ -1399,7 +1386,7 @@ void srv_monitor_task(void*)
if (sync_array_print_long_waits(&waiter, &sema)
&& sema == old_sema && os_thread_eq(waiter, old_waiter)) {
#if defined(WITH_WSREP) && defined(WITH_INNODB_DISALLOW_WRITES)
if (!os_event_is_set(srv_allow_writes_event)) {
if (UNIV_UNLIKELY(innodb_disallow_writes)) {
fprintf(stderr,
"WSREP: avoiding InnoDB self crash due to "
"long semaphore wait of > %lu seconds\n"
@ -1636,26 +1623,17 @@ srv_shutdown_print_master_pending(
#ifdef UNIV_DEBUG
/** Waits in loop as long as master thread is disabled (debug) */
static
void
srv_master_do_disabled_loop(void)
static void srv_master_do_disabled_loop()
{
if (!srv_master_thread_disabled_debug) {
/* We return here to avoid changing op_info. */
return;
}
srv_main_thread_op_info = "disabled";
while (srv_master_thread_disabled_debug) {
os_event_set(srv_master_thread_disabled_event);
if (srv_shutdown_state != SRV_SHUTDOWN_NONE) {
break;
}
os_thread_sleep(100000);
}
srv_main_thread_op_info = "";
if (!srv_master_thread_disabled_debug)
return;
srv_main_thread_op_info = "disabled";
mysql_mutex_lock(&LOCK_global_system_variables);
while (srv_master_thread_disabled_debug)
mysql_cond_wait(&srv_master_thread_disabled_cond,
&LOCK_global_system_variables);
mysql_mutex_unlock(&LOCK_global_system_variables);
srv_main_thread_op_info = "";
}
/** Disables master thread. It's used by:
@ -1663,22 +1641,25 @@ srv_master_do_disabled_loop(void)
@param[in] save immediate result from check function */
void
srv_master_thread_disabled_debug_update(THD*, st_mysql_sys_var*, void*,
const void* save)
const void* save)
{
/* This method is protected by mutex, as every SET GLOBAL .. */
ut_ad(srv_master_thread_disabled_event != NULL);
const bool disable = *static_cast<const my_bool*>(save);
const int64_t sig_count = os_event_reset(
srv_master_thread_disabled_event);
srv_master_thread_disabled_debug = disable;
mysql_mutex_assert_owner(&LOCK_global_system_variables);
const bool disable= *static_cast<const my_bool*>(save);
srv_master_thread_disabled_debug= disable;
if (!disable)
mysql_cond_signal(&srv_master_thread_disabled_cond);
}
if (disable) {
os_event_wait_low(
srv_master_thread_disabled_event, sig_count);
}
/** Enable the master thread on shutdown. */
void srv_master_thread_enable()
{
if (srv_master_thread_disabled_debug)
{
mysql_mutex_lock(&LOCK_global_system_variables);
srv_master_thread_disabled_debug= FALSE;
mysql_cond_signal(&srv_master_thread_disabled_cond);
mysql_mutex_unlock(&LOCK_global_system_variables);
}
}
#endif /* UNIV_DEBUG */

11
storage/innobase/srv/srv0start.cc

@ -96,7 +96,6 @@ Created 2/16/1996 Heikki Tuuri
#include "row0row.h"
#include "row0mysql.h"
#include "btr0pcur.h"
#include "os0event.h"
#include "zlib.h"
#include "ut0crc32.h"
@ -825,7 +824,7 @@ static void srv_shutdown_threads()
{
ut_ad(!srv_undo_sources);
srv_shutdown_state = SRV_SHUTDOWN_EXIT_THREADS;
ut_d(srv_master_thread_enable());
lock_sys.timeout_timer.reset();
srv_master_timer.reset();
@ -1488,8 +1487,8 @@ file_checked:
recv_sys.apply(true);
if (recv_sys.found_corrupt_log
|| recv_sys.found_corrupt_fs) {
if (recv_sys.is_corrupt_log()
|| recv_sys.is_corrupt_fs()) {
return(srv_init_abort(DB_CORRUPTION));
}
@ -1922,7 +1921,7 @@ skip_monitors:
/* Create thread(s) that handles key rotation. This is
needed already here as log_preflush_pool_modified_pages
will flush dirty pages and that might need e.g.
fil_crypt_threads_event. */
fil_crypt_threads_cond. */
fil_system_enter();
fil_crypt_threads_init();
fil_system_exit();
@ -1941,6 +1940,8 @@ void srv_shutdown_bg_undo_sources()
{
srv_shutdown_state = SRV_SHUTDOWN_INITIATED;
ut_d(srv_master_thread_enable());
if (srv_undo_sources) {
ut_ad(!srv_read_only_mode);
fts_optimize_shutdown();

52
storage/innobase/sync/sync0debug.cc

@ -33,7 +33,7 @@ Created 2012-08-21 Sunny Bains
#include "sync0sync.h"
#include "sync0debug.h"
#include "srv0start.h"
#include "fil0fil.h"
#include "lock0lock.h"
#include <vector>
#include <string>
@ -397,17 +397,10 @@ LatchDebug::LatchDebug()
LEVEL_MAP_INSERT(SYNC_ANY_LATCH);
LEVEL_MAP_INSERT(SYNC_POOL);
LEVEL_MAP_INSERT(SYNC_POOL_MANAGER);
LEVEL_MAP_INSERT(SYNC_WORK_QUEUE);
LEVEL_MAP_INSERT(SYNC_FTS_TOKENIZE);
LEVEL_MAP_INSERT(SYNC_FTS_OPTIMIZE);
LEVEL_MAP_INSERT(SYNC_RECV);
LEVEL_MAP_INSERT(SYNC_PURGE_QUEUE);
LEVEL_MAP_INSERT(SYNC_TRX);
LEVEL_MAP_INSERT(SYNC_RW_TRX_HASH_ELEMENT);
LEVEL_MAP_INSERT(SYNC_READ_VIEW);
LEVEL_MAP_INSERT(SYNC_TRX_SYS);
LEVEL_MAP_INSERT(SYNC_LOCK_SYS);
LEVEL_MAP_INSERT(SYNC_LOCK_WAIT_SYS);
LEVEL_MAP_INSERT(SYNC_INDEX_ONLINE_LOG);
LEVEL_MAP_INSERT(SYNC_IBUF_BITMAP_MUTEX);
LEVEL_MAP_INSERT(SYNC_IBUF_MUTEX);
@ -628,12 +621,6 @@ LatchDebug::check_order(
case SYNC_NO_ORDER_CHECK:
break;
case SYNC_RECV:
case SYNC_WORK_QUEUE:
case SYNC_FTS_TOKENIZE:
case SYNC_FTS_OPTIMIZE:
case SYNC_LOCK_SYS:
case SYNC_LOCK_WAIT_SYS:
case SYNC_RW_TRX_HASH_ELEMENT:
case SYNC_READ_VIEW:
case SYNC_TRX_SYS:
@ -673,17 +660,6 @@ LatchDebug::check_order(
break;
case SYNC_TRX:
/* Either the thread must own the lock_sys.mutex, or
it is allowed to own only ONE trx_t::mutex. */
if (less(latches, level) != NULL) {
basic_check(latches, level, level - 1);
ut_a(find(latches, SYNC_LOCK_SYS) != 0);
}
break;
case SYNC_IBUF_PESS_INSERT_MUTEX:
ut_a(find(latches, SYNC_IBUF_MUTEX) == 0);
break;
@ -944,13 +920,6 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(FIL_SYSTEM, SYNC_ANY_LATCH, fil_system_mutex_key);
LATCH_ADD_MUTEX(FTS_DELETE, SYNC_FTS_OPTIMIZE, fts_delete_mutex_key);
LATCH_ADD_MUTEX(FTS_DOC_ID, SYNC_FTS_OPTIMIZE, fts_doc_id_mutex_key);
LATCH_ADD_MUTEX(FTS_PLL_TOKENIZE, SYNC_FTS_TOKENIZE,
fts_pll_tokenize_mutex_key);
LATCH_ADD_MUTEX(IBUF_BITMAP, SYNC_IBUF_BITMAP_MUTEX,
ibuf_bitmap_mutex_key);
@ -965,8 +934,6 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(RECALC_POOL, SYNC_STATS_AUTO_RECALC,
recalc_pool_mutex_key);
LATCH_ADD_MUTEX(RECV_SYS, SYNC_RECV, recv_sys_mutex_key);
LATCH_ADD_MUTEX(REDO_RSEG, SYNC_REDO_RSEG, redo_rseg_mutex_key);
LATCH_ADD_MUTEX(NOREDO_RSEG, SYNC_NOREDO_RSEG, noredo_rseg_mutex_key);
@ -992,13 +959,6 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(TRX_POOL_MANAGER, SYNC_POOL_MANAGER,
trx_pool_manager_mutex_key);
LATCH_ADD_MUTEX(TRX, SYNC_TRX, trx_mutex_key);
LATCH_ADD_MUTEX(LOCK_SYS, SYNC_LOCK_SYS, lock_mutex_key);
LATCH_ADD_MUTEX(LOCK_SYS_WAIT, SYNC_LOCK_WAIT_SYS,
lock_wait_mutex_key);
LATCH_ADD_MUTEX(TRX_SYS, SYNC_TRX_SYS, trx_sys_mutex_key);
LATCH_ADD_MUTEX(SRV_SYS_TASKS, SYNC_ANY_LATCH, srv_threads_mutex_key);
@ -1015,19 +975,9 @@ sync_latch_meta_init()
LATCH_ADD_MUTEX(INDEX_ONLINE_LOG, SYNC_INDEX_ONLINE_LOG,
index_online_log_key);
LATCH_ADD_MUTEX(WORK_QUEUE, SYNC_WORK_QUEUE, PFS_NOT_INSTRUMENTED);
/* JAN: TODO: Add PFS instrumentation */
LATCH_ADD_MUTEX(DEFRAGMENT_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(BTR_DEFRAGMENT_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(FIL_CRYPT_STAT_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(FIL_CRYPT_DATA_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(FIL_CRYPT_THREADS_MUTEX, SYNC_NO_ORDER_CHECK,
PFS_NOT_INSTRUMENTED);
LATCH_ADD_MUTEX(RW_TRX_HASH_ELEMENT, SYNC_RW_TRX_HASH_ELEMENT,
rw_trx_hash_element_mutex_key);
LATCH_ADD_MUTEX(READ_VIEW, SYNC_READ_VIEW, read_view_mutex_key);

1
storage/innobase/sync/sync0sync.cc

@ -65,7 +65,6 @@ mysql_pfs_key_t srv_innodb_monitor_mutex_key;
mysql_pfs_key_t srv_misc_tmpfile_mutex_key;
mysql_pfs_key_t srv_monitor_file_mutex_key;
mysql_pfs_key_t buf_dblwr_mutex_key;
mysql_pfs_key_t trx_mutex_key;
mysql_pfs_key_t trx_pool_mutex_key;
mysql_pfs_key_t trx_pool_manager_mutex_key;
mysql_pfs_key_t lock_mutex_key;

10
storage/innobase/trx/trx0i_s.cc

@ -423,7 +423,7 @@ fill_trx_row(
{
const char* s;
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
row->trx_id = trx_get_id_for_print(trx);
row->trx_started = trx->start_time;
@ -1046,7 +1046,7 @@ add_trx_relevant_locks_to_cache(
requested lock row, or NULL or
undefined */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
/* If transaction is waiting we add the wait lock and all locks
from another transactions that are blocking the wait lock. */
@ -1194,7 +1194,7 @@ static void fetch_data_into_cache_low(trx_i_s_cache_t *cache, const trx_t *trx)
static void fetch_data_into_cache(trx_i_s_cache_t *cache)
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
trx_i_s_cache_clear(cache);
/* Capture the state of transactions */
@ -1225,9 +1225,9 @@ trx_i_s_possibly_fetch_data_into_cache(
/* We need to read trx_sys and record/table lock queues */
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
fetch_data_into_cache(cache);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
/* update cache last read time */
cache->last_read = my_interval_timer();

22
storage/innobase/trx/trx0roll.cc

@ -153,7 +153,6 @@ inline void trx_t::rollback_low(trx_savept_t *savept)
@return error code or DB_SUCCESS */
dberr_t trx_t::rollback(trx_savept_t *savept)
{
ut_ad(!trx_mutex_own(this));
if (state == TRX_STATE_NOT_STARTED)
{
error_state= DB_SUCCESS;
@ -690,9 +689,9 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element,
void trx_roll_report_progress()
{
time_t now = time(NULL);
mutex_enter(&recv_sys.mutex);
mysql_mutex_lock(&recv_sys.mutex);
bool report = recv_sys.report(now);
mutex_exit(&recv_sys.mutex);
mysql_mutex_unlock(&recv_sys.mutex);
if (report) {
trx_roll_count_callback_arg arg;
@ -723,10 +722,10 @@ static my_bool trx_rollback_recovered_callback(rw_trx_hash_element_t *element,
mutex_enter(&element->mutex);
if (trx_t *trx= element->trx)
{
mutex_enter(&trx->mutex);
trx->mutex.wr_lock();
if (trx_state_eq(trx, TRX_STATE_ACTIVE) && trx->is_recovered)
trx_list->push_back(trx);
mutex_exit(&trx->mutex);
trx->mutex.wr_unlock();
}
mutex_exit(&element->mutex);
return 0;
@ -768,10 +767,10 @@ void trx_rollback_recovered(bool all)
trx_list.pop_back();
ut_ad(trx);
ut_d(trx_mutex_enter(trx));
ut_d(trx->mutex.wr_lock());
ut_ad(trx->is_recovered);
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE));
ut_d(trx_mutex_exit(trx));
ut_d(trx->mutex.wr_unlock());
if (srv_shutdown_state != SRV_SHUTDOWN_NONE && !srv_undo_sources &&
srv_fast_shutdown)
@ -865,8 +864,6 @@ trx_roll_graph_build(
que_fork_t* fork;
que_thr_t* thr;
ut_ad(trx_mutex_own(trx));
heap = mem_heap_create(512);
fork = que_fork_create(NULL, NULL, QUE_FORK_ROLLBACK, heap);
fork->trx = trx;
@ -891,8 +888,6 @@ trx_rollback_start(
partial undo), 0 if we are rolling back
the entire transaction */
{
ut_ad(trx_mutex_own(trx));
/* Initialize the rollback field in the transaction */
ut_ad(!trx->roll_limit);
@ -965,14 +960,13 @@ trx_rollback_step(
roll_limit = node->savept ? node->savept->least_undo_no : 0;
trx_mutex_enter(trx);
trx->mutex.wr_lock();
trx_commit_or_rollback_prepare(trx);
node->undo_thr = trx_rollback_start(trx, roll_limit);
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
} else {
ut_ad(node->state == ROLL_NODE_WAIT);

31
storage/innobase/trx/trx0trx.cc

@ -47,6 +47,7 @@ Created 3/26/1996 Heikki Tuuri
#include "trx0xa.h"
#include "ut0pool.h"
#include "ut0vec.h"
#include "sync0sync.h"
#include <set>
#include <new>
@ -202,7 +203,7 @@ struct TrxFactory {
trx->trx_savepoints,
&trx_named_savept_t::trx_savepoints);
mutex_create(LATCH_ID_TRX, &trx->mutex);
trx->mutex.init();
}
/** Release resources held by the transaction object.
@ -239,7 +240,7 @@ struct TrxFactory {
UT_DELETE(trx->xid);
ut_free(trx->detailed_error);
mutex_free(&trx->mutex);
trx->mutex.destroy();
trx->mod_tables.~trx_mod_tables_t();
@ -482,9 +483,9 @@ inline void trx_t::commit_state()
makes modifications to the database, will get an lsn larger than the
committing transaction T. In the case where the log flush fails, and
T never gets committed, also T2 will never get committed. */
trx_mutex_enter(this);
mutex.wr_lock();
state= TRX_STATE_COMMITTED_IN_MEMORY;
trx_mutex_exit(this);
mutex.wr_unlock();
ut_ad(id || !is_referenced());
}
@ -1267,10 +1268,10 @@ trx_update_mod_tables_timestamp(
/* recheck while holding the mutex that blocks
table->acquire() */
mutex_enter(&dict_sys.mutex);
mutex_enter(&lock_sys.mutex);
mysql_mutex_lock(&lock_sys.mutex);
const bool do_evict = !table->get_ref_count()
&& !UT_LIST_GET_LEN(table->locks);
mutex_exit(&lock_sys.mutex);
mysql_mutex_unlock(&lock_sys.mutex);
if (do_evict) {
dict_sys.remove(table, true);
}
@ -1477,7 +1478,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr)
}
lock.was_chosen_as_wsrep_victim= false;
#endif /* WITH_WSREP */
trx_mutex_enter(this);
mutex.wr_lock();
dict_operation= TRX_DICT_OP_NONE;
DBUG_LOG("trx", "Commit in memory: " << this);
@ -1485,7 +1486,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr)
assert_freed();
trx_init(this);
trx_mutex_exit(this);
mutex.wr_unlock();
ut_a(error_state == DB_SUCCESS);
if (!srv_read_only_mode)
@ -1873,7 +1874,7 @@ trx_print_latched(
ulint max_query_len) /*!< in: max query length to print,
or 0 to use the default max length */
{
ut_ad(lock_mutex_own());
mysql_mutex_assert_owner(&lock_sys.mutex);
trx_print_low(f, trx, max_query_len,
lock_number_of_rows_locked(&trx->lock),
@ -1896,11 +1897,11 @@ trx_print(
ulint n_trx_locks;
ulint heap_size;
lock_mutex_enter();
mysql_mutex_lock(&lock_sys.mutex);
n_rec_locks = lock_number_of_rows_locked(&trx->lock);
n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks);
heap_size = mem_heap_get_size(trx->lock.lock_heap);
lock_mutex_exit();
mysql_mutex_unlock(&lock_sys.mutex);
trx_print_low(f, trx, max_query_len,
n_rec_locks, n_trx_locks, heap_size);
@ -2010,9 +2011,9 @@ trx_prepare(
DBUG_EXECUTE_IF("ib_trx_crash_during_xa_prepare_step", DBUG_SUICIDE(););
ut_a(trx->state == TRX_STATE_ACTIVE);
trx_mutex_enter(trx);
trx->mutex.wr_lock();
trx->state = TRX_STATE_PREPARED;
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
if (lsn) {
/* Depending on the my.cnf options, we may now write the log
@ -2154,7 +2155,7 @@ static my_bool trx_get_trx_by_xid_callback(rw_trx_hash_element_t *element,
mutex_enter(&element->mutex);
if (trx_t *trx= element->trx)
{
trx_mutex_enter(trx);
trx->mutex.wr_lock();
if (trx->is_recovered &&
(trx_state_eq(trx, TRX_STATE_PREPARED) ||
trx_state_eq(trx, TRX_STATE_PREPARED_RECOVERED)) &&
@ -2171,7 +2172,7 @@ static my_bool trx_get_trx_by_xid_callback(rw_trx_hash_element_t *element,
arg->trx= trx;
found= 1;
}
trx_mutex_exit(trx);
trx->mutex.wr_unlock();
}
mutex_exit(&element->mutex);
return found;

23
storage/innobase/ut/ut0wqueue.cc

@ -38,10 +38,7 @@ ib_wqueue_create(void)
ib_wqueue_t* wq = static_cast<ib_wqueue_t*>(
ut_malloc_nokey(sizeof(*wq)));
/* Function ib_wqueue_create() has not been used anywhere,
not necessary to instrument this mutex */
mutex_create(LATCH_ID_WORK_QUEUE, &wq->mutex);
mysql_mutex_init(0, &wq->mutex, nullptr);
wq->items = ib_list_create();
@ -55,7 +52,7 @@ ib_wqueue_free(
/*===========*/
ib_wqueue_t* wq) /*!< in: work queue */
{
mutex_free(&wq->mutex);
mysql_mutex_destroy(&wq->mutex);
ib_list_free(wq->items);
ut_free(wq);
@ -70,13 +67,13 @@ void
ib_wqueue_add(ib_wqueue_t* wq, void* item, mem_heap_t* heap, bool wq_locked)
{
if (!wq_locked) {
mutex_enter(&wq->mutex);
mysql_mutex_lock(&wq->mutex);
}
ib_list_add_last(wq->items, item, heap);
if (!wq_locked) {
mutex_exit(&wq->mutex);
mysql_mutex_unlock(&wq->mutex);
}
}
@ -90,7 +87,7 @@ ib_wqueue_nowait(
{
ib_list_node_t* node = NULL;
mutex_enter(&wq->mutex);
mysql_mutex_lock(&wq->mutex);
if(!ib_list_is_empty(wq->items)) {
node = ib_list_get_first(wq->items);
@ -100,7 +97,7 @@ ib_wqueue_nowait(
}
}
mutex_exit(&wq->mutex);
mysql_mutex_unlock(&wq->mutex);
return (node ? node->data : NULL);
}
@ -109,9 +106,9 @@ ib_wqueue_nowait(
@return whether the queue is empty */
bool ib_wqueue_is_empty(ib_wqueue_t* wq)
{
mutex_enter(&wq->mutex);
mysql_mutex_lock(&wq->mutex);
bool is_empty = ib_list_is_empty(wq->items);
mutex_exit(&wq->mutex);
mysql_mutex_unlock(&wq->mutex);
return is_empty;
}
@ -125,9 +122,9 @@ ib_wqueue_len(
{
ulint len = 0;
mutex_enter(&wq->mutex);
mysql_mutex_lock(&wq->mutex);
len = ib_list_len(wq->items);
mutex_exit(&wq->mutex);
mysql_mutex_unlock(&wq->mutex);
return(len);
}
Loading…
Cancel
Save