Commit d09f1469 authored by Sergey Vojtovich's avatar Sergey Vojtovich

MDEV-14756 - Remove trx_sys_t::rw_trx_list

Reduce divergence between trx_sys_t::rw_trx_hash and trx_sys_t::rw_trx_list
by not adding recovered COMMITTED transactions to trx_sys_t::rw_trx_list.

Such transactions are discarded immediately without creating trx object.

This also required to split rollback and cleanup phases of recovery. To
reflect these updates the following renames happened:
trx_rollback_or_clean_all_recovered() -> trx_rollback_all_recovered()
trx_rollback_or_clean_is_active -> trx_rollback_is_active
trx_rollback_or_clean_recovered() -> trx_rollback_recovered()
trx_cleanup_at_db_startup() -> trx_cleanup_recovered()

Also removed a hack from lock_trx_release_locks(). Instead let recovery
rollback thread to skip committed XA transactions.
parent 07c0bac0
...@@ -7782,7 +7782,7 @@ btr_free_externally_stored_field( ...@@ -7782,7 +7782,7 @@ btr_free_externally_stored_field(
MLOG_4BYTES, &mtr); MLOG_4BYTES, &mtr);
/* Zero out the BLOB length. If the server /* Zero out the BLOB length. If the server
crashes during the execution of this function, crashes during the execution of this function,
trx_rollback_or_clean_all_recovered() could trx_rollback_all_recovered() could
dereference the half-deleted BLOB, fetching a dereference the half-deleted BLOB, fetching a
wrong prefix for the BLOB. */ wrong prefix for the BLOB. */
mlog_write_ulint(field_ref + BTR_EXTERN_LEN + 4, mlog_write_ulint(field_ref + BTR_EXTERN_LEN + 4,
......
...@@ -33,7 +33,7 @@ Created 3/26/1996 Heikki Tuuri ...@@ -33,7 +33,7 @@ Created 3/26/1996 Heikki Tuuri
#include "mtr0mtr.h" #include "mtr0mtr.h"
#include "trx0sys.h" #include "trx0sys.h"
extern bool trx_rollback_or_clean_is_active; extern bool trx_rollback_is_active;
extern const trx_t* trx_roll_crash_recv_trx; extern const trx_t* trx_roll_crash_recv_trx;
/*******************************************************************//** /*******************************************************************//**
...@@ -71,12 +71,11 @@ trx_roll_must_shutdown(); ...@@ -71,12 +71,11 @@ trx_roll_must_shutdown();
Rollback or clean up any incomplete transactions which were Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was encountered in crash recovery. If the transaction already was
committed, then we clean up a possible insert undo log. If the committed, then we clean up a possible insert undo log. If the
transaction was not yet committed, then we roll it back. */ transaction was not yet committed, then we roll it back.
@param all true=roll back all recovered active transactions;
false=roll back any incomplete dictionary transaction */
void void
trx_rollback_or_clean_recovered( trx_rollback_recovered(bool all);
/*============================*/
ibool all); /*!< in: FALSE=roll back dictionary transactions;
TRUE=roll back all non-PREPARED transactions */
/*******************************************************************//** /*******************************************************************//**
Rollback or clean up any incomplete transactions which were Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was encountered in crash recovery. If the transaction already was
...@@ -86,11 +85,7 @@ Note: this is done in a background thread. ...@@ -86,11 +85,7 @@ Note: this is done in a background thread.
@return a dummy parameter */ @return a dummy parameter */
extern "C" extern "C"
os_thread_ret_t os_thread_ret_t
DECLARE_THREAD(trx_rollback_or_clean_all_recovered)( DECLARE_THREAD(trx_rollback_all_recovered)(void*);
/*================================================*/
void* arg MY_ATTRIBUTE((unused)));
/*!< in: a dummy parameter required by
os_thread_create */
/*********************************************************************//** /*********************************************************************//**
Creates a rollback command node struct. Creates a rollback command node struct.
@return own: rollback node struct */ @return own: rollback node struct */
......
...@@ -246,14 +246,6 @@ trx_commit_low( ...@@ -246,14 +246,6 @@ trx_commit_low(
trx_t* trx, /*!< in/out: transaction */ trx_t* trx, /*!< in/out: transaction */
mtr_t* mtr); /*!< in/out: mini-transaction (will be committed), mtr_t* mtr); /*!< in/out: mini-transaction (will be committed),
or NULL if trx made no modifications */ or NULL if trx made no modifications */
/****************************************************************//**
Cleans up a transaction at database startup. The cleanup is needed if
the transaction already got to the middle of a commit when the database
crashed, and we cannot roll it back. */
void
trx_cleanup_at_db_startup(
/*======================*/
trx_t* trx); /*!< in: transaction */
/**********************************************************************//** /**********************************************************************//**
Does the transaction commit for MySQL. Does the transaction commit for MySQL.
@return DB_SUCCESS or error number */ @return DB_SUCCESS or error number */
......
...@@ -7630,8 +7630,6 @@ lock_trx_release_locks( ...@@ -7630,8 +7630,6 @@ lock_trx_release_locks(
lock_mutex_enter(); lock_mutex_enter();
} }
trx_mutex_enter(trx);
/* The following assignment makes the transaction committed in memory /* The following assignment makes the transaction committed in memory
and makes its changes to data visible to other transactions. and makes its changes to data visible to other transactions.
NOTE that there is a small discrepancy from the strict formal NOTE that there is a small discrepancy from the strict formal
...@@ -7647,7 +7645,9 @@ lock_trx_release_locks( ...@@ -7647,7 +7645,9 @@ lock_trx_release_locks(
committed. */ committed. */
/*--------------------------------------*/ /*--------------------------------------*/
trx_mutex_enter(trx);
trx->state = TRX_STATE_COMMITTED_IN_MEMORY; trx->state = TRX_STATE_COMMITTED_IN_MEMORY;
trx_mutex_exit(trx);
/*--------------------------------------*/ /*--------------------------------------*/
if (trx->is_referenced()) { if (trx->is_referenced()) {
...@@ -7658,41 +7658,18 @@ lock_trx_release_locks( ...@@ -7658,41 +7658,18 @@ lock_trx_release_locks(
while (trx->is_referenced()) { while (trx->is_referenced()) {
trx_mutex_exit(trx);
DEBUG_SYNC_C("waiting_trx_is_not_referenced"); DEBUG_SYNC_C("waiting_trx_is_not_referenced");
/** Doing an implicit to explicit conversion /** Doing an implicit to explicit conversion
should not be expensive. */ should not be expensive. */
ut_delay(ut_rnd_interval(0, srv_spin_wait_delay)); ut_delay(ut_rnd_interval(0, srv_spin_wait_delay));
trx_mutex_enter(trx);
} }
trx_mutex_exit(trx);
lock_mutex_enter(); lock_mutex_enter();
trx_mutex_enter(trx);
} }
ut_ad(!trx->is_referenced()); ut_ad(!trx->is_referenced());
/* If the background thread trx_rollback_or_clean_recovered()
is still active then there is a chance that the rollback
thread may see this trx as COMMITTED_IN_MEMORY and goes ahead
to clean it up calling trx_cleanup_at_db_startup(). This can
happen in the case we are committing a trx here that is left
in PREPARED state during the crash. Note that commit of the
rollback of a PREPARED trx happens in the recovery thread
while the rollback of other transactions happen in the
background thread. To avoid this race we unconditionally unset
the is_recovered flag. */
trx->is_recovered = false;
trx_mutex_exit(trx);
if (release_lock) { if (release_lock) {
lock_release(trx); lock_release(trx);
......
...@@ -1949,7 +1949,7 @@ logs_empty_and_mark_files_at_shutdown(void) ...@@ -1949,7 +1949,7 @@ logs_empty_and_mark_files_at_shutdown(void)
goto wait_suspend_loop; goto wait_suspend_loop;
} else if (btr_defragment_thread_active) { } else if (btr_defragment_thread_active) {
thread_name = "btr_defragment_thread"; thread_name = "btr_defragment_thread";
} else if (srv_fast_shutdown != 2 && trx_rollback_or_clean_is_active) { } else if (srv_fast_shutdown != 2 && trx_rollback_is_active) {
thread_name = "rollback of recovered transactions"; thread_name = "rollback of recovered transactions";
} else { } else {
thread_name = NULL; thread_name = NULL;
......
...@@ -3426,8 +3426,8 @@ recv_recovery_rollback_active(void) ...@@ -3426,8 +3426,8 @@ recv_recovery_rollback_active(void)
/* Rollback the uncommitted transactions which have no user /* Rollback the uncommitted transactions which have no user
session */ session */
trx_rollback_or_clean_is_active = true; trx_rollback_is_active = true;
os_thread_create(trx_rollback_or_clean_all_recovered, 0, 0); os_thread_create(trx_rollback_all_recovered, 0, 0);
} }
} }
......
...@@ -70,7 +70,7 @@ row_ext_cache_fill( ...@@ -70,7 +70,7 @@ row_ext_cache_fill(
} else { } else {
/* Fetch at most ext->max_len of the column. /* Fetch at most ext->max_len of the column.
The column should be non-empty. However, The column should be non-empty. However,
trx_rollback_or_clean_all_recovered() may try to trx_rollback_all_recovered() may try to
access a half-deleted BLOB if the server previously access a half-deleted BLOB if the server previously
crashed during the execution of crashed during the execution of
btr_free_externally_stored_field(). */ btr_free_externally_stored_field(). */
......
...@@ -1649,7 +1649,7 @@ innobase_start_or_create_for_mysql() ...@@ -1649,7 +1649,7 @@ innobase_start_or_create_for_mysql()
+ 1 /* dict_stats_thread */ + 1 /* dict_stats_thread */
+ 1 /* fts_optimize_thread */ + 1 /* fts_optimize_thread */
+ 1 /* recv_writer_thread */ + 1 /* recv_writer_thread */
+ 1 /* trx_rollback_or_clean_all_recovered */ + 1 /* trx_rollback_all_recovered */
+ 128 /* added as margin, for use of + 128 /* added as margin, for use of
InnoDB Memcached etc. */ InnoDB Memcached etc. */
+ max_connections + max_connections
...@@ -2471,7 +2471,7 @@ innobase_start_or_create_for_mysql() ...@@ -2471,7 +2471,7 @@ innobase_start_or_create_for_mysql()
The data dictionary latch should guarantee that there is at The data dictionary latch should guarantee that there is at
most one data dictionary transaction active at a time. */ most one data dictionary transaction active at a time. */
if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO) { if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO) {
trx_rollback_or_clean_recovered(FALSE); trx_rollback_recovered(false);
} }
/* Fix-up truncate of tables in the system tablespace /* Fix-up truncate of tables in the system tablespace
......
...@@ -300,7 +300,7 @@ trx_purge_add_undo_to_history(const trx_t* trx, trx_undo_t*& undo, mtr_t* mtr) ...@@ -300,7 +300,7 @@ trx_purge_add_undo_to_history(const trx_t* trx, trx_undo_t*& undo, mtr_t* mtr)
continue to execute user transactions. */ continue to execute user transactions. */
ut_ad(srv_undo_sources ut_ad(srv_undo_sources
|| ((srv_startup_is_before_trx_rollback_phase || ((srv_startup_is_before_trx_rollback_phase
|| trx_rollback_or_clean_is_active) || trx_rollback_is_active)
&& purge_sys->state == PURGE_STATE_INIT) && purge_sys->state == PURGE_STATE_INIT)
|| (srv_force_recovery >= SRV_FORCE_NO_BACKGROUND || (srv_force_recovery >= SRV_FORCE_NO_BACKGROUND
&& purge_sys->state == PURGE_STATE_DISABLED) && purge_sys->state == PURGE_STATE_DISABLED)
......
...@@ -54,8 +54,8 @@ Created 3/26/1996 Heikki Tuuri ...@@ -54,8 +54,8 @@ Created 3/26/1996 Heikki Tuuri
rollback */ rollback */
static const ulint TRX_ROLL_TRUNC_THRESHOLD = 1; static const ulint TRX_ROLL_TRUNC_THRESHOLD = 1;
/** true if trx_rollback_or_clean_all_recovered() thread is active */ /** true if trx_rollback_all_recovered() thread is active */
bool trx_rollback_or_clean_is_active; bool trx_rollback_is_active;
/** In crash recovery, the current trx to be rolled back; NULL otherwise */ /** In crash recovery, the current trx to be rolled back; NULL otherwise */
const trx_t* trx_roll_crash_recv_trx; const trx_t* trx_roll_crash_recv_trx;
...@@ -734,7 +734,7 @@ ibool ...@@ -734,7 +734,7 @@ ibool
trx_rollback_resurrected( trx_rollback_resurrected(
/*=====================*/ /*=====================*/
trx_t* trx, /*!< in: transaction to rollback or clean */ trx_t* trx, /*!< in: transaction to rollback or clean */
ibool* all) /*!< in/out: FALSE=roll back dictionary transactions; bool* all) /*!< in/out: FALSE=roll back dictionary transactions;
TRUE=roll back all non-PREPARED transactions */ TRUE=roll back all non-PREPARED transactions */
{ {
ut_ad(trx_sys_mutex_own()); ut_ad(trx_sys_mutex_own());
...@@ -752,14 +752,6 @@ trx_rollback_resurrected( ...@@ -752,14 +752,6 @@ trx_rollback_resurrected(
} }
switch (trx->state) { switch (trx->state) {
case TRX_STATE_COMMITTED_IN_MEMORY:
trx_mutex_exit(trx);
trx_sys_mutex_exit();
ib::info() << "Cleaning up trx with id " << ib::hex(trx->id);
trx_cleanup_at_db_startup(trx);
trx_free_resurrected(trx);
return(TRUE);
case TRX_STATE_ACTIVE: case TRX_STATE_ACTIVE:
if (!srv_is_being_started if (!srv_is_being_started
&& !srv_undo_sources && srv_fast_shutdown) { && !srv_undo_sources && srv_fast_shutdown) {
...@@ -767,7 +759,7 @@ trx_rollback_resurrected( ...@@ -767,7 +759,7 @@ trx_rollback_resurrected(
trx->state = TRX_STATE_PREPARED; trx->state = TRX_STATE_PREPARED;
trx_sys->n_prepared_trx++; trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++; trx_sys->n_prepared_recovered_trx++;
*all = FALSE; *all = false;
goto func_exit; goto func_exit;
} }
trx_mutex_exit(trx); trx_mutex_exit(trx);
...@@ -788,6 +780,8 @@ trx_rollback_resurrected( ...@@ -788,6 +780,8 @@ trx_rollback_resurrected(
return(TRUE); return(TRUE);
} }
return(FALSE); return(FALSE);
case TRX_STATE_COMMITTED_IN_MEMORY:
ut_ad(trx->xid);
case TRX_STATE_PREPARED: case TRX_STATE_PREPARED:
goto func_exit; goto func_exit;
case TRX_STATE_NOT_STARTED: case TRX_STATE_NOT_STARTED:
...@@ -848,12 +842,11 @@ trx_roll_must_shutdown() ...@@ -848,12 +842,11 @@ trx_roll_must_shutdown()
Rollback or clean up any incomplete transactions which were Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was encountered in crash recovery. If the transaction already was
committed, then we clean up a possible insert undo log. If the committed, then we clean up a possible insert undo log. If the
transaction was not yet committed, then we roll it back. */ transaction was not yet committed, then we roll it back.
@param all true=roll back all recovered active transactions;
false=roll back any incomplete dictionary transaction */
void void
trx_rollback_or_clean_recovered( trx_rollback_recovered(bool all)
/*============================*/
ibool all) /*!< in: FALSE=roll back dictionary transactions;
TRUE=roll back all non-PREPARED transactions */
{ {
trx_t* trx; trx_t* trx;
...@@ -917,11 +910,7 @@ Note: this is done in a background thread. ...@@ -917,11 +910,7 @@ Note: this is done in a background thread.
@return a dummy parameter */ @return a dummy parameter */
extern "C" extern "C"
os_thread_ret_t os_thread_ret_t
DECLARE_THREAD(trx_rollback_or_clean_all_recovered)( DECLARE_THREAD(trx_rollback_all_recovered)(void*)
/*================================================*/
void* arg MY_ATTRIBUTE((unused)))
/*!< in: a dummy parameter required by
os_thread_create */
{ {
my_thread_init(); my_thread_init();
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
...@@ -930,9 +919,9 @@ DECLARE_THREAD(trx_rollback_or_clean_all_recovered)( ...@@ -930,9 +919,9 @@ DECLARE_THREAD(trx_rollback_or_clean_all_recovered)(
pfs_register_thread(trx_rollback_clean_thread_key); pfs_register_thread(trx_rollback_clean_thread_key);
#endif /* UNIV_PFS_THREAD */ #endif /* UNIV_PFS_THREAD */
trx_rollback_or_clean_recovered(TRUE); trx_rollback_recovered(true);
trx_rollback_or_clean_is_active = false; trx_rollback_is_active = false;
my_thread_end(); my_thread_end();
/* We count the number of threads in os_thread_exit(). A created /* We count the number of threads in os_thread_exit(). A created
......
...@@ -815,167 +815,6 @@ trx_resurrect_table_locks( ...@@ -815,167 +815,6 @@ trx_resurrect_table_locks(
} }
} }
/****************************************************************//**
Resurrect the transactions that were doing inserts the time of the
crash, they need to be undone.
@return trx_t instance */
static
trx_t*
trx_resurrect_insert(
/*=================*/
trx_undo_t* undo, /*!< in: entry to UNDO */
trx_rseg_t* rseg) /*!< in: rollback segment */
{
trx_t* trx;
trx = trx_allocate_for_background();
ut_d(trx->start_file = __FILE__);
ut_d(trx->start_line = __LINE__);
trx->rsegs.m_redo.rseg = rseg;
/* For transactions with active data will not have rseg size = 1
or will not qualify for purge limit criteria. So it is safe to increment
this trx_ref_count w/o mutex protection. */
++trx->rsegs.m_redo.rseg->trx_ref_count;
*trx->xid = undo->xid;
trx->id = undo->trx_id;
trx->rsegs.m_redo.old_insert = undo;
trx->is_recovered = true;
/* This is single-threaded startup code, we do not need the
protection of trx->mutex or trx_sys->mutex here. */
if (undo->state != TRX_UNDO_ACTIVE) {
/* Prepared transactions are left in the prepared state
waiting for a commit or abort decision from MySQL */
if (undo->state == TRX_UNDO_PREPARED) {
ib::info() << "Transaction "
<< trx_get_id_for_print(trx)
<< " was in the XA prepared state.";
trx->state = TRX_STATE_PREPARED;
trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++;
} else {
trx->state = TRX_STATE_COMMITTED_IN_MEMORY;
}
/* We give a dummy value for the trx no; this should have no
relevance since purge is not interested in committed
transaction numbers, unless they are in the history
list, in which case it looks the number from the disk based
undo log structure */
trx->no = trx->id;
} else {
trx->state = TRX_STATE_ACTIVE;
/* A running transaction always has the number
field inited to TRX_ID_MAX */
trx->no = TRX_ID_MAX;
}
if (undo->dict_operation) {
trx_set_dict_operation(trx, TRX_DICT_OP_TABLE);
trx->table_id = undo->table_id;
}
if (!undo->empty) {
trx->undo_no = undo->top_undo_no + 1;
trx->undo_rseg_space = undo->rseg->space;
}
return(trx);
}
/****************************************************************//**
Prepared transactions are left in the prepared state waiting for a
commit or abort decision from MySQL */
static
void
trx_resurrect_update_in_prepared_state(
/*===================================*/
trx_t* trx, /*!< in,out: transaction */
const trx_undo_t* undo) /*!< in: update UNDO record */
{
/* This is single-threaded startup code, we do not need the
protection of trx->mutex or trx_sys->mutex here. */
if (undo->state == TRX_UNDO_PREPARED) {
ib::info() << "Transaction " << trx_get_id_for_print(trx)
<< " was in the XA prepared state.";
if (trx_state_eq(trx, TRX_STATE_NOT_STARTED)) {
trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++;
} else {
ut_ad(trx_state_eq(trx, TRX_STATE_PREPARED));
}
trx->state = TRX_STATE_PREPARED;
} else {
trx->state = TRX_STATE_COMMITTED_IN_MEMORY;
}
}
/****************************************************************//**
Resurrect the transactions that were doing updates the time of the
crash, they need to be undone. */
static
void
trx_resurrect_update(
/*=================*/
trx_t* trx, /*!< in/out: transaction */
trx_undo_t* undo, /*!< in/out: update UNDO record */
trx_rseg_t* rseg) /*!< in/out: rollback segment */
{
trx->rsegs.m_redo.rseg = rseg;
/* For transactions with active data will not have rseg size = 1
or will not qualify for purge limit criteria. So it is safe to increment
this trx_ref_count w/o mutex protection. */
++trx->rsegs.m_redo.rseg->trx_ref_count;
*trx->xid = undo->xid;
trx->id = undo->trx_id;
trx->rsegs.m_redo.undo = undo;
trx->is_recovered = true;
/* This is single-threaded startup code, we do not need the
protection of trx->mutex or trx_sys->mutex here. */
if (undo->state != TRX_UNDO_ACTIVE) {
trx_resurrect_update_in_prepared_state(trx, undo);
/* We give a dummy value for the trx number */
trx->no = trx->id;
} else {
trx->state = TRX_STATE_ACTIVE;
/* A running transaction always has the number field inited to
TRX_ID_MAX */
trx->no = TRX_ID_MAX;
}
if (undo->dict_operation) {
trx_set_dict_operation(trx, TRX_DICT_OP_TABLE);
trx->table_id = undo->table_id;
}
if (!undo->empty && undo->top_undo_no >= trx->undo_no) {
trx->undo_no = undo->top_undo_no + 1;
trx->undo_rseg_space = undo->rseg->space;
}
}
/** Mapping read-write transactions from id to transaction instance, for /** Mapping read-write transactions from id to transaction instance, for
creating read views and during trx id lookup for MVCC and locking. */ creating read views and during trx id lookup for MVCC and locking. */
struct TrxTrack { struct TrxTrack {
...@@ -1004,24 +843,86 @@ struct TrxTrackCmp { ...@@ -1004,24 +843,86 @@ struct TrxTrackCmp {
typedef std::set<TrxTrack, TrxTrackCmp, ut_allocator<TrxTrack> > typedef std::set<TrxTrack, TrxTrackCmp, ut_allocator<TrxTrack> >
TrxIdSet; TrxIdSet;
static inline void trx_sys_add_trx_at_init(trx_t *trx, trx_undo_t *undo,
uint64_t *rows_to_undo,
TrxIdSet *set)
{
ut_ad(trx->id != 0);
ut_ad(trx->is_recovered);
set->insert(TrxTrack(trx->id, trx)); /**
if (trx_state_eq(trx, TRX_STATE_ACTIVE) || Resurrect the transactions that were doing inserts/updates the time of the
trx_state_eq(trx, TRX_STATE_PREPARED)) crash, they need to be undone.
*/
static void trx_resurrect(trx_undo_t *undo, trx_rseg_t *rseg,
ib_time_t start_time, uint64_t *rows_to_undo,
TrxIdSet *set, bool is_old_insert)
{
trx_state_t state;
/*
This is single-threaded startup code, we do not need the
protection of trx->mutex or trx_sys->mutex here.
*/
switch (undo->state)
{
case TRX_UNDO_ACTIVE:
state= TRX_STATE_ACTIVE;
break;
case TRX_UNDO_PREPARED:
/*
Prepared transactions are left in the prepared state
waiting for a commit or abort decision from MySQL
*/
ib::info() << "Transaction " << undo->trx_id
<< " was in the XA prepared state.";
state= TRX_STATE_PREPARED;
trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++;
break;
default:
if (is_old_insert && srv_force_recovery < SRV_FORCE_NO_TRX_UNDO)
trx_undo_commit_cleanup(undo, false);
return;
}
trx_t *trx= trx_allocate_for_background();
trx->state= state;
ut_d(trx->start_file= __FILE__);
ut_d(trx->start_line= __LINE__);
ut_ad(trx->no == TRX_ID_MAX);
if (is_old_insert)
trx->rsegs.m_redo.old_insert= undo;
else
trx->rsegs.m_redo.undo= undo;
if (!undo->empty)
{
trx->undo_no= undo->top_undo_no + 1;
trx->undo_rseg_space= undo->rseg->space;
}
trx->rsegs.m_redo.rseg= rseg;
/*
For transactions with active data will not have rseg size = 1
or will not qualify for purge limit criteria. So it is safe to increment
this trx_ref_count w/o mutex protection.
*/
++trx->rsegs.m_redo.rseg->trx_ref_count;
*trx->xid= undo->xid;
trx->id= undo->trx_id;
trx->is_recovered= true;
trx->start_time= start_time;
if (undo->dict_operation)
{ {
trx_sys->rw_trx_hash.insert(trx); trx_set_dict_operation(trx, TRX_DICT_OP_TABLE);
trx_sys->rw_trx_hash.put_pins(trx); trx->table_id= undo->table_id;
trx_sys->rw_trx_ids.push_back(trx->id);
trx_resurrect_table_locks(trx, undo);
if (trx_state_eq(trx, TRX_STATE_ACTIVE))
*rows_to_undo+= trx->undo_no;
} }
set->insert(TrxTrack(trx->id, trx));
trx_sys->rw_trx_hash.insert(trx);
trx_sys->rw_trx_hash.put_pins(trx);
trx_sys->rw_trx_ids.push_back(trx->id);
trx_resurrect_table_locks(trx, undo);
if (trx_state_eq(trx, TRX_STATE_ACTIVE))
*rows_to_undo+= trx->undo_no;
#ifdef UNIV_DEBUG #ifdef UNIV_DEBUG
trx->in_rw_trx_list= true; trx->in_rw_trx_list= true;
if (trx->id > trx_sys->rw_max_trx_id) if (trx->id > trx_sys->rw_max_trx_id)
...@@ -1029,6 +930,7 @@ static inline void trx_sys_add_trx_at_init(trx_t *trx, trx_undo_t *undo, ...@@ -1029,6 +930,7 @@ static inline void trx_sys_add_trx_at_init(trx_t *trx, trx_undo_t *undo,
#endif #endif
} }
/** Initialize (resurrect) transactions at startup. */ /** Initialize (resurrect) transactions at startup. */
void void
trx_lists_init_at_db_start() trx_lists_init_at_db_start()
...@@ -1061,36 +963,50 @@ trx_lists_init_at_db_start() ...@@ -1061,36 +963,50 @@ trx_lists_init_at_db_start()
/* Resurrect transactions that were doing inserts /* Resurrect transactions that were doing inserts
using the old separate insert_undo log. */ using the old separate insert_undo log. */
for (undo = UT_LIST_GET_FIRST(rseg->old_insert_list); undo = UT_LIST_GET_FIRST(rseg->old_insert_list);
undo != NULL; while (undo) {
undo = UT_LIST_GET_NEXT(undo_list, undo)) { trx_undo_t* next = UT_LIST_GET_NEXT(undo_list, undo);
trx_t* trx = trx_resurrect_insert(undo, rseg); trx_resurrect(undo, rseg, start_time, &rows_to_undo,
trx->start_time = start_time; &set, true);
trx_sys_add_trx_at_init(trx, undo, &rows_to_undo, undo = next;
&set);
} }
/* Ressurrect other transactions. */ /* Ressurrect other transactions. */
for (undo = UT_LIST_GET_FIRST(rseg->undo_list); for (undo = UT_LIST_GET_FIRST(rseg->undo_list);
undo != NULL; undo != NULL;
undo = UT_LIST_GET_NEXT(undo_list, undo)) { undo = UT_LIST_GET_NEXT(undo_list, undo)) {
trx_t *trx = trx_sys->rw_trx_hash.find(undo->trx_id);
/* Check if trx_id was already registered first. */ if (!trx) {
TrxIdSet::iterator it = trx_resurrect(undo, rseg, start_time,
set.find(TrxTrack(undo->trx_id)); &rows_to_undo, &set, false);
trx_t *trx= it == set.end() ? 0 : it->m_trx; } else {
ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE) ||
if (trx == NULL) { trx_state_eq(trx, TRX_STATE_PREPARED));
trx = trx_allocate_for_background(); ut_ad(trx->start_time == start_time);
trx->start_time = start_time; ut_ad(trx->is_recovered);
ut_ad(trx->rsegs.m_redo.rseg == rseg);
ut_d(trx->start_file = __FILE__); ut_ad(trx->rsegs.m_redo.rseg->trx_ref_count);
ut_d(trx->start_line = __LINE__);
trx->rsegs.m_redo.undo = undo;
if (!undo->empty
&& undo->top_undo_no >= trx->undo_no) {
if (trx_state_eq(trx,
TRX_STATE_ACTIVE)) {
rows_to_undo -= trx->undo_no;
rows_to_undo +=
undo->top_undo_no + 1;
}
trx->undo_no = undo->top_undo_no + 1;
trx->undo_rseg_space =
undo->rseg->space;
}
trx_resurrect_table_locks(trx, undo);
if (trx_state_eq(trx, TRX_STATE_PREPARED)) {
trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++;
}
} }
trx_resurrect_update(trx, undo, rseg);
trx_sys_add_trx_at_init(trx, undo, &rows_to_undo,
&set);
} }
} }
...@@ -2046,50 +1962,6 @@ trx_commit( ...@@ -2046,50 +1962,6 @@ trx_commit(
trx_commit_low(trx, mtr); trx_commit_low(trx, mtr);
} }
/****************************************************************//**
Cleans up a transaction at database startup. The cleanup is needed if
the transaction already got to the middle of a commit when the database
crashed, and we cannot roll it back. */
void
trx_cleanup_at_db_startup(
/*======================*/
trx_t* trx) /*!< in: transaction */
{
ut_ad(trx->is_recovered);
ut_ad(!trx->rsegs.m_noredo.undo);
ut_ad(!trx->rsegs.m_redo.undo);
if (trx_undo_t*& undo = trx->rsegs.m_redo.old_insert) {
ut_ad(undo->rseg == trx->rsegs.m_redo.rseg);
trx_undo_commit_cleanup(undo, false);
undo = NULL;
}
memset(&trx->rsegs, 0x0, sizeof(trx->rsegs));
trx->undo_no = 0;
trx->undo_rseg_space = 0;
trx->last_sql_stat_start.least_undo_no = 0;
trx_sys_mutex_enter();
ut_a(!trx->read_only);
UT_LIST_REMOVE(trx_sys->rw_trx_list, trx);
ut_d(trx->in_rw_trx_list = FALSE);
trx_sys_mutex_exit();
/* Change the transaction state without mutex protection, now
that it no longer is in the trx_list. Recovered transactions
are never placed in the mysql_trx_list. */
ut_ad(trx->is_recovered);
ut_ad(!trx->in_rw_trx_list);
ut_ad(!trx->in_mysql_trx_list);
DBUG_LOG("trx", "Cleanup at startup: " << trx);
trx->state = TRX_STATE_NOT_STARTED;
}
/********************************************************************//** /********************************************************************//**
Assigns a read view for a consistent read query. All the consistent reads Assigns a read view for a consistent read query. All the consistent reads
within the same transaction will get the same read view, which is created within the same transaction will get the same read view, which is created
......
...@@ -1643,7 +1643,6 @@ trx_undo_free_prepared( ...@@ -1643,7 +1643,6 @@ trx_undo_free_prepared(
/* fall through */ /* fall through */
case TRX_UNDO_ACTIVE: case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns /* lock_trx_release_locks() assigns
trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY, trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked also for transactions that we faked
to TRX_STATE_PREPARED in trx_rollback_resurrected(). */ to TRX_STATE_PREPARED in trx_rollback_resurrected(). */
...@@ -1673,7 +1672,6 @@ trx_undo_free_prepared( ...@@ -1673,7 +1672,6 @@ trx_undo_free_prepared(
/* fall through */ /* fall through */
case TRX_UNDO_ACTIVE: case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns /* lock_trx_release_locks() assigns
trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY, trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked also for transactions that we faked
to TRX_STATE_PREPARED in trx_rollback_resurrected(). */ to TRX_STATE_PREPARED in trx_rollback_resurrected(). */
......
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