Commit 7f6658db authored by unknown's avatar unknown

set_var.cc, mysqld.cc, ha_innodb.cc, sql_class.h:

  Add a settable session variable innodb_support_xa; setting it to 0 can save up to 10 % of CPU time and 150 bytes of space in each undo log
trx0trx.h, trx0undo.c, trx0trx.c, trx0roll.c:
  Enable XA if innodb_support_xa is not set to 0; make prepare to do log fsync's according to innodb_flush_log_at_trx_commit


innobase/trx/trx0roll.c:
  Enable XA if innodb_support_xa is not set to 0; make prepare to do log fsync's according to innodb_flush_log_at_trx_commit
innobase/trx/trx0trx.c:
  Enable XA if innodb_support_xa is not set to 0; make prepare to do log fsync's according to innodb_flush_log_at_trx_commit
innobase/trx/trx0undo.c:
  Enable XA if innodb_support_xa is not set to 0; make prepare to do log fsync's according to innodb_flush_log_at_trx_commit
innobase/include/trx0trx.h:
  Enable XA if innodb_support_xa is not set to 0; make prepare to do log fsync's according to innodb_flush_log_at_trx_commit
sql/sql_class.h:
  Add a settable session variable innodb_support_xa; setting it to 0 can save up to 10 % of CPU time and 150 bytes of space in each undo log
sql/ha_innodb.cc:
  Add a settable session variable innodb_support_xa; setting it to 0 can save up to 10 % of CPU time and 150 bytes of space in each undo log
sql/mysqld.cc:
  Add a settable session variable innodb_support_xa; setting it to 0 can save up to 10 % of CPU time and 150 bytes of space in each undo log
sql/set_var.cc:
  Add a settable session variable innodb_support_xa; setting it to 0 can save up to 10 % of CPU time and 150 bytes of space in each undo log
parent d83bb4ed
......@@ -369,6 +369,11 @@ struct trx_struct{
XID xid; /* X/Open XA transaction
identification to identify a
transaction branch */
ibool support_xa; /* normally we do the XA two-phase
commit steps, but by setting this to
FALSE, one can save CPU time and about
150 bytes in the undo log size as then
we skip XA steps */
dulint no; /* transaction serialization number ==
max trx id when the transaction is
moved to COMMITTED_IN_MEMORY state */
......
......@@ -441,16 +441,8 @@ trx_rollback_or_clean_all_without_sess(
trx = UT_LIST_GET_NEXT(trx_list, trx);
} else if (trx->conc_state == TRX_PREPARED) {
/* Roll back all prepared transactions if
innobase_force_recovery > 0 in my.cnf */
if (srv_force_recovery > 0) {
trx->conc_state = TRX_ACTIVE;
break;
} else {
trx->sess = trx_dummy_sess;
trx = UT_LIST_GET_NEXT(trx_list, trx);
}
trx->sess = trx_dummy_sess;
trx = UT_LIST_GET_NEXT(trx_list, trx);
} else {
break;
}
......@@ -461,7 +453,7 @@ trx_rollback_or_clean_all_without_sess(
if (trx == NULL) {
ut_print_timestamp(stderr);
fprintf(stderr,
" InnoDB: Rollback of uncommitted transactions completed\n");
" InnoDB: Rollback of non-prepared transactions completed\n");
mem_heap_free(heap);
......
......@@ -93,6 +93,8 @@ trx_create(
trx->id = ut_dulint_zero;
trx->no = ut_dulint_max;
trx->support_xa = TRUE;
trx->check_foreigns = TRUE;
trx->check_unique_secondary = TRUE;
......@@ -453,9 +455,15 @@ trx_lists_init_at_db_start(void)
ut_dulint_get_high(trx->id),
ut_dulint_get_low(trx->id));
trx->conc_state = TRX_ACTIVE;
if (srv_force_recovery == 0) {
/* trx->conc_state = TRX_PREPARED;*/
trx->conc_state = TRX_PREPARED;
} else {
fprintf(stderr,
"InnoDB: Since innodb_force_recovery > 0, we will rollback it anyway.\n");
trx->conc_state = TRX_ACTIVE;
}
} else {
trx->conc_state =
TRX_COMMITTED_IN_MEMORY;
......@@ -511,15 +519,20 @@ trx_lists_init_at_db_start(void)
commit or abort decision from MySQL */
if (undo->state == TRX_UNDO_PREPARED) {
fprintf(stderr,
fprintf(stderr,
"InnoDB: Transaction %lu %lu was in the XA prepared state.\n",
ut_dulint_get_high(trx->id),
ut_dulint_get_low(trx->id));
ut_dulint_get_high(trx->id),
ut_dulint_get_low(trx->id));
trx->conc_state = TRX_ACTIVE;
if (srv_force_recovery == 0) {
trx->conc_state = TRX_PREPARED;
} else {
fprintf(stderr,
"InnoDB: Since innodb_force_recovery > 0, we will rollback it anyway.\n");
/* trx->conc_state =
TRX_PREPARED; */
trx->conc_state = TRX_ACTIVE;
}
} else {
trx->conc_state =
TRX_COMMITTED_IN_MEMORY;
......@@ -823,9 +836,6 @@ trx_commit_off_kernel(
trx->read_view = NULL;
}
/* fprintf(stderr, "Trx %lu commit finished\n",
ut_dulint_get_low(trx->id)); */
if (must_flush_log) {
mutex_exit(&kernel_mutex);
......@@ -869,14 +879,15 @@ trx_commit_off_kernel(
/* Do nothing */
} else if (srv_flush_log_at_trx_commit == 1) {
if (srv_unix_file_flush_method == SRV_UNIX_NOSYNC) {
/* Write the log but do not flush it to disk */
/* Write the log but do not flush it to disk */
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, FALSE);
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP,
FALSE);
} else {
/* Write the log to the log files AND flush
them to disk */
/* Write the log to the log files AND flush
them to disk */
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, TRUE);
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, TRUE);
}
} else if (srv_flush_log_at_trx_commit == 2) {
......@@ -1747,12 +1758,11 @@ Prepares a transaction. */
void
trx_prepare_off_kernel(
/*==================*/
/*===================*/
trx_t* trx) /* in: transaction */
{
page_t* update_hdr_page;
trx_rseg_t* rseg;
trx_undo_t* undo;
ibool must_flush_log = FALSE;
dulint lsn;
mtr_t mtr;
......@@ -1779,19 +1789,18 @@ trx_prepare_off_kernel(
mutex_enter(&(rseg->mutex));
if (trx->insert_undo != NULL) {
trx_undo_set_state_at_prepare(trx, trx->insert_undo,
&mtr);
}
undo = trx->update_undo;
if (undo) {
/* It is not necessary to obtain trx->undo_mutex here
because only a single OS thread is allowed to do the
transaction prepare for this transaction. */
trx_undo_set_state_at_prepare(trx, trx->insert_undo,
&mtr);
}
if (trx->update_undo) {
update_hdr_page = trx_undo_set_state_at_prepare(trx,
undo, &mtr);
trx->update_undo, &mtr);
}
mutex_exit(&(rseg->mutex));
......@@ -1815,17 +1824,48 @@ trx_prepare_off_kernel(
/*--------------------------------------*/
if (must_flush_log) {
/* Depending on the my.cnf options, we may now write the log
buffer to the log files, making the prepared state of the
transaction durable if the OS does not crash. We may also
flush the log files to disk, making the prepared state of the
transaction durable also at an OS crash or a power outage.
The idea in InnoDB's group prepare 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 prepares the whole
group. Note that this group prepare 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.
TODO: find out if MySQL holds some mutex when calling this.
That would spoil our group prepare algorithm. */
mutex_exit(&kernel_mutex);
/* Write the log to the log files AND flush them to disk */
/*-------------------------------------*/
if (srv_flush_log_at_trx_commit == 0) {
/* Do nothing */
} else if (srv_flush_log_at_trx_commit == 1) {
if (srv_unix_file_flush_method == SRV_UNIX_NOSYNC) {
/* Write the log but do not flush it to disk */
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, TRUE);
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP,
FALSE);
} else {
/* Write the log to the log files AND flush
them to disk */
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, TRUE);
}
} else if (srv_flush_log_at_trx_commit == 2) {
/* Write the log but do not flush it to disk */
log_write_up_to(lsn, LOG_WAIT_ONE_GROUP, FALSE);
} else {
ut_error;
}
/*-------------------------------------*/
mutex_enter(&kernel_mutex);
}
}
......
......@@ -596,7 +596,7 @@ trx_undo_read_xid(
}
/*******************************************************************
Adds the XA XID after an undo log old-style header. */
Adds space for the XA XID after an undo log old-style header. */
static
void
trx_undo_header_add_space_for_xid(
......@@ -1488,6 +1488,7 @@ trx_undo_create(
/*============*/
/* out: undo log object, NULL if did not
succeed: out of space */
trx_t* trx, /* in: transaction */
trx_rseg_t* rseg, /* in: rollback segment memory copy */
ulint type, /* in: type of the log: TRX_UNDO_INSERT or
TRX_UNDO_UPDATE */
......@@ -1530,7 +1531,10 @@ trx_undo_create(
offset = trx_undo_header_create(undo_page, trx_id, mtr);
trx_undo_header_add_space_for_xid(undo_page, undo_page + offset, mtr);
if (trx->support_xa) {
trx_undo_header_add_space_for_xid(undo_page,
undo_page + offset, mtr);
}
undo = trx_undo_mem_create(rseg, id, type, trx_id, xid,
page_no, offset);
......@@ -1547,6 +1551,7 @@ trx_undo_reuse_cached(
/*==================*/
/* out: the undo log memory object, NULL if
none cached */
trx_t* trx, /* in: transaction */
trx_rseg_t* rseg, /* in: rollback segment memory object */
ulint type, /* in: type of the log: TRX_UNDO_INSERT or
TRX_UNDO_UPDATE */
......@@ -1597,16 +1602,22 @@ trx_undo_reuse_cached(
if (type == TRX_UNDO_INSERT) {
offset = trx_undo_insert_header_reuse(undo_page, trx_id, mtr);
trx_undo_header_add_space_for_xid(undo_page, undo_page + offset,
mtr);
if (trx->support_xa) {
trx_undo_header_add_space_for_xid(undo_page,
undo_page + offset, mtr);
}
} else {
ut_a(mach_read_from_2(undo_page + TRX_UNDO_PAGE_HDR
+ TRX_UNDO_PAGE_TYPE)
== TRX_UNDO_UPDATE);
offset = trx_undo_header_create(undo_page, trx_id, mtr);
trx_undo_header_add_space_for_xid(undo_page, undo_page + offset,
mtr);
if (trx->support_xa) {
trx_undo_header_add_space_for_xid(undo_page,
undo_page + offset, mtr);
}
}
trx_undo_mem_init_for_reuse(undo, trx_id, xid, offset);
......@@ -1674,11 +1685,11 @@ trx_undo_assign_undo(
#endif /* UNIV_SYNC_DEBUG */
mutex_enter(&(rseg->mutex));
undo = trx_undo_reuse_cached(rseg, type, trx->id, &trx->xid, &mtr);
undo = trx_undo_reuse_cached(trx, rseg, type, trx->id, &trx->xid,
&mtr);
if (undo == NULL) {
undo = trx_undo_create(rseg, type, trx->id, &trx->xid, &mtr);
undo = trx_undo_create(trx, rseg, type, trx->id, &trx->xid,
&mtr);
if (undo == NULL) {
/* Did not succeed */
......
......@@ -690,6 +690,10 @@ check_trx_exists(
trx->mysql_query_str = &(thd->query);
trx->active_trans = 0;
/* Update the info whether we should skip XA steps that eat
CPU time */
trx->support_xa = (ibool)(thd->variables.innodb_support_xa);
thd->ha_data[innobase_hton.slot] = trx;
} else {
if (trx->magic_n != TRX_MAGIC_N) {
......@@ -1434,6 +1438,9 @@ innobase_commit(
trx = check_trx_exists(thd);
/* Update the info whether we should skip XA steps that eat CPU time */
trx->support_xa = (ibool)(thd->variables.innodb_support_xa);
/* Release a possible FIFO ticket and search latch. Since we will
reserve the kernel mutex, we have to release the search system latch
first to obey the latching order. */
......@@ -1620,6 +1627,9 @@ innobase_rollback(
trx = check_trx_exists(thd);
/* Update the info whether we should skip XA steps that eat CPU time */
trx->support_xa = (ibool)(thd->variables.innodb_support_xa);
/* Release a possible FIFO ticket and search latch. Since we will
reserve the kernel mutex, we have to release the search system latch
first to obey the latching order. */
......@@ -6308,6 +6318,11 @@ innobase_xa_prepare(
int error = 0;
trx_t* trx;
if (!thd->variables.innodb_support_xa) {
return(0);
}
trx = check_trx_exists(thd);
trx->xid=thd->transaction.xid;
......
......@@ -4211,6 +4211,7 @@ enum options_mysqld
OPT_INNODB_STATUS_FILE,
OPT_INNODB_MAX_DIRTY_PAGES_PCT,
OPT_INNODB_TABLE_LOCKS,
OPT_INNODB_SUPPORT_XA,
OPT_INNODB_OPEN_FILES,
OPT_INNODB_AUTOEXTEND_INCREMENT,
OPT_INNODB_SYNC_SPIN_LOOPS,
......@@ -4514,6 +4515,11 @@ Disable with --skip-innodb-doublewrite.", (gptr*) &innobase_use_doublewrite,
(gptr*) &global_system_variables.innodb_table_locks,
(gptr*) &global_system_variables.innodb_table_locks,
0, GET_BOOL, OPT_ARG, 1, 0, 0, 0, 0, 0},
{"innodb_support_xa", OPT_INNODB_SUPPORT_XA,
"Enable InnoDB support for the XA two-phase commit",
(gptr*) &global_system_variables.innodb_support_xa,
(gptr*) &global_system_variables.innodb_support_xa,
0, GET_BOOL, OPT_ARG, 1, 0, 0, 0, 0, 0},
#endif /* End HAVE_INNOBASE_DB */
{"isam", OPT_ISAM, "Enable ISAM (if this version of MySQL supports it). \
Disable with --skip-isam.",
......
......@@ -392,6 +392,8 @@ sys_var_long_ptr sys_innodb_max_purge_lag("innodb_max_purge_lag",
&srv_max_purge_lag);
sys_var_thd_bool sys_innodb_table_locks("innodb_table_locks",
&SV::innodb_table_locks);
sys_var_thd_bool sys_innodb_support_xa("innodb_support_xa",
&SV::innodb_support_xa);
sys_var_long_ptr sys_innodb_autoextend_increment("innodb_autoextend_increment",
&srv_auto_extend_increment);
sys_var_long_ptr sys_innodb_sync_spin_loops("innodb_sync_spin_loops",
......@@ -689,6 +691,7 @@ sys_var *sys_variables[]=
&sys_innodb_max_dirty_pages_pct,
&sys_innodb_max_purge_lag,
&sys_innodb_table_locks,
&sys_innodb_support_xa,
&sys_innodb_max_purge_lag,
&sys_innodb_autoextend_increment,
&sys_innodb_sync_spin_loops,
......@@ -810,6 +813,7 @@ struct show_var_st init_vars[]= {
{"innodb_open_files", (char*) &innobase_open_files, SHOW_LONG },
{sys_innodb_sync_spin_loops.name, (char*) &sys_innodb_sync_spin_loops, SHOW_SYS},
{sys_innodb_table_locks.name, (char*) &sys_innodb_table_locks, SHOW_SYS},
{sys_innodb_support_xa.name, (char*) &sys_innodb_support_xa, SHOW_SYS},
{sys_innodb_thread_concurrency.name, (char*) &sys_innodb_thread_concurrency, SHOW_SYS},
{sys_innodb_thread_sleep_delay.name, (char*) &sys_innodb_thread_sleep_delay, SHOW_SYS},
#endif
......
......@@ -578,6 +578,7 @@ struct system_variables
#endif /* HAVE_REPLICATION */
#ifdef HAVE_INNOBASE_DB
my_bool innodb_table_locks;
my_bool innodb_support_xa;
#endif /* HAVE_INNOBASE_DB */
#ifdef HAVE_NDBCLUSTER_DB
ulong ndb_autoincrement_prefetch_sz;
......
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