Commit cfbbf542 authored by Marko Mäkelä's avatar Marko Mäkelä

MDEV-7962: Follow-up fix for 10.4

Replace wsrep_on() with trx_t::is_wsrep() where possible.

Also, rename some functions to member functions and
remove unused DBUG_EXECUTE_IF instrumentation:

trx_t::commit(): Renamed from trx_commit().

trx_t::commit_low(): Renamed from trx_commit_low().

trx_t::commit_in_memory(): Renamed from trx_commit_in_memory().
parent 0632b803
...@@ -7079,7 +7079,7 @@ prepare_inplace_alter_table_dict( ...@@ -7079,7 +7079,7 @@ prepare_inplace_alter_table_dict(
goto error_handling; goto error_handling;
} }
trx_commit(ctx->trx); ctx->trx->commit();
trx_start_for_ddl(ctx->trx, op); trx_start_for_ddl(ctx->trx, op);
if (!ctx->new_table->fts if (!ctx->new_table->fts
...@@ -11101,7 +11101,7 @@ ha_innobase::commit_inplace_alter_table( ...@@ -11101,7 +11101,7 @@ ha_innobase::commit_inplace_alter_table(
logical sense the commit in the file-based logical sense the commit in the file-based
data structures happens here. */ data structures happens here. */
trx_commit_low(trx, &mtr); trx->commit_low(&mtr);
} }
/* If server crashes here, the dictionary in /* If server crashes here, the dictionary in
......
...@@ -181,17 +181,6 @@ trx_start_for_ddl_low( ...@@ -181,17 +181,6 @@ trx_start_for_ddl_low(
trx_start_for_ddl_low((t), (o)) trx_start_for_ddl_low((t), (o))
#endif /* UNIV_DEBUG */ #endif /* UNIV_DEBUG */
/****************************************************************//**
Commits a transaction. */
void
trx_commit(
/*=======*/
trx_t* trx); /*!< in/out: transaction */
/** Commit a transaction and a mini-transaction.
@param[in,out] trx transaction
@param[in,out] mtr mini-transaction (NULL if no modifications) */
void trx_commit_low(trx_t* trx, mtr_t* mtr);
/**********************************************************************//** /**********************************************************************//**
Does the transaction commit for MySQL. Does the transaction commit for MySQL.
@return DB_SUCCESS or error number */ @return DB_SUCCESS or error number */
...@@ -898,10 +887,10 @@ struct trx_t { ...@@ -898,10 +887,10 @@ struct trx_t {
defer flush of the logs to disk defer flush of the logs to disk
until after we release the until after we release the
mutex. */ mutex. */
bool must_flush_log_later;/*!< this flag is set to TRUE in bool must_flush_log_later;/*!< set in commit()
trx_commit() if flush_log_later was if flush_log_later was
TRUE, and there were modifications by set and redo log was written;
the transaction; in that case we must in that case we will
flush the log in flush the log in
trx_commit_complete_for_mysql() */ trx_commit_complete_for_mysql() */
ulint duplicates; /*!< TRX_DUP_IGNORE | TRX_DUP_REPLACE */ ulint duplicates; /*!< TRX_DUP_IGNORE | TRX_DUP_REPLACE */
...@@ -1132,11 +1121,20 @@ struct trx_t { ...@@ -1132,11 +1121,20 @@ struct trx_t {
@param[in] table_id table identifier */ @param[in] table_id table identifier */
void evict_table(table_id_t table_id); void evict_table(table_id_t table_id);
private:
/** Mark a transaction committed in the main memory data structures. */
inline void commit_in_memory(const mtr_t *mtr);
public:
/** Commit the transaction. */
void commit();
bool is_referenced() /** Commit the transaction in a mini-transaction.
{ @param mtr mini-transaction (if there are any persistent modifications) */
return n_ref > 0; void commit_low(mtr_t *mtr= nullptr);
}
bool is_referenced() const { return n_ref > 0; }
void reference() void reference()
......
...@@ -6865,7 +6865,7 @@ DeadlockChecker::trx_rollback() ...@@ -6865,7 +6865,7 @@ DeadlockChecker::trx_rollback()
print("*** WE ROLL BACK TRANSACTION (1)\n"); print("*** WE ROLL BACK TRANSACTION (1)\n");
#ifdef WITH_WSREP #ifdef WITH_WSREP
if (wsrep_on(trx->mysql_thd) && wsrep_thd_is_SR(trx->mysql_thd)) { if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd); wsrep_handle_SR_rollback(m_start->mysql_thd, trx->mysql_thd);
} }
#endif #endif
...@@ -6956,8 +6956,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx) ...@@ -6956,8 +6956,7 @@ DeadlockChecker::check_and_resolve(const lock_t* lock, trx_t* trx)
print("*** WE ROLL BACK TRANSACTION (2)\n"); print("*** WE ROLL BACK TRANSACTION (2)\n");
#ifdef WITH_WSREP #ifdef WITH_WSREP
if (wsrep_on(trx->mysql_thd) if (trx->is_wsrep() && wsrep_thd_is_SR(trx->mysql_thd)) {
&& wsrep_thd_is_SR(trx->mysql_thd)) {
wsrep_handle_SR_rollback(trx->mysql_thd, wsrep_handle_SR_rollback(trx->mysql_thd,
victim_trx->mysql_thd); victim_trx->mysql_thd);
} }
......
...@@ -59,7 +59,7 @@ static bool trx_rollback_finish(trx_t* trx) ...@@ -59,7 +59,7 @@ static bool trx_rollback_finish(trx_t* trx)
trx->mod_tables.clear(); trx->mod_tables.clear();
bool finished = trx->error_state == DB_SUCCESS; bool finished = trx->error_state == DB_SUCCESS;
if (UNIV_LIKELY(finished)) { if (UNIV_LIKELY(finished)) {
trx_commit(trx); trx->commit();
} else { } else {
ut_a(trx->error_state == DB_INTERRUPTED); ut_a(trx->error_state == DB_INTERRUPTED);
ut_ad(!srv_is_being_started); ut_ad(!srv_is_being_started);
...@@ -84,7 +84,7 @@ static bool trx_rollback_finish(trx_t* trx) ...@@ -84,7 +84,7 @@ static bool trx_rollback_finish(trx_t* trx)
ut_free(undo); ut_free(undo);
undo = NULL; undo = NULL;
} }
trx_commit_low(trx, NULL); trx->commit_low();
} }
trx->lock.que_state = TRX_QUE_RUNNING; trx->lock.que_state = TRX_QUE_RUNNING;
......
...@@ -1338,316 +1338,255 @@ void trx_t::evict_table(table_id_t table_id) ...@@ -1338,316 +1338,255 @@ void trx_t::evict_table(table_id_t table_id)
} }
} }
/****************************************************************//** /** Mark a transaction committed in the main memory data structures. */
Commits a transaction in memory. */ inline void trx_t::commit_in_memory(const mtr_t *mtr)
static
void
trx_commit_in_memory(
/*=================*/
trx_t* trx, /*!< in/out: transaction */
const mtr_t* mtr) /*!< in: mini-transaction of
trx_write_serialisation_history(), or NULL if
the transaction did not modify anything */
{ {
trx->must_flush_log_later = false; must_flush_log_later= false;
trx->read_view.close(); read_view.close();
if (trx_is_autocommit_non_locking(trx)) {
ut_ad(trx->id == 0);
ut_ad(trx->read_only);
ut_a(!trx->is_recovered);
ut_ad(trx->rsegs.m_redo.rseg == NULL);
/* Note: We are asserting without holding the lock mutex. But if (trx_is_autocommit_non_locking(this))
that is OK because this transaction is not waiting and cannot {
be rolled back and no new locks can (or should) be added ut_ad(id == 0);
because it is flagged as a non-locking read-only transaction. */ ut_ad(read_only);
ut_a(!is_recovered);
ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0); ut_ad(!rsegs.m_redo.rseg);
/* This state change is not protected by any mutex, therefore /* Note: We are asserting without holding the lock mutex. But
there is an inherent race here around state transition during that is OK because this transaction is not waiting and cannot
printouts. We ignore this race for the sake of efficiency. be rolled back and no new locks can (or should) be added
However, the trx_sys_t::mutex will protect the trx_t instance because it is flagged as a non-locking read-only transaction. */
and it cannot be removed from the trx_list and freed ut_a(UT_LIST_GET_LEN(lock.trx_locks) == 0);
without first acquiring the trx_sys_t::mutex. */
/* This state change is not protected by any mutex, therefore
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE)); there is an inherent race here around state transition during
printouts. We ignore this race for the sake of efficiency.
MONITOR_INC(MONITOR_TRX_NL_RO_COMMIT); However, the trx_sys_t::mutex will protect the trx_t instance
and it cannot be removed from the trx_list and freed
DBUG_LOG("trx", "Autocommit in memory: " << trx); without first acquiring the trx_sys_t::mutex. */
trx->state = TRX_STATE_NOT_STARTED; ut_ad(trx_state_eq(this, TRX_STATE_ACTIVE));
} else {
MONITOR_INC(MONITOR_TRX_NL_RO_COMMIT);
DBUG_LOG("trx", "Autocommit in memory: " << this);
state= TRX_STATE_NOT_STARTED;
}
else
{
#ifdef UNIV_DEBUG #ifdef UNIV_DEBUG
if (!UT_LIST_GET_LEN(trx->lock.trx_locks)) { if (!UT_LIST_GET_LEN(lock.trx_locks))
for (lock_list::iterator it for (auto l : lock.table_locks)
= trx->lock.table_locks.begin(); ut_ad(!l);
it != trx->lock.table_locks.end();
it++) {
ut_ad(!*it);
}
}
#endif /* UNIV_DEBUG */ #endif /* UNIV_DEBUG */
trx->commit_state(); commit_state();
if (trx->id) {
trx_sys.deregister_rw(trx);
/* Wait for any implicit-to-explicit lock
conversions to cease, so that there will be no
race condition in lock_release(). */
while (UNIV_UNLIKELY(trx->is_referenced())) {
ut_delay(srv_spin_wait_delay);
}
trx->release_locks();
trx->id = 0;
} else {
ut_ad(trx->read_only || !trx->rsegs.m_redo.rseg);
trx->release_locks();
}
DEBUG_SYNC_C("after_trx_committed_in_memory");
if (trx->read_only || !trx->rsegs.m_redo.rseg) { if (id)
MONITOR_INC(MONITOR_TRX_RO_COMMIT); {
} else { trx_sys.deregister_rw(this);
trx_update_mod_tables_timestamp(trx);
MONITOR_INC(MONITOR_TRX_RW_COMMIT); /* Wait for any implicit-to-explicit lock conversions to cease,
trx->is_recovered = false; so that there will be no race condition in lock_release(). */
} while (UNIV_UNLIKELY(is_referenced()))
ut_delay(srv_spin_wait_delay);
while (dict_table_t* table = UT_LIST_GET_FIRST( release_locks();
trx->lock.evicted_tables)) { id= 0;
UT_LIST_REMOVE(trx->lock.evicted_tables, table); }
dict_mem_table_free(table); else
} {
} ut_ad(read_only || !rsegs.m_redo.rseg);
release_locks();
ut_ad(!trx->rsegs.m_redo.undo); }
ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0);
if (trx_rseg_t* rseg = trx->rsegs.m_redo.rseg) {
mutex_enter(&rseg->mutex);
ut_ad(rseg->trx_ref_count > 0);
--rseg->trx_ref_count;
mutex_exit(&rseg->mutex);
if (trx_undo_t*& insert = trx->rsegs.m_redo.old_insert) {
ut_ad(insert->rseg == rseg);
trx_undo_commit_cleanup(insert, false);
insert = NULL;
}
}
ut_ad(!trx->rsegs.m_redo.old_insert);
if (mtr != NULL) {
if (trx_undo_t*& undo = trx->rsegs.m_noredo.undo) {
ut_ad(undo->rseg == trx->rsegs.m_noredo.rseg);
trx_undo_commit_cleanup(undo, true);
undo = NULL;
}
/* NOTE that we could possibly make a group commit more DEBUG_SYNC_C("after_trx_committed_in_memory");
efficient here: call os_thread_yield here to allow also other
trxs to come to commit! */
/*-------------------------------------*/ if (read_only || !rsegs.m_redo.rseg)
{
MONITOR_INC(MONITOR_TRX_RO_COMMIT);
}
else
{
trx_update_mod_tables_timestamp(this);
MONITOR_INC(MONITOR_TRX_RW_COMMIT);
is_recovered= false;
}
/* Depending on the my.cnf options, we may now write the log while (dict_table_t *table= UT_LIST_GET_FIRST(lock.evicted_tables))
buffer to the log files, making the transaction durable if {
the OS does not crash. We may also flush the log files to UT_LIST_REMOVE(lock.evicted_tables, table);
disk, making the transaction durable also at an OS crash or a dict_mem_table_free(table);
power outage. }
}
The idea in InnoDB's group commit is that a group of ut_ad(!rsegs.m_redo.undo);
transactions gather behind a trx doing a physical disk write ut_ad(UT_LIST_GET_LEN(lock.evicted_tables) == 0);
to log files, and when that physical write has been completed,
one of those transactions does a write which commits the whole
group. Note that this group commit will only bring benefit if
there are > 2 users in the database. Then at least 2 users can
gather behind one doing the physical log write to disk.
If we are calling trx_commit() under prepare_commit_mutex, we if (trx_rseg_t *rseg= rsegs.m_redo.rseg)
will delay possible log write and flush to a separate function {
trx_commit_complete_for_mysql(), which is only called when the mutex_enter(&rseg->mutex);
thread has released the mutex. This is to make the ut_ad(rseg->trx_ref_count > 0);
group commit algorithm to work. Otherwise, the prepare_commit --rseg->trx_ref_count;
mutex would serialize all commits and prevent a group of mutex_exit(&rseg->mutex);
transactions from gathering. */
lsn_t lsn = mtr->commit_lsn();
if (lsn == 0) {
/* Nothing to be done. */
} else if (trx->flush_log_later) {
/* Do nothing yet */
trx->must_flush_log_later = true;
} else if (srv_flush_log_at_trx_commit == 0) {
/* Do nothing */
} else {
trx_flush_log_if_needed(lsn, trx);
}
trx->commit_lsn = lsn; if (trx_undo_t *&insert= rsegs.m_redo.old_insert)
{
ut_ad(insert->rseg == rseg);
trx_undo_commit_cleanup(insert, false);
insert= nullptr;
}
}
/* Tell server some activity has happened, since the trx ut_ad(!rsegs.m_redo.old_insert);
does changes something. Background utility threads like
master thread, purge thread or page_cleaner thread might
have some work to do. */
srv_active_wake_master_thread();
}
ut_ad(!trx->rsegs.m_noredo.undo); if (mtr)
{
if (trx_undo_t *&undo= rsegs.m_noredo.undo)
{
ut_ad(undo->rseg == rsegs.m_noredo.rseg);
trx_undo_commit_cleanup(undo, true);
undo= nullptr;
}
/* Free all savepoints, starting from the first. */ /* NOTE that we could possibly make a group commit more efficient
trx_named_savept_t* savep = UT_LIST_GET_FIRST(trx->trx_savepoints); here: call os_thread_yield here to allow also other trxs to come
to commit! */
/*-------------------------------------*/
/* Depending on the my.cnf options, we may now write the log
buffer to the log files, making the transaction durable if the OS
does not crash. We may also flush the log files to disk, making
the transaction durable also at an OS crash or a power outage.
The idea in InnoDB's group commit is that a group of transactions
gather behind a trx doing a physical disk write to log files, and
when that physical write has been completed, one of those
transactions does a write which commits the whole group. Note that
this group commit will only bring benefit if there are > 2 users
in the database. Then at least 2 users can gather behind one doing
the physical log write to disk.
If we are calling trx_t::commit() under prepare_commit_mutex, we
will delay possible log write and flush to a separate function
trx_commit_complete_for_mysql(), which is only called when the
thread has released the mutex. This is to make the group commit
algorithm to work. Otherwise, the prepare_commit mutex would
serialize all commits and prevent a group of transactions from
gathering. */
commit_lsn= mtr->commit_lsn();
if (!commit_lsn)
/* Nothing to be done. */;
else if (flush_log_later)
/* Do nothing yet */
must_flush_log_later= true;
else if (srv_flush_log_at_trx_commit)
trx_flush_log_if_needed(commit_lsn, this);
/* Tell server some activity has happened, since the trx does
changes something. Background utility threads like master thread,
purge thread or page_cleaner thread might have some work to do. */
srv_active_wake_master_thread();
}
trx_roll_savepoints_free(trx, savep); ut_ad(!rsegs.m_noredo.undo);
if (trx->fts_trx != NULL) { /* Free all savepoints, starting from the first. */
trx_finalize_for_fts(trx, trx->undo_no != 0); trx_named_savept_t *savep= UT_LIST_GET_FIRST(trx_savepoints);
}
trx_mutex_enter(trx); trx_roll_savepoints_free(this, savep);
trx->dict_operation = TRX_DICT_OP_NONE;
trx->lock.was_chosen_as_deadlock_victim = false;
DBUG_LOG("trx", "Commit in memory: " << trx); if (fts_trx)
trx->state = TRX_STATE_NOT_STARTED; trx_finalize_for_fts(this, undo_no != 0);
assert_trx_is_free(trx); trx_mutex_enter(this);
dict_operation= TRX_DICT_OP_NONE;
lock.was_chosen_as_deadlock_victim= false;
trx_init(trx); DBUG_LOG("trx", "Commit in memory: " << this);
state= TRX_STATE_NOT_STARTED;
trx_mutex_exit(trx); assert_trx_is_free(this);
trx_init(this);
trx_mutex_exit(this);
ut_a(trx->error_state == DB_SUCCESS); ut_a(error_state == DB_SUCCESS);
if (!srv_read_only_mode) { if (!srv_read_only_mode)
srv_wake_purge_thread_if_not_active(); srv_wake_purge_thread_if_not_active();
}
} }
/** Commit a transaction and a mini-transaction. /** Commit the transaction in a mini-transaction.
@param[in,out] trx transaction @param mtr mini-transaction (if there are any persistent modifications) */
@param[in,out] mtr mini-transaction (NULL if no modifications) */ void trx_t::commit_low(mtr_t *mtr)
void trx_commit_low(trx_t* trx, mtr_t* mtr)
{ {
assert_trx_nonlocking_or_in_list(trx); assert_trx_nonlocking_or_in_list(this);
ut_ad(!trx_state_eq(trx, TRX_STATE_COMMITTED_IN_MEMORY)); ut_ad(!trx_state_eq(this, TRX_STATE_COMMITTED_IN_MEMORY));
ut_ad(!mtr || mtr->is_active()); ut_ad(!mtr || mtr->is_active());
ut_d(bool aborted = trx->in_rollback ut_d(bool aborted = in_rollback && error_state == DB_DEADLOCK);
&& trx->error_state == DB_DEADLOCK); ut_ad(!mtr == (aborted || !has_logged_or_recovered()));
ut_ad(!mtr == (aborted || !trx->has_logged_or_recovered())); ut_ad(!mtr || !aborted);
ut_ad(!mtr || !aborted);
/* undo_no is non-zero if we're doing the final commit. */
/* undo_no is non-zero if we're doing the final commit. */ if (fts_trx && undo_no)
if (trx->fts_trx != NULL && trx->undo_no != 0) { {
dberr_t error; ut_a(!trx_is_autocommit_non_locking(this));
dberr_t error= fts_commit(this);
ut_a(!trx_is_autocommit_non_locking(trx)); /* FTS-FIXME: Temporarily tolerate DB_DUPLICATE_KEY instead of
dying. This is a possible scenario if there is a crash between
error = fts_commit(trx); insert to DELETED table committing and transaction committing. The
fix would be able to return error from this function */
/* FTS-FIXME: Temporarily tolerate DB_DUPLICATE_KEY ut_a(error == DB_SUCCESS || error == DB_DUPLICATE_KEY);
instead of dying. This is a possible scenario if there }
is a crash between insert to DELETED table committing
and transaction committing. The fix would be able to
return error from this function */
if (error != DB_SUCCESS && error != DB_DUPLICATE_KEY) {
/* FTS-FIXME: once we can return values from this
function, we should do so and signal an error
instead of just dying. */
ut_error;
}
}
#ifndef DBUG_OFF #ifndef DBUG_OFF
const bool debug_sync = trx->mysql_thd && trx->has_logged_persistent(); const bool debug_sync= mysql_thd && has_logged_persistent();
#endif #endif
if (mtr != NULL) { if (mtr)
trx_write_serialisation_history(trx, mtr); {
trx_write_serialisation_history(this, mtr);
/* The following call commits the mini-transaction, making the
whole transaction committed in the file-based world, at this /* The following call commits the mini-transaction, making the
log sequence number. The transaction becomes 'durable' when whole transaction committed in the file-based world, at this log
we write the log to disk, but in the logical sense the commit sequence number. The transaction becomes 'durable' when we write
in the file-based data structures (undo logs etc.) happens the log to disk, but in the logical sense the commit in the
here. file-based data structures (undo logs etc.) happens here.
NOTE that transaction numbers, which are assigned only to NOTE that transaction numbers, which are assigned only to
transactions with an update undo log, do not necessarily come transactions with an update undo log, do not necessarily come in
in exactly the same order as commit lsn's, if the transactions exactly the same order as commit lsn's, if the transactions have
have different rollback segments. To get exactly the same different rollback segments. To get exactly the same order we
order we should hold the kernel mutex up to this point, should hold the kernel mutex up to this point, adding to the
adding to the contention of the kernel mutex. However, if contention of the kernel mutex. However, if a transaction T2 is
a transaction T2 is able to see modifications made by able to see modifications made by a transaction T1, T2 will always
a transaction T1, T2 will always get a bigger transaction get a bigger transaction number and a bigger commit lsn than T1. */
number and a bigger commit lsn than T1. */
mtr->commit();
/*--------------*/ }
mtr_commit(mtr);
DBUG_EXECUTE_IF("ib_crash_during_trx_commit_in_mem",
if (trx->has_logged()) {
log_write_up_to(mtr->commit_lsn(),
true);
DBUG_SUICIDE();
});
/*--------------*/
}
#ifndef DBUG_OFF #ifndef DBUG_OFF
/* In case of this function is called from a stack executing if (debug_sync)
THD::release_resources -> ... DEBUG_SYNC_C("before_trx_state_committed_in_memory");
innobase_connection_close() ->
trx_rollback_for_mysql... -> .
mysql's thd does not seem to have
thd->debug_sync_control defined any longer. However the stack
is possible only with a prepared trx not updating any data.
*/
if (debug_sync) {
DEBUG_SYNC_C("before_trx_state_committed_in_memory");
}
#endif #endif
trx_commit_in_memory(trx, mtr); commit_in_memory(mtr);
} }
/****************************************************************//**
Commits a transaction. */
void
trx_commit(
/*=======*/
trx_t* trx) /*!< in/out: transaction */
{
mtr_t* mtr;
mtr_t local_mtr;
DBUG_EXECUTE_IF("ib_trx_commit_crash_before_trx_commit_start",
DBUG_SUICIDE(););
if (trx->has_logged_or_recovered()) { void trx_t::commit()
mtr = &local_mtr; {
mtr->start(); mtr_t *mtr= nullptr;
} else { mtr_t local_mtr;
mtr = NULL;
}
trx_commit_low(trx, mtr); if (has_logged_or_recovered())
{
mtr= &local_mtr;
local_mtr.start();
}
commit_low(mtr);
#ifdef WITH_WSREP #ifdef WITH_WSREP
/* Serialization history has been written and the /* Serialization history has been written and the transaction is
transaction is committed in memory, which makes committed in memory, which makes this commit ordered. Release commit
this commit ordered. Release commit order critical order critical section. */
section. */ if (mtr && is_wsrep())
if (wsrep_on(trx->mysql_thd)) wsrep_commit_ordered(mysql_thd);
{
wsrep_commit_ordered(trx->mysql_thd);
}
#endif /* WITH_WSREP */ #endif /* WITH_WSREP */
} }
...@@ -1741,11 +1680,8 @@ trx_commit_step( ...@@ -1741,11 +1680,8 @@ trx_commit_step(
trx_commit_or_rollback_prepare(trx); trx_commit_or_rollback_prepare(trx);
trx->lock.que_state = TRX_QUE_COMMITTING; trx->lock.que_state = TRX_QUE_COMMITTING;
trx->commit();
trx_commit(trx);
ut_ad(trx->lock.wait_thr == NULL); ut_ad(trx->lock.wait_thr == NULL);
trx->lock.que_state = TRX_QUE_RUNNING; trx->lock.que_state = TRX_QUE_RUNNING;
thr = NULL; thr = NULL;
...@@ -1783,9 +1719,7 @@ trx_commit_for_mysql( ...@@ -1783,9 +1719,7 @@ trx_commit_for_mysql(
case TRX_STATE_PREPARED: case TRX_STATE_PREPARED:
case TRX_STATE_PREPARED_RECOVERED: case TRX_STATE_PREPARED_RECOVERED:
trx->op_info = "committing"; trx->op_info = "committing";
trx->commit();
trx_commit(trx);
MONITOR_DEC(MONITOR_TRX_ACTIVE); MONITOR_DEC(MONITOR_TRX_ACTIVE);
trx->op_info = ""; trx->op_info = "";
return(DB_SUCCESS); return(DB_SUCCESS);
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment