summaryrefslogtreecommitdiff
path: root/storage/innobase
diff options
context:
space:
mode:
Diffstat (limited to 'storage/innobase')
-rw-r--r--storage/innobase/btr/btr0cur.cc29
-rw-r--r--storage/innobase/fil/fil0fil.cc1
-rw-r--r--storage/innobase/fts/fts0fts.cc2
-rw-r--r--storage/innobase/handler/ha_innodb.cc230
-rw-r--r--storage/innobase/handler/handler0alter.cc4
-rw-r--r--storage/innobase/include/ha_prototypes.h2
-rw-r--r--storage/innobase/include/trx0trx.h5
-rw-r--r--storage/innobase/include/univ.i2
-rw-r--r--storage/innobase/lock/lock0lock.cc62
-rw-r--r--storage/innobase/log/log0recv.cc1
-rw-r--r--storage/innobase/row/row0upd.cc33
-rw-r--r--storage/innobase/srv/srv0start.cc4
-rw-r--r--storage/innobase/trx/trx0trx.cc12
13 files changed, 302 insertions, 85 deletions
diff --git a/storage/innobase/btr/btr0cur.cc b/storage/innobase/btr/btr0cur.cc
index 9efa2d8f8bd..7eeefe725c9 100644
--- a/storage/innobase/btr/btr0cur.cc
+++ b/storage/innobase/btr/btr0cur.cc
@@ -3,7 +3,7 @@
Copyright (c) 1994, 2019, Oracle and/or its affiliates. All Rights Reserved.
Copyright (c) 2008, Google Inc.
Copyright (c) 2012, Facebook Inc.
-Copyright (c) 2015, 2020, MariaDB Corporation.
+Copyright (c) 2015, 2021, MariaDB Corporation.
Portions of this file contain modifications contributed and copyrighted by
Google, Inc. Those modifications are gratefully acknowledged and are described
@@ -67,6 +67,9 @@ Created 10/16/1994 Heikki Tuuri
#include "srv0start.h"
#include "mysql_com.h"
#include "dict0stats.h"
+#ifdef WITH_WSREP
+#include "mysql/service_wsrep.h"
+#endif /* WITH_WSREP */
/** Buffered B-tree operation types, introduced as part of delete buffering. */
enum btr_op_t {
@@ -3244,7 +3247,8 @@ btr_cur_ins_lock_and_undo(
/* Check if there is predicate or GAP lock preventing the insertion */
if (!(flags & BTR_NO_LOCKING_FLAG)) {
- if (dict_index_is_spatial(index)) {
+ const unsigned type = index->type;
+ if (UNIV_UNLIKELY(type & DICT_SPATIAL)) {
lock_prdt_t prdt;
rtr_mbr_t mbr;
@@ -3261,9 +3265,30 @@ btr_cur_ins_lock_and_undo(
index, thr, mtr, &prdt);
*inherit = false;
} else {
+#ifdef WITH_WSREP
+ trx_t* trx= thr_get_trx(thr);
+ /* If transaction scanning an unique secondary
+ key is wsrep high priority thread (brute
+ force) this scanning may involve GAP-locking
+ in the index. As this locking happens also
+ when applying replication events in high
+ priority applier threads, there is a
+ probability for lock conflicts between two
+ wsrep high priority threads. To avoid this
+ GAP-locking we mark that this transaction
+ is using unique key scan here. */
+ if ((type & (DICT_CLUSTERED | DICT_UNIQUE)) == DICT_UNIQUE
+ && trx->is_wsrep()
+ && wsrep_thd_is_BF(trx->mysql_thd, false)) {
+ trx->wsrep_UK_scan= true;
+ }
+#endif /* WITH_WSREP */
err = lock_rec_insert_check_and_lock(
flags, rec, btr_cur_get_block(cursor),
index, thr, mtr, inherit);
+#ifdef WITH_WSREP
+ trx->wsrep_UK_scan= false;
+#endif /* WITH_WSREP */
}
}
diff --git a/storage/innobase/fil/fil0fil.cc b/storage/innobase/fil/fil0fil.cc
index 2734d6bc68d..d000bc471d9 100644
--- a/storage/innobase/fil/fil0fil.cc
+++ b/storage/innobase/fil/fil0fil.cc
@@ -2206,7 +2206,6 @@ fil_rename_tablespace(
ut_ad(strchr(new_file_name, OS_PATH_SEPARATOR) != NULL);
if (!recv_recovery_is_on()) {
- fil_name_write_rename(id, old_file_name, new_file_name);
mysql_mutex_lock(&log_sys.mutex);
}
diff --git a/storage/innobase/fts/fts0fts.cc b/storage/innobase/fts/fts0fts.cc
index a5945fea791..9a7d33ca34c 100644
--- a/storage/innobase/fts/fts0fts.cc
+++ b/storage/innobase/fts/fts0fts.cc
@@ -1695,7 +1695,7 @@ fts_drop_tables(
error = fts_drop_common_tables(trx, &fts_table);
- if (error == DB_SUCCESS) {
+ if (error == DB_SUCCESS && table->fts) {
error = fts_drop_all_index_tables(trx, table->fts);
}
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index e72b307854d..d306311365d 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -63,6 +63,7 @@ this program; if not, write to the Free Software Foundation, Inc.,
#include <my_service_manager.h>
#include <key.h>
+#include <sql_manager.h>
/* Include necessary InnoDB headers */
#include "btr0btr.h"
@@ -1594,9 +1595,7 @@ thd_to_trx_id(
return(thd_to_trx(thd)->id);
}
-static int
-wsrep_abort_transaction(handlerton* hton, THD *bf_thd, THD *victim_thd,
- my_bool signal);
+static void wsrep_abort_transaction(handlerton*, THD *, THD *, my_bool);
static int innobase_wsrep_set_checkpoint(handlerton* hton, const XID* xid);
static int innobase_wsrep_get_checkpoint(handlerton* hton, XID* xid);
#endif /* WITH_WSREP */
@@ -4687,6 +4686,8 @@ static void innobase_kill_query(handlerton*, THD *thd, enum thd_kill_levels)
if (lock_t *lock= trx->lock.wait_lock)
{
trx_mutex_enter(trx);
+ if (trx->is_wsrep() && wsrep_thd_is_aborting(thd))
+ trx->lock.was_chosen_as_deadlock_victim= TRUE;
lock_cancel_waiting_and_release(lock);
trx_mutex_exit(trx);
}
@@ -17167,7 +17168,8 @@ innodb_io_capacity_update(
" higher than innodb_io_capacity_max %lu",
in_val, srv_max_io_capacity);
- srv_max_io_capacity = in_val * 2;
+ srv_max_io_capacity = (in_val & ~(~0UL >> 1))
+ ? in_val : in_val * 2;
push_warning_printf(thd, Sql_condition::WARN_LEVEL_WARN,
ER_WRONG_ARGUMENTS,
@@ -18647,49 +18649,54 @@ static struct st_mysql_storage_engine innobase_storage_engine=
#ifdef WITH_WSREP
-/** This function is used to kill one transaction.
+struct bg_wsrep_kill_trx_arg {
+ my_thread_id thd_id, bf_thd_id;
+ trx_id_t trx_id, bf_trx_id;
+ bool signal;
+};
-This transaction was open on this node (not-yet-committed), and a
-conflicting writeset from some other node that was being applied
-caused a locking conflict. First committed (from other node)
-wins, thus open transaction is rolled back. BF stands for
-brute-force: any transaction can get aborted by galera any time
-it is necessary.
+/** Kill one transaction from a background manager thread
-This conflict can happen only when the replicated writeset (from
-other node) is being applied, not when it’s waiting in the queue.
-If our local transaction reached its COMMIT and this conflicting
-writeset was in the queue, then it should fail the local
-certification test instead.
-
-A brute force abort is only triggered by a locking conflict
-between a writeset being applied by an applier thread (slave thread)
-and an open transaction on the node, not by a Galera writeset
-comparison as in the local certification failure.
+wsrep_innobase_kill_one_trx() is invoked when lock_sys.mutex and trx mutex
+are taken, wsrep_thd_bf_abort() cannot be used there as it takes THD mutexes
+that must be taken before lock_sys.mutex and trx mutex. That's why
+wsrep_innobase_kill_one_trx only posts the killing task to the manager thread
+and the actual killing happens asynchronously here.
-@param[in] bf_thd Brute force (BF) thread
-@param[in,out] victim_trx Vimtim trx to be killed
-@param[in] signal Should victim be signaled */
-int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
+As no mutexes were held we don't know whether THD or trx pointers are still
+valid, so we need to pass thread/trx ids and perform a lookup.
+*/
+static void bg_wsrep_kill_trx(void *void_arg)
{
- ut_ad(bf_thd);
- ut_ad(victim_trx);
- ut_ad(lock_mutex_own());
- ut_ad(trx_mutex_own(victim_trx));
+ bg_wsrep_kill_trx_arg *arg= (bg_wsrep_kill_trx_arg *)void_arg;
+ THD *thd, *bf_thd;
+ trx_t *victim_trx;
+ bool aborting= false;
- DBUG_ENTER("wsrep_innobase_kill_one_trx");
+ if ((bf_thd= find_thread_by_id(arg->bf_thd_id)))
+ wsrep_thd_LOCK(bf_thd);
+ if ((thd= find_thread_by_id(arg->thd_id)))
+ wsrep_thd_LOCK(thd);
- THD *thd= victim_trx->mysql_thd;
- ut_ad(thd);
- /* Note that bf_trx might not exist here e.g. on MDL conflict
- case (test: galera_concurrent_ctas). Similarly, BF thread
- could be also acquiring MDL-lock causing victim to be
- aborted. However, we have not yet called innobase_trx_init()
- for BF transaction (test: galera_many_columns)*/
- trx_t* bf_trx= thd_to_trx(bf_thd);
- DBUG_ASSERT(wsrep_on(bf_thd));
+ if (!thd || !bf_thd || !(victim_trx= thd_to_trx(thd)))
+ goto ret0;
- wsrep_thd_LOCK(thd);
+ lock_mutex_enter();
+ trx_mutex_enter(victim_trx);
+ if (victim_trx->id != arg->trx_id)
+ {
+ /* apparently victim trx was meanwhile rolled back.
+ tell bf thd not to wait, in case it already started to */
+ trx_t *trx= thd_to_trx(bf_thd);
+ if (lock_t *lock= trx->lock.wait_lock) {
+ trx_mutex_enter(trx);
+ lock_cancel_waiting_and_release(lock);
+ trx_mutex_exit(trx);
+ }
+ goto ret1;
+ }
+
+ DBUG_ASSERT(wsrep_on(bf_thd));
WSREP_LOG_CONFLICT(bf_thd, thd, TRUE);
@@ -18697,7 +18704,7 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
"seqno: %lld client_state: %s client_mode: %s transaction_mode: %s "
"query: %s",
wsrep_thd_is_BF(bf_thd, false) ? "BF" : "normal",
- bf_trx ? bf_trx->id : TRX_ID_MAX,
+ arg->bf_trx_id,
thd_get_thread_id(bf_thd),
wsrep_thd_trx_seqno(bf_thd),
wsrep_thd_client_state_str(bf_thd),
@@ -18722,30 +18729,86 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
if (wsrep_thd_set_wsrep_aborter(bf_thd, thd))
{
WSREP_DEBUG("innodb kill transaction skipped due to wsrep_aborter set");
- wsrep_thd_UNLOCK(thd);
- DBUG_RETURN(0);
+ goto ret1;
}
- /* Note that we need to release this as it will be acquired
- below in wsrep-lib */
- wsrep_thd_UNLOCK(thd);
- DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort");
-
- if (wsrep_thd_bf_abort(bf_thd, thd, signal))
- {
- lock_t* wait_lock = victim_trx->lock.wait_lock;
- if (wait_lock) {
- DBUG_ASSERT(victim_trx->is_wsrep());
- WSREP_DEBUG("victim has wait flag: %lu",
- thd_get_thread_id(thd));
+ aborting= true;
- WSREP_DEBUG("canceling wait lock");
- victim_trx->lock.was_chosen_as_deadlock_victim= TRUE;
- lock_cancel_waiting_and_release(wait_lock);
+ret1:
+ trx_mutex_exit(victim_trx);
+ lock_mutex_exit();
+ret0:
+ if (thd) {
+ wsrep_thd_UNLOCK(thd);
+ if (aborting) {
+ DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort");
+ wsrep_thd_bf_abort(bf_thd, thd, arg->signal);
}
+ wsrep_thd_kill_UNLOCK(thd);
+ }
+ if (bf_thd) {
+ wsrep_thd_UNLOCK(bf_thd);
+ wsrep_thd_kill_UNLOCK(bf_thd);
}
+ free(arg);
+}
- DBUG_RETURN(0);
+/** This function is used to kill one transaction.
+
+This transaction was open on this node (not-yet-committed), and a
+conflicting writeset from some other node that was being applied
+caused a locking conflict. First committed (from other node)
+wins, thus open transaction is rolled back. BF stands for
+brute-force: any transaction can get aborted by galera any time
+it is necessary.
+
+This conflict can happen only when the replicated writeset (from
+other node) is being applied, not when it’s waiting in the queue.
+If our local transaction reached its COMMIT and this conflicting
+writeset was in the queue, then it should fail the local
+certification test instead.
+
+A brute force abort is only triggered by a locking conflict
+between a writeset being applied by an applier thread (slave thread)
+and an open transaction on the node, not by a Galera writeset
+comparison as in the local certification failure.
+
+@param[in] bf_thd Brute force (BF) thread
+@param[in,out] victim_trx Vimtim trx to be killed
+@param[in] signal Should victim be signaled */
+void
+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));
+
+ DBUG_ENTER("wsrep_innobase_kill_one_trx");
+
+ DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort",
+ {
+ const char act[]=
+ "now "
+ "SIGNAL sync.before_wsrep_thd_abort_reached "
+ "WAIT_FOR signal.before_wsrep_thd_abort";
+ DBUG_ASSERT(!debug_sync_set_action(bf_thd,
+ STRING_WITH_LEN(act)));
+ };);
+
+ trx_t* bf_trx= thd_to_trx(bf_thd);
+ bg_wsrep_kill_trx_arg *arg = (bg_wsrep_kill_trx_arg*)malloc(sizeof(*arg));
+ arg->thd_id = thd_get_thread_id(victim_trx->mysql_thd);
+ arg->trx_id = victim_trx->id;
+ arg->bf_thd_id = thd_get_thread_id(bf_thd);
+ arg->bf_trx_id = bf_trx ? bf_trx->id : TRX_ID_MAX;
+ arg->signal = signal;
+ mysql_manager_submit(bg_wsrep_kill_trx, arg);
+
+ DBUG_VOID_RETURN;
}
/** This function forces the victim transaction to abort. Aborting the
@@ -18758,14 +18821,14 @@ int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal)
@return -1 victim thread was aborted (no transaction)
*/
static
-int
+void
wsrep_abort_transaction(
handlerton*,
THD *bf_thd,
THD *victim_thd,
my_bool signal)
{
- DBUG_ENTER("wsrep_innobase_abort_thd");
+ DBUG_ENTER("wsrep_abort_transaction");
ut_ad(bf_thd);
ut_ad(victim_thd);
@@ -18779,16 +18842,45 @@ wsrep_abort_transaction(
if (victim_trx) {
lock_mutex_enter();
trx_mutex_enter(victim_trx);
- int rcode= wsrep_innobase_kill_one_trx(bf_thd,
- victim_trx, signal);
+ victim_trx->lock.was_chosen_as_wsrep_victim= true;
trx_mutex_exit(victim_trx);
lock_mutex_exit();
- DBUG_RETURN(rcode);
+
+ wsrep_thd_kill_LOCK(victim_thd);
+ wsrep_thd_LOCK(victim_thd);
+ bool aborting= !wsrep_thd_set_wsrep_aborter(bf_thd, victim_thd);
+ wsrep_thd_UNLOCK(victim_thd);
+ if (aborting) {
+ DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort");
+ DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort",
+ {
+ const char act[]=
+ "now "
+ "SIGNAL sync.before_wsrep_thd_abort_reached "
+ "WAIT_FOR signal.before_wsrep_thd_abort";
+ DBUG_ASSERT(!debug_sync_set_action(bf_thd,
+ STRING_WITH_LEN(act)));
+ };);
+ wsrep_thd_bf_abort(bf_thd, victim_thd, signal);
+ }
+ wsrep_thd_kill_UNLOCK(victim_thd);
+ DBUG_VOID_RETURN;
} else {
+ DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort",
+ {
+ const char act[]=
+ "now "
+ "SIGNAL sync.before_wsrep_thd_abort_reached "
+ "WAIT_FOR signal.before_wsrep_thd_abort";
+ DBUG_ASSERT(!debug_sync_set_action(bf_thd,
+ STRING_WITH_LEN(act)));
+ };);
+ wsrep_thd_kill_LOCK(victim_thd);
wsrep_thd_bf_abort(bf_thd, victim_thd, signal);
+ wsrep_thd_kill_UNLOCK(victim_thd);
}
- DBUG_RETURN(-1);
+ DBUG_VOID_RETURN;
}
static
@@ -20625,11 +20717,11 @@ innobase_get_computed_value(
field = dtuple_get_nth_v_field(row, col->v_pos);
- my_bitmap_map* old_write_set = dbug_tmp_use_all_columns(mysql_table, mysql_table->write_set);
- my_bitmap_map* old_read_set = dbug_tmp_use_all_columns(mysql_table, mysql_table->read_set);
+ MY_BITMAP *old_write_set = dbug_tmp_use_all_columns(mysql_table, &mysql_table->write_set);
+ MY_BITMAP *old_read_set = dbug_tmp_use_all_columns(mysql_table, &mysql_table->read_set);
ret = mysql_table->update_virtual_field(mysql_table->field[col->m_col.ind]);
- dbug_tmp_restore_column_map(mysql_table->read_set, old_read_set);
- dbug_tmp_restore_column_map(mysql_table->write_set, old_write_set);
+ dbug_tmp_restore_column_map(&mysql_table->read_set, old_read_set);
+ dbug_tmp_restore_column_map(&mysql_table->write_set, old_write_set);
if (ret != 0) {
DBUG_RETURN(NULL);
diff --git a/storage/innobase/handler/handler0alter.cc b/storage/innobase/handler/handler0alter.cc
index 54a20a95a31..ac17a7a8a3f 100644
--- a/storage/innobase/handler/handler0alter.cc
+++ b/storage/innobase/handler/handler0alter.cc
@@ -3360,9 +3360,9 @@ innobase_row_to_mysql(
}
}
if (table->vfield) {
- my_bitmap_map* old_read_set = tmp_use_all_columns(table, table->read_set);
+ MY_BITMAP* old_read_set = tmp_use_all_columns(table, &table->read_set);
table->update_virtual_fields(table->file, VCOL_UPDATE_FOR_READ);
- tmp_restore_column_map(table->read_set, old_read_set);
+ tmp_restore_column_map(&table->read_set, old_read_set);
}
}
diff --git a/storage/innobase/include/ha_prototypes.h b/storage/innobase/include/ha_prototypes.h
index 41068e7c56c..820214de774 100644
--- a/storage/innobase/include/ha_prototypes.h
+++ b/storage/innobase/include/ha_prototypes.h
@@ -218,7 +218,7 @@ innobase_casedn_str(
char* a); /*!< in/out: string to put in lower case */
#ifdef WITH_WSREP
-int wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal);
+void wsrep_innobase_kill_one_trx(THD *bf_thd, trx_t *victim_trx, bool signal);
ulint wsrep_innobase_mysql_sort(int mysql_type, uint charset_number,
unsigned char* str, ulint str_length,
unsigned int buf_length);
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 2aa0ed1e0e3..f1a13e6ea59 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2016, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2015, 2020, MariaDB Corporation.
+Copyright (c) 2015, 2021, 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
@@ -792,6 +792,9 @@ public:
/** whether wsrep_on(mysql_thd) held at the start of transaction */
bool wsrep;
bool is_wsrep() const { return UNIV_UNLIKELY(wsrep); }
+ /** true, if BF thread is performing unique secondary index scanning */
+ bool wsrep_UK_scan;
+ bool is_wsrep_UK_scan() const { return UNIV_UNLIKELY(wsrep_UK_scan); }
#else /* WITH_WSREP */
bool is_wsrep() const { return false; }
#endif /* WITH_WSREP */
diff --git a/storage/innobase/include/univ.i b/storage/innobase/include/univ.i
index 51fdc2a0f20..6f633a45499 100644
--- a/storage/innobase/include/univ.i
+++ b/storage/innobase/include/univ.i
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1994, 2016, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2013, 2020, MariaDB Corporation.
+Copyright (c) 2013, 2021, MariaDB Corporation.
Copyright (c) 2008, Google Inc.
Portions of this file contain modifications contributed and copyrighted by
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index ef04d2bb66b..cec1eb22d28 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2014, 2020, MariaDB Corporation.
+Copyright (c) 2014, 2021, 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
@@ -619,6 +619,11 @@ static void wsrep_assert_no_bf_bf_wait(
{
ut_ad(!lock_rec1 || lock_get_type_low(lock_rec1) == LOCK_REC);
ut_ad(lock_get_type_low(lock_rec2) == LOCK_REC);
+ ut_ad(lock_mutex_own());
+
+ /* Note that we are holding lock_sys->mutex, thus we should
+ not acquire THD::LOCK_thd_data mutex below to avoid mutexing
+ order violation. */
if (!trx1->is_wsrep() || !lock_rec2->trx->is_wsrep())
return;
@@ -696,6 +701,7 @@ lock_rec_has_to_wait(
{
ut_ad(trx && lock2);
ut_ad(lock_get_type_low(lock2) == LOCK_REC);
+ ut_ad(lock_mutex_own());
if (trx == lock2->trx
|| lock_mode_compatible(
@@ -776,9 +782,23 @@ lock_rec_has_to_wait(
}
#ifdef WITH_WSREP
- /* There should not be two conflicting locks that are
- brute force. If there is it is a bug. */
- wsrep_assert_no_bf_bf_wait(NULL, lock2, trx);
+ /* New lock request from a transaction is using unique key
+ scan and this transaction is a wsrep high priority transaction
+ (brute force). If conflicting transaction is also wsrep high
+ priority transaction we should avoid lock conflict because
+ ordering of these transactions is already decided and
+ conflicting transaction will be later replayed. Note
+ that thread holding conflicting lock can't be
+ committed or rolled back while we hold
+ lock_sys->mutex. */
+ if (trx->is_wsrep_UK_scan()
+ && wsrep_thd_is_BF(lock2->trx->mysql_thd, false)) {
+ return false;
+ }
+
+ /* There should not be two conflicting locks that are
+ brute force. If there is it is a bug. */
+ wsrep_assert_no_bf_bf_wait(NULL, lock2, trx);
#endif /* WITH_WSREP */
return true;
@@ -5495,6 +5515,19 @@ lock_sec_rec_modify_check_and_lock(
heap_no = page_rec_get_heap_no(rec);
+#ifdef WITH_WSREP
+ trx_t *trx= thr_get_trx(thr);
+ /* If transaction scanning an unique secondary key is wsrep
+ high priority thread (brute force) this scanning may involve
+ GAP-locking in the index. As this locking happens also when
+ applying replication events in high priority applier threads,
+ there is a probability for lock conflicts between two wsrep
+ high priority threads. To avoid this GAP-locking we mark that
+ this transaction is using unique key scan here. */
+ if (trx->is_wsrep() && wsrep_thd_is_BF(trx->mysql_thd, false))
+ trx->wsrep_UK_scan= true;
+#endif /* WITH_WSREP */
+
/* Another transaction cannot have an implicit lock on the record,
because when we come here, we already have modified the clustered
index record, and this would not have been possible if another active
@@ -5503,6 +5536,10 @@ lock_sec_rec_modify_check_and_lock(
err = lock_rec_lock(TRUE, LOCK_X | LOCK_REC_NOT_GAP,
block, heap_no, index, thr);
+#ifdef WITH_WSREP
+ trx->wsrep_UK_scan= false;
+#endif /* WITH_WSREP */
+
#ifdef UNIV_DEBUG
{
mem_heap_t* heap = NULL;
@@ -5594,9 +5631,26 @@ lock_sec_rec_read_check_and_lock(
return DB_SUCCESS;
}
+#ifdef WITH_WSREP
+ trx_t *trx= thr_get_trx(thr);
+ /* If transaction scanning an unique secondary key is wsrep
+ high priority thread (brute force) this scanning may involve
+ GAP-locking in the index. As this locking happens also when
+ applying replication events in high priority applier threads,
+ there is a probability for lock conflicts between two wsrep
+ high priority threads. To avoid this GAP-locking we mark that
+ this transaction is using unique key scan here. */
+ if (trx->is_wsrep() && wsrep_thd_is_BF(trx->mysql_thd, false))
+ trx->wsrep_UK_scan= true;
+#endif /* WITH_WSREP */
+
err = lock_rec_lock(FALSE, gap_mode | mode,
block, heap_no, index, thr);
+#ifdef WITH_WSREP
+ trx->wsrep_UK_scan= false;
+#endif /* WITH_WSREP */
+
ut_ad(lock_rec_queue_validate(FALSE, block, rec, index, offsets));
return(err);
diff --git a/storage/innobase/log/log0recv.cc b/storage/innobase/log/log0recv.cc
index e7f5ef1e260..6317444f326 100644
--- a/storage/innobase/log/log0recv.cc
+++ b/storage/innobase/log/log0recv.cc
@@ -3605,6 +3605,7 @@ completed:
mutex_enter(&recv_sys.mutex);
recv_sys.apply_log_recs = true;
+ recv_no_ibuf_operations = false;
mutex_exit(&recv_sys.mutex);
diff --git a/storage/innobase/row/row0upd.cc b/storage/innobase/row/row0upd.cc
index aee20477ac0..84f5bcd4a4e 100644
--- a/storage/innobase/row/row0upd.cc
+++ b/storage/innobase/row/row0upd.cc
@@ -1,7 +1,7 @@
/*****************************************************************************
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All Rights Reserved.
-Copyright (c) 2015, 2020, MariaDB Corporation.
+Copyright (c) 2015, 2021, 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
@@ -1999,7 +1999,9 @@ row_upd_sec_index_entry(
break;
}
- if (!index->is_committed()) {
+ bool uncommitted = !index->is_committed();
+
+ if (uncommitted) {
/* The index->online_status may change if the index is
or was being created online, but not committed yet. It
is protected by index->lock. */
@@ -2202,11 +2204,38 @@ row_upd_sec_index_entry(
mem_heap_empty(heap);
+ DEBUG_SYNC_C_IF_THD(trx->mysql_thd,
+ "before_row_upd_sec_new_index_entry");
+
+ uncommitted = !index->is_committed();
+ if (uncommitted) {
+ mtr.start();
+ /* The index->online_status may change if the index is
+ being rollbacked. It is protected by index->lock. */
+
+ mtr_s_lock_index(index, &mtr);
+
+ switch (dict_index_get_online_status(index)) {
+ case ONLINE_INDEX_COMPLETE:
+ case ONLINE_INDEX_CREATION:
+ break;
+ case ONLINE_INDEX_ABORTED:
+ case ONLINE_INDEX_ABORTED_DROPPED:
+ mtr_commit(&mtr);
+ goto func_exit;
+ }
+
+ }
+
/* Build a new index entry */
entry = row_build_index_entry(node->upd_row, node->upd_ext,
index, heap);
ut_a(entry);
+ if (uncommitted) {
+ mtr_commit(&mtr);
+ }
+
/* Insert new index entry */
err = row_ins_sec_index_entry(index, entry, thr, !node->is_delete);
diff --git a/storage/innobase/srv/srv0start.cc b/storage/innobase/srv/srv0start.cc
index a3678e20955..9ea554ec622 100644
--- a/storage/innobase/srv/srv0start.cc
+++ b/storage/innobase/srv/srv0start.cc
@@ -2103,7 +2103,9 @@ void innodb_shutdown()
srv_sys_space.shutdown();
if (srv_tmp_space.get_sanity_check_status()) {
- fil_system.temp_space->close();
+ if (fil_system.temp_space) {
+ fil_system.temp_space->close();
+ }
srv_tmp_space.delete_files();
}
srv_tmp_space.shutdown();
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 2ad63df471c..871f8a899fd 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -158,6 +158,10 @@ trx_init(
trx->lock.rec_cached = 0;
trx->lock.table_cached = 0;
+#ifdef WITH_WSREP
+ ut_ad(!trx->wsrep);
+ ut_ad(!trx->wsrep_UK_scan);
+#endif /* WITH_WSREP */
}
/** For managing the life-cycle of the trx_t instance that we get
@@ -366,6 +370,10 @@ trx_t *trx_create()
ut_ad(trx->lock.rec_cached == 0);
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0);
+#ifdef WITH_WSREP
+ ut_ad(!trx->wsrep_UK_scan);
+#endif /* WITH_WSREP */
+
trx_sys.register_trx(trx);
return(trx);
@@ -462,6 +470,10 @@ void trx_t::free()
MEM_NOACCESS(&xid, sizeof xid);
MEM_NOACCESS(&mod_tables, sizeof mod_tables);
MEM_NOACCESS(&detailed_error, sizeof detailed_error);
+#ifdef WITH_WSREP
+ ut_ad(!wsrep_UK_scan);
+ MEM_NOACCESS(&wsrep_UK_scan, sizeof wsrep_UK_scan);
+#endif /* WITH_WSREP */
MEM_NOACCESS(&magic_n, sizeof magic_n);
trx_pools->mem_free(this);
}