summaryrefslogtreecommitdiff
path: root/storage/innobase
diff options
context:
space:
mode:
authorSergey Vojtovich <svoj@mariadb.org>2017-12-21 15:45:40 +0400
committerSergey Vojtovich <svoj@mariadb.org>2018-01-20 16:09:26 +0400
commitd09f14693406ea7612a7010917b39b895d77593f (patch)
tree75fbe7cf853404ab1e3a769e2752fce5b9334f99 /storage/innobase
parent07c0bac03952e4b8509e0858def5c85a2b09b4eb (diff)
downloadmariadb-git-d09f14693406ea7612a7010917b39b895d77593f.tar.gz
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.
Diffstat (limited to 'storage/innobase')
-rw-r--r--storage/innobase/btr/btr0cur.cc2
-rw-r--r--storage/innobase/include/trx0roll.h17
-rw-r--r--storage/innobase/include/trx0trx.h8
-rw-r--r--storage/innobase/lock/lock0lock.cc27
-rw-r--r--storage/innobase/log/log0log.cc2
-rw-r--r--storage/innobase/log/log0recv.cc4
-rw-r--r--storage/innobase/row/row0ext.cc2
-rw-r--r--storage/innobase/srv/srv0start.cc4
-rw-r--r--storage/innobase/trx/trx0purge.cc2
-rw-r--r--storage/innobase/trx/trx0roll.cc37
-rw-r--r--storage/innobase/trx/trx0trx.cc358
-rw-r--r--storage/innobase/trx/trx0undo.cc2
12 files changed, 144 insertions, 321 deletions
diff --git a/storage/innobase/btr/btr0cur.cc b/storage/innobase/btr/btr0cur.cc
index 078375e1b28..4ae2825504f 100644
--- a/storage/innobase/btr/btr0cur.cc
+++ b/storage/innobase/btr/btr0cur.cc
@@ -7782,7 +7782,7 @@ btr_free_externally_stored_field(
MLOG_4BYTES, &mtr);
/* Zero out the BLOB length. If the server
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
wrong prefix for the BLOB. */
mlog_write_ulint(field_ref + BTR_EXTERN_LEN + 4,
diff --git a/storage/innobase/include/trx0roll.h b/storage/innobase/include/trx0roll.h
index f7b999ae70a..ba9c901d4f7 100644
--- a/storage/innobase/include/trx0roll.h
+++ b/storage/innobase/include/trx0roll.h
@@ -33,7 +33,7 @@ Created 3/26/1996 Heikki Tuuri
#include "mtr0mtr.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;
/*******************************************************************//**
@@ -71,12 +71,11 @@ trx_roll_must_shutdown();
Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was
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
-trx_rollback_or_clean_recovered(
-/*============================*/
- ibool all); /*!< in: FALSE=roll back dictionary transactions;
- TRUE=roll back all non-PREPARED transactions */
+trx_rollback_recovered(bool all);
/*******************************************************************//**
Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was
@@ -86,11 +85,7 @@ Note: this is done in a background thread.
@return a dummy parameter */
extern "C"
os_thread_ret_t
-DECLARE_THREAD(trx_rollback_or_clean_all_recovered)(
-/*================================================*/
- void* arg MY_ATTRIBUTE((unused)));
- /*!< in: a dummy parameter required by
- os_thread_create */
+DECLARE_THREAD(trx_rollback_all_recovered)(void*);
/*********************************************************************//**
Creates a rollback command node struct.
@return own: rollback node struct */
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 28f6f2eff12..b7f40a83103 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -246,14 +246,6 @@ trx_commit_low(
trx_t* trx, /*!< in/out: transaction */
mtr_t* mtr); /*!< in/out: mini-transaction (will be committed),
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.
@return DB_SUCCESS or error number */
diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc
index 09b900d1554..09d13b5c8c7 100644
--- a/storage/innobase/lock/lock0lock.cc
+++ b/storage/innobase/lock/lock0lock.cc
@@ -7630,8 +7630,6 @@ lock_trx_release_locks(
lock_mutex_enter();
}
- trx_mutex_enter(trx);
-
/* The following assignment makes the transaction committed in memory
and makes its changes to data visible to other transactions.
NOTE that there is a small discrepancy from the strict formal
@@ -7647,7 +7645,9 @@ lock_trx_release_locks(
committed. */
/*--------------------------------------*/
+ trx_mutex_enter(trx);
trx->state = TRX_STATE_COMMITTED_IN_MEMORY;
+ trx_mutex_exit(trx);
/*--------------------------------------*/
if (trx->is_referenced()) {
@@ -7658,41 +7658,18 @@ lock_trx_release_locks(
while (trx->is_referenced()) {
- trx_mutex_exit(trx);
-
DEBUG_SYNC_C("waiting_trx_is_not_referenced");
/** Doing an implicit to explicit conversion
should not be expensive. */
ut_delay(ut_rnd_interval(0, srv_spin_wait_delay));
-
- trx_mutex_enter(trx);
}
- trx_mutex_exit(trx);
-
lock_mutex_enter();
-
- trx_mutex_enter(trx);
}
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) {
lock_release(trx);
diff --git a/storage/innobase/log/log0log.cc b/storage/innobase/log/log0log.cc
index b7b3b6db066..0b601a78adc 100644
--- a/storage/innobase/log/log0log.cc
+++ b/storage/innobase/log/log0log.cc
@@ -1949,7 +1949,7 @@ loop:
goto wait_suspend_loop;
} else if (btr_defragment_thread_active) {
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";
} else {
thread_name = NULL;
diff --git a/storage/innobase/log/log0recv.cc b/storage/innobase/log/log0recv.cc
index cacf823f596..7c6369e57c2 100644
--- a/storage/innobase/log/log0recv.cc
+++ b/storage/innobase/log/log0recv.cc
@@ -3426,8 +3426,8 @@ recv_recovery_rollback_active(void)
/* Rollback the uncommitted transactions which have no user
session */
- trx_rollback_or_clean_is_active = true;
- os_thread_create(trx_rollback_or_clean_all_recovered, 0, 0);
+ trx_rollback_is_active = true;
+ os_thread_create(trx_rollback_all_recovered, 0, 0);
}
}
diff --git a/storage/innobase/row/row0ext.cc b/storage/innobase/row/row0ext.cc
index 72c68940d11..503f7d0d3e7 100644
--- a/storage/innobase/row/row0ext.cc
+++ b/storage/innobase/row/row0ext.cc
@@ -70,7 +70,7 @@ row_ext_cache_fill(
} else {
/* Fetch at most ext->max_len of the column.
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
crashed during the execution of
btr_free_externally_stored_field(). */
diff --git a/storage/innobase/srv/srv0start.cc b/storage/innobase/srv/srv0start.cc
index 5b0bdc5f605..25bed7d2f0e 100644
--- a/storage/innobase/srv/srv0start.cc
+++ b/storage/innobase/srv/srv0start.cc
@@ -1649,7 +1649,7 @@ innobase_start_or_create_for_mysql()
+ 1 /* dict_stats_thread */
+ 1 /* fts_optimize_thread */
+ 1 /* recv_writer_thread */
- + 1 /* trx_rollback_or_clean_all_recovered */
+ + 1 /* trx_rollback_all_recovered */
+ 128 /* added as margin, for use of
InnoDB Memcached etc. */
+ max_connections
@@ -2471,7 +2471,7 @@ files_checked:
The data dictionary latch should guarantee that there is at
most one data dictionary transaction active at a time. */
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
diff --git a/storage/innobase/trx/trx0purge.cc b/storage/innobase/trx/trx0purge.cc
index 699db7a38a8..6bf52db20ac 100644
--- a/storage/innobase/trx/trx0purge.cc
+++ b/storage/innobase/trx/trx0purge.cc
@@ -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. */
ut_ad(srv_undo_sources
|| ((srv_startup_is_before_trx_rollback_phase
- || trx_rollback_or_clean_is_active)
+ || trx_rollback_is_active)
&& purge_sys->state == PURGE_STATE_INIT)
|| (srv_force_recovery >= SRV_FORCE_NO_BACKGROUND
&& purge_sys->state == PURGE_STATE_DISABLED)
diff --git a/storage/innobase/trx/trx0roll.cc b/storage/innobase/trx/trx0roll.cc
index be4e4c3294a..a25990a8b36 100644
--- a/storage/innobase/trx/trx0roll.cc
+++ b/storage/innobase/trx/trx0roll.cc
@@ -54,8 +54,8 @@ Created 3/26/1996 Heikki Tuuri
rollback */
static const ulint TRX_ROLL_TRUNC_THRESHOLD = 1;
-/** true if trx_rollback_or_clean_all_recovered() thread is active */
-bool trx_rollback_or_clean_is_active;
+/** true if trx_rollback_all_recovered() thread is active */
+bool trx_rollback_is_active;
/** In crash recovery, the current trx to be rolled back; NULL otherwise */
const trx_t* trx_roll_crash_recv_trx;
@@ -734,7 +734,7 @@ ibool
trx_rollback_resurrected(
/*=====================*/
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 */
{
ut_ad(trx_sys_mutex_own());
@@ -752,14 +752,6 @@ func_exit:
}
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:
if (!srv_is_being_started
&& !srv_undo_sources && srv_fast_shutdown) {
@@ -767,7 +759,7 @@ fake_prepared:
trx->state = TRX_STATE_PREPARED;
trx_sys->n_prepared_trx++;
trx_sys->n_prepared_recovered_trx++;
- *all = FALSE;
+ *all = false;
goto func_exit;
}
trx_mutex_exit(trx);
@@ -788,6 +780,8 @@ fake_prepared:
return(TRUE);
}
return(FALSE);
+ case TRX_STATE_COMMITTED_IN_MEMORY:
+ ut_ad(trx->xid);
case TRX_STATE_PREPARED:
goto func_exit;
case TRX_STATE_NOT_STARTED:
@@ -848,12 +842,11 @@ trx_roll_must_shutdown()
Rollback or clean up any incomplete transactions which were
encountered in crash recovery. If the transaction already was
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
-trx_rollback_or_clean_recovered(
-/*============================*/
- ibool all) /*!< in: FALSE=roll back dictionary transactions;
- TRUE=roll back all non-PREPARED transactions */
+trx_rollback_recovered(bool all)
{
trx_t* trx;
@@ -917,11 +910,7 @@ Note: this is done in a background thread.
@return a dummy parameter */
extern "C"
os_thread_ret_t
-DECLARE_THREAD(trx_rollback_or_clean_all_recovered)(
-/*================================================*/
- void* arg MY_ATTRIBUTE((unused)))
- /*!< in: a dummy parameter required by
- os_thread_create */
+DECLARE_THREAD(trx_rollback_all_recovered)(void*)
{
my_thread_init();
ut_ad(!srv_read_only_mode);
@@ -930,9 +919,9 @@ DECLARE_THREAD(trx_rollback_or_clean_all_recovered)(
pfs_register_thread(trx_rollback_clean_thread_key);
#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();
/* We count the number of threads in os_thread_exit(). A created
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index 490f3d6f26b..c08acbc79bb 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -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
creating read views and during trx id lookup for MVCC and locking. */
struct TrxTrack {
@@ -1004,24 +843,86 @@ struct TrxTrackCmp {
typedef std::set<TrxTrack, TrxTrackCmp, ut_allocator<TrxTrack> >
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) ||
- trx_state_eq(trx, TRX_STATE_PREPARED))
+/**
+ Resurrect the transactions that were doing inserts/updates the time of the
+ 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_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;
+ trx_set_dict_operation(trx, TRX_DICT_OP_TABLE);
+ trx->table_id= undo->table_id;
}
+
+ 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
trx->in_rw_trx_list= true;
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,
#endif
}
+
/** Initialize (resurrect) transactions at startup. */
void
trx_lists_init_at_db_start()
@@ -1061,36 +963,50 @@ trx_lists_init_at_db_start()
/* Resurrect transactions that were doing inserts
using the old separate insert_undo log. */
- for (undo = UT_LIST_GET_FIRST(rseg->old_insert_list);
- undo != NULL;
- undo = UT_LIST_GET_NEXT(undo_list, undo)) {
- trx_t* trx = trx_resurrect_insert(undo, rseg);
- trx->start_time = start_time;
- trx_sys_add_trx_at_init(trx, undo, &rows_to_undo,
- &set);
+ undo = UT_LIST_GET_FIRST(rseg->old_insert_list);
+ while (undo) {
+ trx_undo_t* next = UT_LIST_GET_NEXT(undo_list, undo);
+ trx_resurrect(undo, rseg, start_time, &rows_to_undo,
+ &set, true);
+ undo = next;
}
/* Ressurrect other transactions. */
for (undo = UT_LIST_GET_FIRST(rseg->undo_list);
undo != NULL;
undo = UT_LIST_GET_NEXT(undo_list, undo)) {
-
- /* Check if trx_id was already registered first. */
- TrxIdSet::iterator it =
- set.find(TrxTrack(undo->trx_id));
- trx_t *trx= it == set.end() ? 0 : it->m_trx;
-
- if (trx == NULL) {
- trx = trx_allocate_for_background();
- trx->start_time = start_time;
-
- ut_d(trx->start_file = __FILE__);
- ut_d(trx->start_line = __LINE__);
+ trx_t *trx = trx_sys->rw_trx_hash.find(undo->trx_id);
+ if (!trx) {
+ trx_resurrect(undo, rseg, start_time,
+ &rows_to_undo, &set, false);
+ } else {
+ ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE) ||
+ trx_state_eq(trx, TRX_STATE_PREPARED));
+ ut_ad(trx->start_time == start_time);
+ ut_ad(trx->is_recovered);
+ ut_ad(trx->rsegs.m_redo.rseg == rseg);
+ ut_ad(trx->rsegs.m_redo.rseg->trx_ref_count);
+
+ 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(
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
within the same transaction will get the same read view, which is created
diff --git a/storage/innobase/trx/trx0undo.cc b/storage/innobase/trx/trx0undo.cc
index 49c1ab15fe8..372bcaee620 100644
--- a/storage/innobase/trx/trx0undo.cc
+++ b/storage/innobase/trx/trx0undo.cc
@@ -1643,7 +1643,6 @@ trx_undo_free_prepared(
/* fall through */
case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns
- trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked
to TRX_STATE_PREPARED in trx_rollback_resurrected(). */
@@ -1673,7 +1672,6 @@ trx_undo_free_prepared(
/* fall through */
case TRX_UNDO_ACTIVE:
/* lock_trx_release_locks() assigns
- trx->is_recovered=false and
trx->state = TRX_STATE_COMMITTED_IN_MEMORY,
also for transactions that we faked
to TRX_STATE_PREPARED in trx_rollback_resurrected(). */