summaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorMonty <monty@mariadb.org>2020-05-02 13:19:53 +0300
committerMonty <monty@mariadb.org>2020-05-23 12:29:10 +0300
commit4102f1589c23309de968a5bf9511d3228a1b9319 (patch)
tree9450597fff4a510282b2dec366cb102f5aadfc57 /sql
parentd1d472646d578608791dcd33c13ca6b2472e31b2 (diff)
downloadmariadb-git-4102f1589c23309de968a5bf9511d3228a1b9319.tar.gz
Aria will now register it's transactions
MDEV-22531 Remove maria::implicit_commit() MDEV-22607 Assertion `ha_info->ht() != binlog_hton' failed in MYSQL_BIN_LOG::unlog_xa_prepare From the handler point of view, Aria now looks like a transactional engine. One effect of this is that we don't need to call maria::implicit_commit() anymore. This change also forces the server to call trans_commit_stmt() after doing any read or writes to system tables. This work will also make it easier to later allow users to have system tables in other engines than Aria. To handle the case that Aria doesn't support rollback, a new handlerton flag, HTON_NO_ROLLBACK, was added to engines that has transactions without rollback (for the moment only binlog and Aria). Other things - Moved freeing of MARIA_SHARE to a separate function as the MARIA_SHARE can be still part of a transaction even if the table has closed. - Changed Aria checkpoint to use the new MARIA_SHARE free function. This fixes a possible memory leak when using S3 tables - Changed testing of binlog_hton to instead test for HTON_NO_ROLLBACK - Removed checking of has_transaction_manager() in handler.cc as we can assume that as the transaction was started by the engine, it does support transactions. - Added new class 'start_new_trans' that can be used to start indepdendent sub transactions, for example while reading mysql.proc, using help or status tables etc. - open_system_tables...() and open_proc_table_for_Read() doesn't anymore take a Open_tables_backup list. This is now handled by 'start_new_trans'. - Split thd::has_transactions() to thd::has_transactions() and thd::has_transactions_and_rollback() - Added handlerton code to free cached transactions objects. Needed by InnoDB. squash! 2ed35999f2a2d84f1c786a21ade5db716b6f1bbc
Diffstat (limited to 'sql')
-rw-r--r--sql/event_db_repository.cc83
-rw-r--r--sql/event_scheduler.cc2
-rw-r--r--sql/ha_partition.cc6
-rw-r--r--sql/handler.cc27
-rw-r--r--sql/handler.h42
-rw-r--r--sql/log.cc35
-rw-r--r--sql/log_event_server.cc2
-rw-r--r--sql/share/errmsg-utf8.txt2
-rw-r--r--sql/sp.cc72
-rw-r--r--sql/sp.h2
-rw-r--r--sql/sql_base.cc59
-rw-r--r--sql/sql_base.h5
-rw-r--r--sql/sql_class.cc90
-rw-r--r--sql/sql_class.h37
-rw-r--r--sql/sql_delete.cc6
-rw-r--r--sql/sql_help.cc11
-rw-r--r--sql/sql_insert.cc12
-rw-r--r--sql/sql_load.cc4
-rw-r--r--sql/sql_parse.cc60
-rw-r--r--sql/sql_sequence.cc3
-rw-r--r--sql/sql_show.cc13
-rw-r--r--sql/sql_statistics.cc87
-rw-r--r--sql/sql_trigger.cc4
-rw-r--r--sql/sql_truncate.cc2
-rw-r--r--sql/sql_update.cc8
-rw-r--r--sql/transaction.cc13
-rw-r--r--sql/tztime.cc13
27 files changed, 420 insertions, 280 deletions
diff --git a/sql/event_db_repository.cc b/sql/event_db_repository.cc
index af43d92dea7..9e8e5f06043 100644
--- a/sql/event_db_repository.cc
+++ b/sql/event_db_repository.cc
@@ -30,6 +30,7 @@
#include "events.h"
#include "sql_show.h"
#include "lock.h" // MYSQL_LOCK_IGNORE_TIMEOUT
+#include "transaction.h"
/**
@addtogroup Event_Scheduler
@@ -533,23 +534,26 @@ Event_db_repository::fill_schema_events(THD *thd, TABLE_LIST *i_s_table,
const char *db)
{
TABLE *schema_table= i_s_table->table;
- Open_tables_backup open_tables_backup;
TABLE_LIST event_table;
int ret= 0;
-
DBUG_ENTER("Event_db_repository::fill_schema_events");
DBUG_PRINT("info",("db=%s", db? db:"(null)"));
+ start_new_trans new_trans(thd);
+
event_table.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_EVENT_NAME, 0, TL_READ);
- if (open_system_tables_for_read(thd, &event_table, &open_tables_backup))
+ if (open_system_tables_for_read(thd, &event_table))
+ {
+ new_trans.restore_old_transaction();
DBUG_RETURN(TRUE);
+ }
if (table_intact.check(event_table.table, &event_table_def))
{
- close_system_tables(thd, &open_tables_backup);
my_error(ER_EVENT_OPEN_TABLE_FAILED, MYF(0));
- DBUG_RETURN(TRUE);
+ ret= 1;
+ goto err;
}
/*
@@ -566,7 +570,9 @@ Event_db_repository::fill_schema_events(THD *thd, TABLE_LIST *i_s_table,
else
ret= table_scan_all_for_i_s(thd, schema_table, event_table.table);
- close_system_tables(thd, &open_tables_backup);
+err:
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
DBUG_PRINT("info", ("Return code=%d", ret));
DBUG_RETURN(ret);
@@ -612,7 +618,8 @@ Event_db_repository::open_event_table(THD *thd, enum thr_lock_type lock_type,
if (table_intact.check(*table, &event_table_def))
{
- close_thread_tables(thd);
+ thd->commit_whole_transaction_and_close_tables();
+ *table= 0; // Table is now closed
my_error(ER_EVENT_OPEN_TABLE_FAILED, MYF(0));
DBUG_RETURN(TRUE);
}
@@ -742,7 +749,8 @@ Event_db_repository::create_event(THD *thd, Event_parse_data *parse_data,
ret= 0;
end:
- close_thread_tables(thd);
+ if (table)
+ thd->commit_whole_transaction_and_close_tables();
thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
thd->variables.sql_mode= saved_mode;
@@ -784,7 +792,6 @@ Event_db_repository::update_event(THD *thd, Event_parse_data *parse_data,
*/
MDL_savepoint mdl_savepoint= thd->mdl_context.mdl_savepoint();
int ret= 1;
-
DBUG_ENTER("Event_db_repository::update_event");
/* None or both must be set */
@@ -857,7 +864,8 @@ Event_db_repository::update_event(THD *thd, Event_parse_data *parse_data,
ret= 0;
end:
- close_thread_tables(thd);
+ if (table)
+ thd->commit_whole_transaction_and_close_tables();
thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
thd->variables.sql_mode= saved_mode;
@@ -919,7 +927,8 @@ Event_db_repository::drop_event(THD *thd, const LEX_CSTRING *db,
ret= 0;
end:
- close_thread_tables(thd);
+ if (table)
+ thd->commit_whole_transaction_and_close_tables();
thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
DBUG_RETURN(MY_TEST(ret));
@@ -999,12 +1008,16 @@ Event_db_repository::drop_schema_events(THD *thd, const LEX_CSTRING *schema)
TABLE *table= NULL;
READ_RECORD read_record_info;
enum enum_events_table_field field= ET_FIELD_DB;
- MDL_savepoint mdl_savepoint= thd->mdl_context.mdl_savepoint();
DBUG_ENTER("Event_db_repository::drop_schema_events");
DBUG_PRINT("enter", ("field: %d schema: %s", field, schema->str));
+ start_new_trans new_trans(thd);
+
if (open_event_table(thd, TL_WRITE, &table))
+ {
+ new_trans.restore_old_transaction();
DBUG_VOID_RETURN;
+ }
/* only enabled events are in memory, so we go now and delete the rest */
if (init_read_record(&read_record_info, thd, table, NULL, NULL, 1, 0, FALSE))
@@ -1033,13 +1046,8 @@ Event_db_repository::drop_schema_events(THD *thd, const LEX_CSTRING *schema)
end_read_record(&read_record_info);
end:
- close_thread_tables(thd);
- /*
- Make sure to only release the MDL lock on mysql.event, not other
- metadata locks DROP DATABASE might have acquired.
- */
- thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
-
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
DBUG_VOID_RETURN;
}
@@ -1060,29 +1068,29 @@ Event_db_repository::load_named_event(THD *thd, const LEX_CSTRING *dbname,
Event_basic *etn)
{
bool ret;
- Open_tables_backup open_tables_backup;
TABLE_LIST event_table;
-
DBUG_ENTER("Event_db_repository::load_named_event");
DBUG_PRINT("enter",("thd: %p name: %*s", thd,
(int) name->length, name->str));
- event_table.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_EVENT_NAME, 0, TL_READ);
-
+ start_new_trans new_trans(thd);
/* Reset sql_mode during data dictionary operations. */
Sql_mode_instant_set sms(thd, 0);
+ event_table.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_EVENT_NAME, 0, TL_READ);
+
/*
We don't use open_event_table() here to make sure that SHOW
CREATE EVENT works properly in transactional context, and
does not release transactional metadata locks when the
event table is closed.
*/
- if (!(ret= open_system_tables_for_read(thd, &event_table, &open_tables_backup)))
+ if (!(ret= open_system_tables_for_read(thd, &event_table)))
{
if (table_intact.check(event_table.table, &event_table_def))
{
- close_system_tables(thd, &open_tables_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
my_error(ER_EVENT_OPEN_TABLE_FAILED, MYF(0));
DBUG_RETURN(TRUE);
}
@@ -1091,9 +1099,9 @@ Event_db_repository::load_named_event(THD *thd, const LEX_CSTRING *dbname,
my_error(ER_EVENT_DOES_NOT_EXIST, MYF(0), name->str);
else if ((ret= etn->load_from_row(thd, event_table.table)))
my_error(ER_CANNOT_LOAD_FROM_TABLE_V2, MYF(0), "mysql", "event");
-
- close_system_tables(thd, &open_tables_backup);
+ thd->commit_whole_transaction_and_close_tables();
}
+ new_trans.restore_old_transaction();
DBUG_RETURN(ret);
}
@@ -1117,22 +1125,20 @@ update_timing_fields_for_event(THD *thd,
TABLE *table= NULL;
Field **fields;
int ret= 1;
- enum_binlog_format save_binlog_format;
MYSQL_TIME time;
DBUG_ENTER("Event_db_repository::update_timing_fields_for_event");
- /*
- Turn off row binlogging of event timing updates. These are not used
- for RBR of events replicated to the slave.
- */
- save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
-
DBUG_ASSERT(thd->security_ctx->master_access & PRIV_IGNORE_READ_ONLY);
if (open_event_table(thd, TL_WRITE, &table))
- goto end;
+ DBUG_RETURN(1);
fields= table->field;
+ /*
+ Turn off row binlogging of event timing updates. These are not used
+ for RBR of events replicated to the slave.
+ */
+ table->file->row_logging= 0;
if (find_named_event(event_db_name, event_name, table))
goto end;
@@ -1153,12 +1159,9 @@ update_timing_fields_for_event(THD *thd,
}
ret= 0;
-
end:
- if (table)
- close_mysql_tables(thd);
-
- thd->restore_stmt_binlog_format(save_binlog_format);
+ if (thd->commit_whole_transaction_and_close_tables())
+ ret= 1;
DBUG_RETURN(MY_TEST(ret));
}
diff --git a/sql/event_scheduler.cc b/sql/event_scheduler.cc
index 4ecf19c389b..78802a5e109 100644
--- a/sql/event_scheduler.cc
+++ b/sql/event_scheduler.cc
@@ -511,6 +511,8 @@ Event_scheduler::run(THD *thd)
}
DBUG_PRINT("info", ("state=%s", scheduler_states_names[state].str));
free_root(thd->mem_root, MYF(0));
+ /* Ensure we don't have any open tables or table locks */
+ DBUG_ASSERT(thd->lock == 0);
}
LOCK_DATA();
diff --git a/sql/ha_partition.cc b/sql/ha_partition.cc
index 582c9bb110b..18d10710e45 100644
--- a/sql/ha_partition.cc
+++ b/sql/ha_partition.cc
@@ -11014,9 +11014,9 @@ int ha_partition::check_misplaced_rows(uint read_part_id, bool do_repair)
/*
If the engine supports transactions, the failure will be
- rollbacked.
+ rolled back
*/
- if (!m_file[correct_part_id]->has_transactions())
+ if (!m_file[correct_part_id]->has_transactions_and_rollback())
{
/* Log this error, so the DBA can notice it and fix it! */
sql_print_error("Table '%-192s' failed to move/insert a row"
@@ -11040,7 +11040,7 @@ int ha_partition::check_misplaced_rows(uint read_part_id, bool do_repair)
/* Delete row from wrong partition. */
if ((result= m_file[read_part_id]->ha_delete_row(m_rec0)))
{
- if (m_file[correct_part_id]->has_transactions())
+ if (m_file[correct_part_id]->has_transactions_and_rollback())
break;
/*
We have introduced a duplicate, since we failed to remove it
diff --git a/sql/handler.cc b/sql/handler.cc
index 39841cc28d7..41377b7f8b5 100644
--- a/sql/handler.cc
+++ b/sql/handler.cc
@@ -1547,15 +1547,6 @@ int ha_commit_trans(THD *thd, bool all)
wsrep_commit_empty(thd, all);
#endif /* WITH_WSREP */
-#if defined(WITH_ARIA_STORAGE_ENGINE)
- /* This is needed to ensure that repair commits properly */
- if ((error= ha_maria::implicit_commit(thd, TRUE)))
- {
- my_error(ER_ERROR_DURING_COMMIT, MYF(0), error);
- ha_rollback_trans(thd, all);
- DBUG_RETURN(1);
- }
-#endif
DBUG_RETURN(0);
}
@@ -1579,7 +1570,7 @@ int ha_commit_trans(THD *thd, bool all)
the plugin table to innodb and thus plugin_load will call
mysql_close_tables() which calls trans_commit_trans() with maria_hton = 0
*/
- if (rw_trans || (likely(maria_hton) && thd_get_ha_data(thd, maria_hton)))
+ if (rw_trans)
{
/*
Acquire a metadata lock which will ensure that COMMIT is blocked
@@ -1602,14 +1593,6 @@ int ha_commit_trans(THD *thd, bool all)
DEBUG_SYNC(thd, "ha_commit_trans_after_acquire_commit_lock");
}
-#if defined(WITH_ARIA_STORAGE_ENGINE)
- if ((error= ha_maria::implicit_commit(thd, TRUE)))
- {
- my_error(ER_ERROR_DURING_COMMIT, MYF(0), error);
- goto err;
- }
-#endif
-
if (rw_trans &&
opt_readonly &&
!(thd->security_ctx->master_access & PRIV_IGNORE_READ_ONLY) &&
@@ -2001,7 +1984,8 @@ int ha_rollback_trans(THD *thd, bool all)
int err;
handlerton *ht= ha_info->ht();
if ((err= ht->rollback(ht, thd, all)))
- { // cannot happen
+ {
+ // cannot happen
my_error(ER_ERROR_DURING_ROLLBACK, MYF(0), err);
error=1;
#ifdef WITH_WSREP
@@ -4514,7 +4498,6 @@ void handler::mark_trx_read_write_internal()
*/
if (ha_info->is_started())
{
- DBUG_ASSERT(has_transaction_manager());
/*
table_share can be NULL in ha_delete_table(). See implementation
of standalone function ha_delete_table() in sql_base.cc.
@@ -6180,7 +6163,7 @@ extern "C" check_result_t handler_index_cond_check(void* h_arg)
THD *thd= h->table->in_use;
check_result_t res;
- enum thd_kill_levels abort_at= h->has_transactions() ?
+ enum thd_kill_levels abort_at= h->has_rollback() ?
THD_ABORT_SOFTLY : THD_ABORT_ASAP;
if (thd_kill_level(thd) > abort_at)
return CHECK_ABORTED_BY_USER;
@@ -6947,7 +6930,7 @@ bool handler::prepare_for_row_logging()
row_logging_has_trans=
((sql_command_flags[table->in_use->lex->sql_command] &
(CF_SCHEMA_CHANGE | CF_ADMIN_COMMAND)) ||
- table->file->has_transactions());
+ table->file->has_transactions_and_rollback());
}
else
{
diff --git a/sql/handler.h b/sql/handler.h
index e4903172c33..6a283b2dac3 100644
--- a/sql/handler.h
+++ b/sql/handler.h
@@ -1765,6 +1765,14 @@ handlerton *ha_default_tmp_handlerton(THD *thd);
*/
#define HTON_TABLE_MAY_NOT_EXIST_ON_SLAVE (1 << 15)
+/*
+ True if handler cannot rollback transactions. If not true, the transaction
+ will be put in the transactional binlog cache.
+ For some engines, like Aria, the rollback can happen in case of crash, but
+ not trough a handler rollback call.
+*/
+#define HTON_NO_ROLLBACK (1 << 16)
+
class Ha_trx_info;
struct THD_TRANS
@@ -3541,9 +3549,12 @@ public:
virtual const key_map *keys_to_use_for_scanning() { return &key_map_empty; }
/*
- True if changes to the table is persistent (no rollback)
- This is mainly used to decide how to log changes to the table in
- the binary log.
+ True if changes to the table is persistent (if there are no rollback)
+ This is used to decide:
+ - If the table is stored in the transaction or non transactional binary
+ log
+ - How things are tracked in trx and in add_changed_table().
+ - If we can combine several statements under one commit in the binary log.
*/
bool has_transactions()
{
@@ -3551,11 +3562,31 @@ public:
== 0);
}
/*
- True if the underlaying table doesn't support transactions
+ True if table has both transactions and rollback. This is used to decide
+ if we should write the changes to the binary log. If this is true,
+ we don't have to write failed statements to the log as they can be
+ rolled back.
+ */
+ bool has_transactions_and_rollback()
+ {
+ return has_transactions() && has_rollback();
+ }
+ /*
+ True if the underlaying table support transactions and rollback
*/
bool has_transaction_manager()
{
- return ((ha_table_flags() & HA_NO_TRANSACTIONS) == 0);
+ return ((ha_table_flags() & HA_NO_TRANSACTIONS) == 0 && has_rollback());
+ }
+
+ /*
+ True if table has rollback. Used to check if an update on the table
+ can be killed fast.
+ */
+
+ bool has_rollback()
+ {
+ return ((ht->flags & HTON_NO_ROLLBACK) == 0);
}
/**
@@ -4975,7 +5006,6 @@ public:
inline int ha_update_tmp_row(const uchar * old_data, uchar * new_data);
virtual void set_lock_type(enum thr_lock_type lock);
-
friend check_result_t handler_index_cond_check(void* h_arg);
friend check_result_t handler_rowid_filter_check(void *h_arg);
diff --git a/sql/log.cc b/sql/log.cc
index 85ea8f9d663..b74adf59158 100644
--- a/sql/log.cc
+++ b/sql/log.cc
@@ -1701,7 +1701,7 @@ int binlog_init(void *p)
// recover needs to be set to make xa{commit,rollback}_handlerton effective
binlog_hton->recover= binlog_xa_recover_dummy;
}
- binlog_hton->flags= HTON_NOT_USER_SELECTABLE | HTON_HIDDEN;
+ binlog_hton->flags= HTON_NOT_USER_SELECTABLE | HTON_HIDDEN | HTON_NO_ROLLBACK;
return 0;
}
@@ -5611,7 +5611,8 @@ trans_has_updated_trans_table(const THD* thd)
@param thd The client thread that executed the current statement.
@return
- @c true if a transactional table was updated, @c false otherwise.
+ @c true if a transactional table with rollback was updated,
+ @c false otherwise.
*/
bool
stmt_has_updated_trans_table(const THD *thd)
@@ -5621,7 +5622,8 @@ stmt_has_updated_trans_table(const THD *thd)
for (ha_info= thd->transaction->stmt.ha_list; ha_info;
ha_info= ha_info->next())
{
- if (ha_info->is_trx_read_write() && ha_info->ht() != binlog_hton)
+ if (ha_info->is_trx_read_write() &&
+ !(ha_info->ht()->flags & HTON_NO_ROLLBACK))
return (TRUE);
}
return (FALSE);
@@ -10125,14 +10127,31 @@ int TC_LOG_BINLOG::unlog_xa_prepare(THD *thd, bool all)
uint rw_count= ha_count_rw_all(thd, &ha_info);
bool rc= false;
- if (rw_count > 0)
- {
- /* an empty XA-prepare event group is logged */
#ifndef DBUG_OFF
- for (ha_info= thd->transaction->all.ha_list; rw_count > 1 && ha_info;
+ if (rw_count > 1)
+ {
+ /*
+ There must be no binlog_hton used in a transaction consisting of more
+ than 1 engine, *when* (at this point) this transaction has not been
+ binlogged. The one exception is if there is an engine without a
+ prepare method, as in this case the engine doesn't support XA and
+ we have to ignore this check.
+ */
+ bool binlog= false, exist_hton_without_prepare= false;
+ for (ha_info= thd->transaction->all.ha_list; ha_info;
ha_info= ha_info->next())
- DBUG_ASSERT(ha_info->ht() != binlog_hton);
+ {
+ if (ha_info->ht() == binlog_hton)
+ binlog= true;
+ if (!ha_info->ht()->prepare)
+ exist_hton_without_prepare= true;
+ }
+ DBUG_ASSERT(!binlog || exist_hton_without_prepare);
+ }
#endif
+ if (rw_count > 0)
+ {
+ /* an empty XA-prepare event group is logged */
rc= write_empty_xa_prepare(thd, cache_mngr); // normally gains need_unlog
trans_register_ha(thd, true, binlog_hton, 0); // do it for future commmit
}
diff --git a/sql/log_event_server.cc b/sql/log_event_server.cc
index 124fb10d4ea..5910ece12f0 100644
--- a/sql/log_event_server.cc
+++ b/sql/log_event_server.cc
@@ -5569,7 +5569,7 @@ int Rows_log_event::do_apply_event(rpl_group_info *rgi)
if (table)
{
master_had_triggers= table->master_had_triggers;
- bool transactional_table= table->file->has_transactions();
+ bool transactional_table= table->file->has_transactions_and_rollback();
/*
table == NULL means that this table should not be replicated
(this was set up by Table_map_log_event::do_apply_event()
diff --git a/sql/share/errmsg-utf8.txt b/sql/share/errmsg-utf8.txt
index 02ba889aaa4..55dc2eeea22 100644
--- a/sql/share/errmsg-utf8.txt
+++ b/sql/share/errmsg-utf8.txt
@@ -7963,3 +7963,5 @@ ER_KEY_CANT_HAVE_WITHOUT_OVERLAPS
eng "Key %`s cannot have WITHOUT OVERLAPS"
ER_NOT_ALLOWED_IN_THIS_CONTEXT
eng "'%-.128s' is not allowed in this context"
+ER_DATA_WAS_COMMITED_UNDER_ROLLBACK
+ eng "Engine %s does not support rollback. Changes where commited during rollback call"
diff --git a/sql/sp.cc b/sql/sp.cc
index 51bbeeef368..971aa4a143f 100644
--- a/sql/sp.cc
+++ b/sql/sp.cc
@@ -30,6 +30,7 @@
#include "sql_table.h" // write_bin_log
#include "sp_head.h"
#include "sp_cache.h"
+#include "transaction.h"
#include "lock.h" // lock_object_name
#include <my_user.h>
@@ -470,27 +471,31 @@ static Proc_table_intact proc_table_intact;
currently open tables will be saved, and from which will be
restored when we will end work with mysql.proc.
+ NOTES
+ On must have a start_new_trans object active when calling this function
+
@retval
0 Error
@retval
\# Pointer to TABLE object of mysql.proc
*/
-TABLE *open_proc_table_for_read(THD *thd, Open_tables_backup *backup)
+TABLE *open_proc_table_for_read(THD *thd)
{
TABLE_LIST table;
-
DBUG_ENTER("open_proc_table_for_read");
+ DBUG_ASSERT(thd->internal_transaction());
+
table.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_PROC_NAME, NULL, TL_READ);
- if (open_system_tables_for_read(thd, &table, backup))
+ if (open_system_tables_for_read(thd, &table))
DBUG_RETURN(NULL);
if (!proc_table_intact.check(table.table, &proc_table_def))
DBUG_RETURN(table.table);
- close_system_tables(thd, backup);
+ thd->commit_whole_transaction_and_close_tables();
DBUG_RETURN(NULL);
}
@@ -504,6 +509,10 @@ TABLE *open_proc_table_for_read(THD *thd, Open_tables_backup *backup)
@note
Table opened with this call should closed using close_thread_tables().
+ We don't need to use the start_new_transaction object when calling this
+ as there can't be any active transactions when we create or alter
+ stored procedures
+
@retval
0 Error
@retval
@@ -517,7 +526,10 @@ static TABLE *open_proc_table_for_update(THD *thd)
MDL_savepoint mdl_savepoint= thd->mdl_context.mdl_savepoint();
DBUG_ENTER("open_proc_table_for_update");
- table_list.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_PROC_NAME, NULL, TL_WRITE);
+ DBUG_ASSERT(!thd->internal_transaction());
+
+ table_list.init_one_table(&MYSQL_SCHEMA_NAME, &MYSQL_PROC_NAME, NULL,
+ TL_WRITE);
if (!(table= open_system_table_for_update(thd, &table_list)))
DBUG_RETURN(NULL);
@@ -525,7 +537,7 @@ static TABLE *open_proc_table_for_update(THD *thd)
if (!proc_table_intact.check(table, &proc_table_def))
DBUG_RETURN(table);
- close_thread_tables(thd);
+ thd->commit_whole_transaction_and_close_tables();
thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
DBUG_RETURN(NULL);
@@ -683,23 +695,26 @@ Sp_handler::db_find_routine(THD *thd,
longlong modified;
Sp_chistics chistics;
bool saved_time_zone_used= thd->time_zone_used;
+ bool trans_commited= 0;
sql_mode_t sql_mode;
- Open_tables_backup open_tables_state_backup;
Stored_program_creation_ctx *creation_ctx;
AUTHID definer;
-
DBUG_ENTER("db_find_routine");
DBUG_PRINT("enter", ("type: %s name: %.*s",
type_str(),
(int) name->m_name.length, name->m_name.str));
*sphp= 0; // In case of errors
- if (!(table= open_proc_table_for_read(thd, &open_tables_state_backup)))
- DBUG_RETURN(SP_OPEN_TABLE_FAILED);
- /* Reset sql_mode during data dictionary operations. */
+ start_new_trans new_trans(thd);
Sql_mode_instant_set sms(thd, 0);
+ if (!(table= open_proc_table_for_read(thd)))
+ {
+ ret= SP_OPEN_TABLE_FAILED;
+ goto done;
+ }
+
if ((ret= db_find_routine_aux(thd, name, table)) != SP_OK)
goto done;
@@ -741,8 +756,9 @@ Sp_handler::db_find_routine(THD *thd,
creation_ctx= Stored_routine_creation_ctx::load_from_db(thd, name, table);
- close_system_tables(thd, &open_tables_state_backup);
- table= 0;
+ trans_commited= 1;
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
ret= db_load_routine(thd, name, sphp,
sql_mode, params, returns, body, chistics, definer,
@@ -753,8 +769,12 @@ Sp_handler::db_find_routine(THD *thd,
does not affect replication.
*/
thd->time_zone_used= saved_time_zone_used;
- if (table)
- close_system_tables(thd, &open_tables_state_backup);
+ if (!trans_commited)
+ {
+ if (table)
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
+ }
DBUG_RETURN(ret);
}
@@ -1727,7 +1747,6 @@ bool lock_db_routines(THD *thd, const char *db)
{
TABLE *table;
uint key_len;
- Open_tables_backup open_tables_state_backup;
MDL_request_list mdl_requests;
Lock_db_routines_error_handler err_handler;
uchar keybuf[MAX_KEY_LENGTH];
@@ -1735,13 +1754,15 @@ bool lock_db_routines(THD *thd, const char *db)
DBUG_SLOW_ASSERT(ok_for_lower_case_names(db));
+ start_new_trans new_trans(thd);
+
/*
mysql.proc will be re-opened during deletion, so we can ignore
errors when opening the table here. The error handler is
used to avoid getting the same warning twice.
*/
thd->push_internal_handler(&err_handler);
- table= open_proc_table_for_read(thd, &open_tables_state_backup);
+ table= open_proc_table_for_read(thd);
thd->pop_internal_handler();
if (!table)
{
@@ -1750,6 +1771,7 @@ bool lock_db_routines(THD *thd, const char *db)
or is outdated. We therefore only abort mysql_rm_db() if we
have errors not handled by the error handler.
*/
+ new_trans.restore_old_transaction();
DBUG_RETURN(thd->is_error() || thd->killed);
}
@@ -1760,11 +1782,10 @@ bool lock_db_routines(THD *thd, const char *db)
if (nxtres)
{
table->file->print_error(nxtres, MYF(0));
- close_system_tables(thd, &open_tables_state_backup);
- DBUG_RETURN(true);
+ goto error;
}
- if (! table->file->ha_index_read_map(table->record[0], keybuf, (key_part_map)1,
+ if (!table->file->ha_index_read_map(table->record[0], keybuf, (key_part_map)1,
HA_READ_KEY_EXACT))
{
do
@@ -1789,10 +1810,10 @@ bool lock_db_routines(THD *thd, const char *db)
if (nxtres != 0 && nxtres != HA_ERR_END_OF_FILE)
{
table->file->print_error(nxtres, MYF(0));
- close_system_tables(thd, &open_tables_state_backup);
- DBUG_RETURN(true);
+ goto error;
}
- close_system_tables(thd, &open_tables_state_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
/* We should already hold a global IX lock and a schema X lock. */
DBUG_ASSERT(thd->mdl_context.is_lock_owner(MDL_key::BACKUP, "", "",
@@ -1801,6 +1822,10 @@ bool lock_db_routines(THD *thd, const char *db)
MDL_EXCLUSIVE));
DBUG_RETURN(thd->mdl_context.acquire_locks(&mdl_requests,
thd->variables.lock_wait_timeout));
+error:
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
+ DBUG_RETURN(true);
}
@@ -1879,6 +1904,7 @@ sp_drop_db_routines(THD *thd, const char *db)
table->file->ha_index_end();
err_idx_init:
+ trans_commit_stmt(thd);
close_thread_tables(thd);
/*
Make sure to only release the MDL lock on mysql.proc, not other
diff --git a/sql/sp.h b/sql/sp.h
index 72485632261..e92525e1930 100644
--- a/sql/sp.h
+++ b/sql/sp.h
@@ -655,7 +655,7 @@ extern "C" uchar* sp_sroutine_key(const uchar *ptr, size_t *plen,
Routines which allow open/lock and close mysql.proc table even when
we already have some tables open and locked.
*/
-TABLE *open_proc_table_for_read(THD *thd, Open_tables_backup *backup);
+TABLE *open_proc_table_for_read(THD *thd);
bool load_charset(MEM_ROOT *mem_root,
Field *field,
diff --git a/sql/sql_base.cc b/sql/sql_base.cc
index 6078b9a2e5d..c9865394e93 100644
--- a/sql/sql_base.cc
+++ b/sql/sql_base.cc
@@ -4259,7 +4259,7 @@ restart:
list, we still need to call open_and_process_routine() to take
MDL locks on the routines.
*/
- if (thd->locked_tables_mode <= LTM_LOCK_TABLES)
+ if (thd->locked_tables_mode <= LTM_LOCK_TABLES && *sroutine_to_open)
{
/*
Process elements of the prelocking set which are present there
@@ -8881,17 +8881,16 @@ bool is_equal(const LEX_CSTRING *a, const LEX_CSTRING *b)
open_system_tables_for_read()
thd Thread context.
table_list List of tables to open.
- backup Pointer to Open_tables_state instance where
- information about currently open tables will be
- saved, and from which will be restored when we will
- end work with system tables.
NOTES
+ Caller should have used start_new_trans object to start a new
+ transcation when reading system tables.
+
Thanks to restrictions which we put on opening and locking of
system tables for writing, we can open and lock them for reading
- even when we already have some other tables open and locked. One
- must call close_system_tables() to close systems tables opened
- with this call.
+ even when we already have some other tables open and locked.
+ One should call thd->commit_whole_transaction_and_close_tables()
+ to close systems tables opened with this call.
NOTES
In some situations we use this function to open system tables for
@@ -8905,22 +8904,20 @@ bool is_equal(const LEX_CSTRING *a, const LEX_CSTRING *b)
*/
bool
-open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
- Open_tables_backup *backup)
+open_system_tables_for_read(THD *thd, TABLE_LIST *table_list)
{
Query_tables_list query_tables_list_backup;
LEX *lex= thd->lex;
DBUG_ENTER("open_system_tables_for_read");
+ DBUG_ASSERT(thd->internal_transaction());
/*
Besides using new Open_tables_state for opening system tables,
we also have to backup and reset/and then restore part of LEX
which is accessed by open_tables() in order to determine if
prelocking is needed and what tables should be added for it.
- close_system_tables() doesn't require such treatment.
*/
lex->reset_n_backup_query_tables_list(&query_tables_list_backup);
- thd->reset_n_backup_open_tables_state(backup);
thd->lex->sql_command= SQLCOM_SELECT;
/*
@@ -8935,7 +8932,6 @@ open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
MYSQL_LOCK_IGNORE_TIMEOUT : 0))))
{
lex->restore_backup_query_tables_list(&query_tables_list_backup);
- thd->restore_backup_open_tables_state(backup);
DBUG_RETURN(TRUE);
}
@@ -8950,33 +8946,6 @@ open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
DBUG_RETURN(FALSE);
}
-
-/*
- Close system tables, opened with open_system_tables_for_read().
-
- SYNOPSIS
- close_system_tables()
- thd Thread context
- backup Pointer to Open_tables_backup instance which holds
- information about tables which were open before we
- decided to access system tables.
-*/
-
-void
-close_system_tables(THD *thd, Open_tables_backup *backup)
-{
- /*
- Inform the transaction handler that we are closing the
- system tables and we don't need the read view anymore.
- */
- for (TABLE *table= thd->open_tables ; table ; table= table->next)
- table->file->extra(HA_EXTRA_PREPARE_FOR_FORCED_CLOSE);
-
- close_thread_tables(thd);
- thd->restore_backup_open_tables_state(backup);
-}
-
-
/**
A helper function to close a mysql.* table opened
in an auxiliary THD during bootstrap or in the main
@@ -9085,9 +9054,17 @@ open_log_table(THD *thd, TABLE_LIST *one_table, Open_tables_backup *backup)
@param thd The current thread
@param backup [in] the context to restore.
*/
+
void close_log_table(THD *thd, Open_tables_backup *backup)
{
- close_system_tables(thd, backup);
+ /*
+ Inform the transaction handler that we are closing the
+ system tables and we don't need the read view anymore.
+ */
+ for (TABLE *table= thd->open_tables ; table ; table= table->next)
+ table->file->extra(HA_EXTRA_PREPARE_FOR_FORCED_CLOSE);
+ close_thread_tables(thd);
+ thd->restore_backup_open_tables_state(backup);
}
diff --git a/sql/sql_base.h b/sql/sql_base.h
index 0fdb2599ce4..ccfacaf9086 100644
--- a/sql/sql_base.h
+++ b/sql/sql_base.h
@@ -294,9 +294,8 @@ bool is_equal(const LEX_CSTRING *a, const LEX_CSTRING *b);
class Open_tables_backup;
/* Functions to work with system tables. */
-bool open_system_tables_for_read(THD *thd, TABLE_LIST *table_list,
- Open_tables_backup *backup);
-void close_system_tables(THD *thd, Open_tables_backup *backup);
+bool open_system_tables_for_read(THD *thd, TABLE_LIST *table_list);
+void close_system_tables(THD *thd);
void close_mysql_tables(THD *thd);
TABLE *open_system_table_for_update(THD *thd, TABLE_LIST *one_table);
TABLE *open_log_table(THD *thd, TABLE_LIST *one_table, Open_tables_backup *backup);
diff --git a/sql/sql_class.cc b/sql/sql_class.cc
index dda8e00f6bf..000a5bedbf0 100644
--- a/sql/sql_class.cc
+++ b/sql/sql_class.cc
@@ -72,6 +72,7 @@
#include "wsrep_trans_observer.h"
#endif /* WITH_WSREP */
#include "opt_trace.h"
+#include <mysql/psi/mysql_transaction.h>
#ifdef HAVE_SYS_SYSCALL_H
#include <sys/syscall.h>
@@ -2589,7 +2590,8 @@ void THD::add_changed_table(TABLE *table)
{
DBUG_ENTER("THD::add_changed_table(table)");
- DBUG_ASSERT(in_multi_stmt_transaction_mode() && table->file->has_transactions());
+ DBUG_ASSERT(in_multi_stmt_transaction_mode() &&
+ table->file->has_transactions());
add_changed_table(table->s->table_cache_key.str,
(long) table->s->table_cache_key.length);
DBUG_VOID_RETURN;
@@ -5743,6 +5745,92 @@ void THD::mark_transaction_to_rollback(bool all)
/**
+ Commit the whole transaction (both statment and all)
+
+ This is used mainly to commit an independent transaction,
+ like reading system tables.
+
+ @return 0 0k
+ @return <>0 error code. my_error() has been called()
+*/
+
+int THD::commit_whole_transaction_and_close_tables()
+{
+ int error, error2;
+ DBUG_ENTER("THD::commit_whole_transaction_and_close_tables");
+
+ /*
+ This can only happened if we failed to open any table in the
+ new transaction
+ */
+ DBUG_ASSERT(open_tables);
+
+ if (!open_tables) // Safety for production usage
+ DBUG_RETURN(0);
+
+ /*
+ Ensure table was locked (opened with open_and_lock_tables()). If not
+ the THD can't be part of any transactions and doesn't have to call
+ this function.
+ */
+ DBUG_ASSERT(lock);
+
+ error= ha_commit_trans(this, FALSE);
+ /* This will call external_lock to unlock all tables */
+ if ((error2= mysql_unlock_tables(this, lock)))
+ {
+ my_error(ER_ERROR_DURING_COMMIT, MYF(0), error2);
+ error= error2;
+ }
+ lock= 0;
+ if ((error2= ha_commit_trans(this, TRUE)))
+ error= error2;
+ close_thread_tables(this);
+ DBUG_RETURN(error);
+}
+
+/**
+ Start a new independent transaction
+*/
+
+start_new_trans::start_new_trans(THD *thd)
+{
+ org_thd= thd;
+ mdl_savepoint= thd->mdl_context.mdl_savepoint();
+ memcpy(old_ha_data, thd->ha_data, sizeof(old_ha_data));
+ thd->reset_n_backup_open_tables_state(&open_tables_state_backup);
+ bzero(thd->ha_data, sizeof(thd->ha_data));
+ old_transaction= thd->transaction;
+ thd->transaction= &new_transaction;
+ new_transaction.on= 1;
+ in_sub_stmt= thd->in_sub_stmt;
+ thd->in_sub_stmt= 0;
+ server_status= thd->server_status;
+ m_transaction_psi= thd->m_transaction_psi;
+ thd->m_transaction_psi= 0;
+ thd->server_status&= ~(SERVER_STATUS_IN_TRANS |
+ SERVER_STATUS_IN_TRANS_READONLY);
+ thd->server_status|= SERVER_STATUS_AUTOCOMMIT;
+}
+
+
+void start_new_trans::restore_old_transaction()
+{
+ org_thd->transaction= old_transaction;
+ org_thd->restore_backup_open_tables_state(&open_tables_state_backup);
+ ha_close_connection(org_thd);
+ memcpy(org_thd->ha_data, old_ha_data, sizeof(old_ha_data));
+ org_thd->mdl_context.rollback_to_savepoint(mdl_savepoint);
+ org_thd->in_sub_stmt= in_sub_stmt;
+ org_thd->server_status= server_status;
+ if (org_thd->m_transaction_psi)
+ MYSQL_COMMIT_TRANSACTION(org_thd->m_transaction_psi);
+ org_thd->m_transaction_psi= m_transaction_psi;
+ org_thd= 0;
+}
+
+
+/**
Decide on logging format to use for the statement and issue errors
or warnings as needed. The decision depends on the following
parameters:
diff --git a/sql/sql_class.h b/sql/sql_class.h
index d4a95fa3fd8..8669e128bd5 100644
--- a/sql/sql_class.h
+++ b/sql/sql_class.h
@@ -3741,6 +3741,8 @@ public:
{
return server_status & SERVER_STATUS_IN_TRANS;
}
+ /* Commit both statement and full transaction */
+ int commit_whole_transaction_and_close_tables();
void give_protection_error();
inline bool has_read_only_protection()
{
@@ -4765,6 +4767,7 @@ public:
}
void mark_transaction_to_rollback(bool all);
+ bool internal_transaction() { return transaction != &default_transaction; }
private:
/** The current internal error handler for this thread, or NULL. */
@@ -5149,6 +5152,40 @@ public:
};
+
+/*
+ Start a new independent transaction for the THD.
+ The old one is stored in this object and restored when calling
+ restore_old_transaction() or when the object is freed
+*/
+
+class start_new_trans
+{
+ /* container for handler's private per-connection data */
+ Ha_data old_ha_data[MAX_HA];
+ struct THD::st_transactions *old_transaction, new_transaction;
+ Open_tables_backup open_tables_state_backup;
+ MDL_savepoint mdl_savepoint;
+ PSI_transaction_locker *m_transaction_psi;
+ THD *org_thd;
+ uint in_sub_stmt;
+ uint server_status;
+
+public:
+ start_new_trans(THD *thd);
+ ~start_new_trans()
+ {
+ destroy();
+ }
+ void destroy()
+ {
+ if (org_thd) // Safety
+ restore_old_transaction();
+ new_transaction.free();
+ }
+ void restore_old_transaction();
+};
+
/** A short cut for thd->get_stmt_da()->set_ok_status(). */
inline void
diff --git a/sql/sql_delete.cc b/sql/sql_delete.cc
index 071fd2ab8ac..5942efcd601 100644
--- a/sql/sql_delete.cc
+++ b/sql/sql_delete.cc
@@ -875,7 +875,7 @@ cleanup:
deltempfile=NULL;
delete select;
select= NULL;
- transactional_table= table->file->has_transactions();
+ transactional_table= table->file->has_transactions_and_rollback();
if (!transactional_table && deleted > 0)
thd->transaction->stmt.modified_non_trans_table=
@@ -1388,7 +1388,7 @@ void multi_delete::abort_result_set()
*/
if (do_delete && normal_tables &&
(table_being_deleted != delete_tables ||
- !table_being_deleted->table->file->has_transactions()))
+ !table_being_deleted->table->file->has_transactions_and_rollback()))
{
/*
We have to execute the recorded do_deletes() and write info into the
@@ -1537,7 +1537,7 @@ int multi_delete::do_table_deletes(TABLE *table, SORT_INFO *sort_info,
table->file->print_error(local_error, MYF(0));
}
}
- if (last_deleted != deleted && !table->file->has_transactions())
+ if (last_deleted != deleted && !table->file->has_transactions_and_rollback())
thd->transaction->stmt.modified_non_trans_table= TRUE;
end_read_record(&info);
diff --git a/sql/sql_help.cc b/sql/sql_help.cc
index c9307b578fc..3ccab553bfe 100644
--- a/sql/sql_help.cc
+++ b/sql/sql_help.cc
@@ -709,8 +709,9 @@ static bool mysqld_help_internal(THD *thd, const char *mask)
Reset and backup the current open tables state to
make it possible.
*/
- Open_tables_backup open_tables_state_backup;
- if (open_system_tables_for_read(thd, tables, &open_tables_state_backup))
+ start_new_trans new_trans(thd);
+
+ if (open_system_tables_for_read(thd, tables))
goto error2;
/*
@@ -843,11 +844,13 @@ static bool mysqld_help_internal(THD *thd, const char *mask)
}
my_eof(thd);
- close_system_tables(thd, &open_tables_state_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
DBUG_RETURN(FALSE);
error:
- close_system_tables(thd, &open_tables_state_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
error2:
DBUG_RETURN(TRUE);
diff --git a/sql/sql_insert.cc b/sql/sql_insert.cc
index 2b1f10a7cd2..a934427f02a 100644
--- a/sql/sql_insert.cc
+++ b/sql/sql_insert.cc
@@ -1144,7 +1144,7 @@ values_loop_end:
table->file->ha_rnd_end();
}
- transactional_table= table->file->has_transactions();
+ transactional_table= table->file->has_transactions_and_rollback();
if (likely(changed= (info.copied || info.deleted || info.updated)))
{
@@ -2055,7 +2055,7 @@ int write_record(THD *thd, TABLE *table, COPY_INFO *info, select_result *sink)
info->deleted++;
else
info->updated++;
- if (!table->file->has_transactions())
+ if (!table->file->has_transactions_and_rollback())
thd->transaction->stmt.modified_non_trans_table= TRUE;
if (table->triggers &&
table->triggers->process_triggers(thd, TRG_EVENT_DELETE,
@@ -2121,7 +2121,7 @@ ok:
after_trg_or_ignored_err:
if (key)
my_safe_afree(key,table->s->max_unique_length);
- if (!table->file->has_transactions())
+ if (!table->file->has_transactions_and_rollback())
thd->transaction->stmt.modified_non_trans_table= TRUE;
DBUG_RETURN(trg_error);
@@ -4104,13 +4104,13 @@ void select_insert::store_values(List<Item> &values)
bool select_insert::prepare_eof()
{
int error;
- bool const trans_table= table->file->has_transactions();
+ bool const trans_table= table->file->has_transactions_and_rollback();
bool changed;
bool binary_logged= 0;
killed_state killed_status= thd->killed;
DBUG_ENTER("select_insert::prepare_eof");
- DBUG_PRINT("enter", ("trans_table=%d, table_type='%s'",
+ DBUG_PRINT("enter", ("trans_table: %d, table_type: '%s'",
trans_table, table->file->table_type()));
#ifdef WITH_WSREP
@@ -4273,7 +4273,7 @@ void select_insert::abort_result_set()
zero, so no check for that is made.
*/
changed= (info.copied || info.deleted || info.updated);
- transactional_table= table->file->has_transactions();
+ transactional_table= table->file->has_transactions_and_rollback();
if (thd->transaction->stmt.modified_non_trans_table ||
thd->log_current_statement)
{
diff --git a/sql/sql_load.cc b/sql/sql_load.cc
index 8cff557962c..c3e301cd39d 100644
--- a/sql/sql_load.cc
+++ b/sql/sql_load.cc
@@ -436,7 +436,7 @@ int mysql_load(THD *thd, const sql_exchange *ex, TABLE_LIST *table_list,
}
table= table_list->table;
- transactional_table= table->file->has_transactions();
+ transactional_table= table->file->has_transactions_and_rollback();
#ifndef EMBEDDED_LIBRARY
is_concurrent= (table_list->lock_type == TL_WRITE_CONCURRENT_INSERT);
#endif
@@ -1212,7 +1212,7 @@ read_xml_field(THD *thd, COPY_INFO &info, TABLE_LIST *table_list,
bool no_trans_update_stmt;
DBUG_ENTER("read_xml_field");
- no_trans_update_stmt= !table->file->has_transactions();
+ no_trans_update_stmt= !table->file->has_transactions_and_rollback();
for ( ; ; it.rewind())
{
diff --git a/sql/sql_parse.cc b/sql/sql_parse.cc
index 18ccc830816..9d3ed11725c 100644
--- a/sql/sql_parse.cc
+++ b/sql/sql_parse.cc
@@ -437,16 +437,13 @@ bool stmt_causes_implicit_commit(THD *thd, uint mask)
DBUG_RETURN(FALSE);
switch (lex->sql_command) {
- case SQLCOM_DROP_TABLE:
- case SQLCOM_DROP_SEQUENCE:
- skip= (lex->tmp_table() ||
- (thd->variables.option_bits & OPTION_GTID_BEGIN));
- break;
case SQLCOM_ALTER_TABLE:
case SQLCOM_ALTER_SEQUENCE:
/* If ALTER TABLE of non-temporary table, do implicit commit */
skip= (lex->tmp_table());
break;
+ case SQLCOM_DROP_TABLE:
+ case SQLCOM_DROP_SEQUENCE:
case SQLCOM_CREATE_TABLE:
case SQLCOM_CREATE_SEQUENCE:
/*
@@ -1532,42 +1529,6 @@ public:
};
#endif
-/*
- Do an implict commit into the Aria storage engine
-*/
-
-static inline my_bool aria_implicit_commit(THD *thd)
-{
-#if defined(WITH_ARIA_STORAGE_ENGINE)
- if (thd_get_ha_data(thd, maria_hton))
- {
- MDL_request mdl_request;
- bool locked;
- int res;
- Silence_all_errors error_handler;
- DBUG_ASSERT(maria_hton);
-
- MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_COMMIT,
- MDL_EXPLICIT);
- /*
- We have to ignore any errors from acquire_lock and continue even if we
- don't get the lock as Aria can't roll back!
- This function is also called in some cases when the message is already
- sent to the user, so we can't even send a warning.
- */
- thd->push_internal_handler(& error_handler);
- locked= !thd->mdl_context.acquire_lock(&mdl_request,
- thd->variables.lock_wait_timeout);
- thd->pop_internal_handler();
- res= ha_maria::implicit_commit(thd, FALSE);
- if (locked)
- thd->mdl_context.release_lock(mdl_request.ticket);
- return res;
- }
-#endif
- return 0;
-}
-
/**
Perform one connection-level (COM_XXXX) command.
@@ -1921,8 +1882,6 @@ bool dispatch_command(enum enum_server_command command, THD *thd,
*/
char *beginning_of_next_stmt= (char*) parser_state.m_lip.found_semicolon;
- aria_implicit_commit(thd);
-
/* Finalize server status flags after executing a statement. */
thd->update_server_status();
thd->protocol->end_statement();
@@ -3170,14 +3129,13 @@ mysql_create_routine(THD *thd, LEX *lex)
statement takes metadata locks should be detected by a deadlock
detector in MDL subsystem and reported as errors.
- No need to commit/rollback statement transaction, it's not started.
-
TODO: Long-term we should either ensure that implicit GRANT statement
is written into binary log as a separate statement or make both
creation of routine and implicit GRANT parts of one fully atomic
statement.
*/
- DBUG_ASSERT(thd->transaction->stmt.is_empty());
+ if (trans_commit_stmt(thd))
+ goto wsrep_error_label;
close_thread_tables(thd);
/*
Check if the definer exists on slave,
@@ -3221,7 +3179,9 @@ mysql_create_routine(THD *thd, LEX *lex)
#endif
return false;
}
-#ifdef WITH_WSREP
+ (void) trans_commit_stmt(thd);
+
+#if !defined(NO_EMBEDDED_ACCESS_CHECKS) || defined(WITH_WSREP)
wsrep_error_label:
#endif
return true;
@@ -6049,7 +6009,6 @@ finish:
trans_commit_stmt(thd);
thd->get_stmt_da()->set_overwrite_status(false);
}
- aria_implicit_commit(thd);
}
/* Free tables. Set stage 'closing tables' */
@@ -6571,14 +6530,13 @@ drop_routine(THD *thd, LEX *lex)
statement takes metadata locks should be detected by a deadlock
detector in MDL subsystem and reported as errors.
- No need to commit/rollback statement transaction, it's not started.
-
TODO: Long-term we should either ensure that implicit REVOKE statement
is written into binary log as a separate statement or make both
dropping of routine and implicit REVOKE parts of one fully atomic
statement.
*/
- DBUG_ASSERT(thd->transaction->stmt.is_empty());
+ if (trans_commit_stmt(thd))
+ sp_result= SP_INTERNAL_ERROR;
close_thread_tables(thd);
if (sp_result != SP_KEY_NOT_FOUND &&
diff --git a/sql/sql_sequence.cc b/sql/sql_sequence.cc
index 45bc519816a..dd12cb85558 100644
--- a/sql/sql_sequence.cc
+++ b/sql/sql_sequence.cc
@@ -301,8 +301,7 @@ bool sequence_insert(THD *thd, LEX *lex, TABLE_LIST *org_table_list)
if (!temporary_table)
{
/*
- The following code works like open_system_tables_for_read() and
- close_system_tables()
+ The following code works like open_system_tables_for_read()
The idea is:
- Copy the table_list object for the sequence that was created
- Backup the current state of open tables and create a new
diff --git a/sql/sql_show.cc b/sql/sql_show.cc
index 2528134f4ee..db5b4d1c5fd 100644
--- a/sql/sql_show.cc
+++ b/sql/sql_show.cc
@@ -6105,7 +6105,7 @@ static my_bool iter_schema_engines(THD *thd, plugin_ref plugin,
table->field[1]->store(option_name, strlen(option_name), scs);
table->field[2]->store(plugin_decl(plugin)->descr,
strlen(plugin_decl(plugin)->descr), scs);
- tmp= &yesno[MY_TEST(hton->commit)];
+ tmp= &yesno[MY_TEST(hton->commit && !(hton->flags & HTON_NO_ROLLBACK))];
table->field[3]->store(tmp->str, tmp->length, scs);
table->field[3]->set_notnull();
tmp= &yesno[MY_TEST(hton->prepare)];
@@ -6492,7 +6492,6 @@ int fill_schema_proc(THD *thd, TABLE_LIST *tables, COND *cond)
TABLE *table= tables->table;
bool full_access;
char definer[USER_HOST_BUFF_SIZE];
- Open_tables_backup open_tables_state_backup;
enum enum_schema_tables schema_table_idx=
get_schema_table_idx(tables->schema_table);
DBUG_ENTER("fill_schema_proc");
@@ -6507,8 +6506,12 @@ int fill_schema_proc(THD *thd, TABLE_LIST *tables, COND *cond)
proc_tables.lock_type= TL_READ;
full_access= !check_table_access(thd, SELECT_ACL, &proc_tables, FALSE,
1, TRUE);
- if (!(proc_table= open_proc_table_for_read(thd, &open_tables_state_backup)))
+
+ start_new_trans new_trans(thd);
+
+ if (!(proc_table= open_proc_table_for_read(thd)))
{
+ new_trans.restore_old_transaction();
DBUG_RETURN(1);
}
@@ -6550,7 +6553,9 @@ err:
if (proc_table->file->inited)
(void) proc_table->file->ha_index_end();
- close_system_tables(thd, &open_tables_state_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
+
thd->variables.sql_mode = sql_mode_was;
DBUG_RETURN(res);
}
diff --git a/sql/sql_statistics.cc b/sql/sql_statistics.cc
index a6c489f02be..f51f63020a3 100644
--- a/sql/sql_statistics.cc
+++ b/sql/sql_statistics.cc
@@ -230,17 +230,17 @@ index_stat_def= {INDEX_STAT_N_FIELDS, index_stat_fields, 4, index_stat_pk_col};
Open all statistical tables and lock them
*/
-static int open_stat_tables(THD *thd, TABLE_LIST *tables,
- Open_tables_backup *backup, bool for_write)
+static int open_stat_tables(THD *thd, TABLE_LIST *tables, bool for_write)
{
int rc;
-
Dummy_error_handler deh; // suppress errors
+ DBUG_ASSERT(thd->internal_transaction());
+
thd->push_internal_handler(&deh);
init_table_list_for_stat_tables(tables, for_write);
init_mdl_requests(tables);
thd->in_sub_stmt|= SUB_STMT_STAT_TABLES;
- rc= open_system_tables_for_read(thd, tables, backup);
+ rc= open_system_tables_for_read(thd, tables);
thd->in_sub_stmt&= ~SUB_STMT_STAT_TABLES;
thd->pop_internal_handler();
@@ -253,7 +253,7 @@ static int open_stat_tables(THD *thd, TABLE_LIST *tables,
stat_table_intact.check(tables[COLUMN_STAT].table, &column_stat_def) ||
stat_table_intact.check(tables[INDEX_STAT].table, &index_stat_def)))
{
- close_system_tables(thd, backup);
+ close_thread_tables(thd);
rc= 1;
}
@@ -270,13 +270,12 @@ static int open_stat_tables(THD *thd, TABLE_LIST *tables,
stat tables need to be adjusted accordingly.
*/
static inline int open_stat_table_for_ddl(THD *thd, TABLE_LIST *table,
- const LEX_CSTRING *stat_tab_name,
- Open_tables_backup *backup)
+ const LEX_CSTRING *stat_tab_name)
{
table->init_one_table(&MYSQL_SCHEMA_NAME, stat_tab_name, NULL, TL_WRITE);
No_such_table_error_handler nst_handler;
thd->push_internal_handler(&nst_handler);
- int res= open_system_tables_for_read(thd, table, backup);
+ int res= open_system_tables_for_read(thd, table);
thd->pop_internal_handler();
return res;
}
@@ -2875,18 +2874,18 @@ int collect_statistics_for_table(THD *thd, TABLE *table)
int update_statistics_for_table(THD *thd, TABLE *table)
{
TABLE_LIST tables[STATISTICS_TABLES];
- Open_tables_backup open_tables_backup;
uint i;
int err;
enum_binlog_format save_binlog_format;
int rc= 0;
TABLE *stat_table;
-
DBUG_ENTER("update_statistics_for_table");
DEBUG_SYNC(thd, "statistics_update_start");
- if (open_stat_tables(thd, tables, &open_tables_backup, TRUE))
+ start_new_trans new_trans(thd);
+
+ if (open_stat_tables(thd, tables, TRUE))
DBUG_RETURN(rc);
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -2936,8 +2935,9 @@ int update_statistics_for_table(THD *thd, TABLE *table)
}
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ if (thd->commit_whole_transaction_and_close_tables())
+ rc= 1;
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
@@ -3291,7 +3291,6 @@ int read_statistics_for_tables_if_needed(THD *thd, TABLE_LIST *tables)
int read_statistics_for_tables(THD *thd, TABLE_LIST *tables)
{
TABLE_LIST stat_tables[STATISTICS_TABLES];
- Open_tables_backup open_tables_backup;
DBUG_ENTER("read_statistics_for_tables");
@@ -3332,7 +3331,9 @@ int read_statistics_for_tables(THD *thd, TABLE_LIST *tables)
if (!statistics_for_tables_is_needed(thd, tables))
DBUG_RETURN(0);
- if (open_stat_tables(thd, stat_tables, &open_tables_backup, FALSE))
+ start_new_trans new_trans(thd);
+
+ if (open_stat_tables(thd, stat_tables, FALSE))
DBUG_RETURN(1);
for (TABLE_LIST *tl= tables; tl; tl= tl->next_global)
@@ -3364,7 +3365,8 @@ int read_statistics_for_tables(THD *thd, TABLE_LIST *tables)
}
}
- close_system_tables(thd, &open_tables_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
DBUG_RETURN(0);
}
@@ -3404,8 +3406,10 @@ int delete_statistics_for_table(THD *thd, const LEX_CSTRING *db,
Open_tables_backup open_tables_backup;
int rc= 0;
DBUG_ENTER("delete_statistics_for_table");
+
+ start_new_trans new_trans(thd);
- if (open_stat_tables(thd, tables, &open_tables_backup, TRUE))
+ if (open_stat_tables(thd, tables, TRUE))
DBUG_RETURN(0);
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -3448,8 +3452,8 @@ int delete_statistics_for_table(THD *thd, const LEX_CSTRING *db,
rc= 1;
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ thd->commit_whole_transaction_and_close_tables();
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
@@ -3481,12 +3485,12 @@ int delete_statistics_for_column(THD *thd, TABLE *tab, Field *col)
enum_binlog_format save_binlog_format;
TABLE *stat_table;
TABLE_LIST tables;
- Open_tables_backup open_tables_backup;
int rc= 0;
DBUG_ENTER("delete_statistics_for_column");
- if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[1],
- &open_tables_backup))
+ start_new_trans new_trans(thd);
+
+ if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[1]))
DBUG_RETURN(0);
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -3502,8 +3506,9 @@ int delete_statistics_for_column(THD *thd, TABLE *tab, Field *col)
}
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ if (thd->commit_whole_transaction_and_close_tables())
+ rc= 1;
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
@@ -3538,12 +3543,12 @@ int delete_statistics_for_index(THD *thd, TABLE *tab, KEY *key_info,
enum_binlog_format save_binlog_format;
TABLE *stat_table;
TABLE_LIST tables;
- Open_tables_backup open_tables_backup;
int rc= 0;
DBUG_ENTER("delete_statistics_for_index");
- if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[2],
- &open_tables_backup))
+ start_new_trans new_trans(thd);
+
+ if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[2]))
DBUG_RETURN(0);
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -3579,8 +3584,9 @@ int delete_statistics_for_index(THD *thd, TABLE *tab, KEY *key_info,
rc= 1;
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ if (thd->commit_whole_transaction_and_close_tables())
+ rc= 1;
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
@@ -3622,14 +3628,13 @@ int rename_table_in_stat_tables(THD *thd, const LEX_CSTRING *db,
enum_binlog_format save_binlog_format;
TABLE *stat_table;
TABLE_LIST tables[STATISTICS_TABLES];
- Open_tables_backup open_tables_backup;
int rc= 0;
DBUG_ENTER("rename_table_in_stat_tables");
- if (open_stat_tables(thd, tables, &open_tables_backup, TRUE))
- {
+ start_new_trans new_trans(thd);
+
+ if (open_stat_tables(thd, tables, TRUE))
DBUG_RETURN(0); // not an error
- }
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -3678,8 +3683,9 @@ int rename_table_in_stat_tables(THD *thd, const LEX_CSTRING *db,
}
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ if (thd->commit_whole_transaction_and_close_tables())
+ rc= 1;
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
@@ -3712,15 +3718,15 @@ int rename_column_in_stat_tables(THD *thd, TABLE *tab, Field *col,
enum_binlog_format save_binlog_format;
TABLE *stat_table;
TABLE_LIST tables;
- Open_tables_backup open_tables_backup;
int rc= 0;
DBUG_ENTER("rename_column_in_stat_tables");
if (tab->s->tmp_table != NO_TMP_TABLE)
DBUG_RETURN(0);
- if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[1],
- &open_tables_backup))
+ start_new_trans new_trans(thd);
+
+ if (open_stat_table_for_ddl(thd, &tables, &stat_table_name[1]))
DBUG_RETURN(rc);
save_binlog_format= thd->set_current_stmt_binlog_format_stmt();
@@ -3737,8 +3743,9 @@ int rename_column_in_stat_tables(THD *thd, TABLE *tab, Field *col,
}
thd->restore_stmt_binlog_format(save_binlog_format);
-
- close_system_tables(thd, &open_tables_backup);
+ if (thd->commit_whole_transaction_and_close_tables())
+ rc= 1;
+ new_trans.restore_old_transaction();
DBUG_RETURN(rc);
}
diff --git a/sql/sql_trigger.cc b/sql/sql_trigger.cc
index 726ba6d3cf2..0b2d2d8ca1c 100644
--- a/sql/sql_trigger.cc
+++ b/sql/sql_trigger.cc
@@ -590,10 +590,8 @@ bool mysql_create_or_drop_trigger(THD *thd, TABLE_LIST *tables, bool create)
table->triggers->create_trigger(thd, tables, &stmt_query):
table->triggers->drop_trigger(thd, tables, &stmt_query));
- if (result)
- goto end;
-
close_all_tables_for_name(thd, table->s, HA_EXTRA_NOT_USED, NULL);
+
/*
Reopen the table if we were under LOCK TABLES.
Ignore the return value for now. It's better to
diff --git a/sql/sql_truncate.cc b/sql/sql_truncate.cc
index a745293ec31..8ed996c0a8d 100644
--- a/sql/sql_truncate.cc
+++ b/sql/sql_truncate.cc
@@ -246,7 +246,7 @@ Sql_cmd_truncate_table::handler_truncate(THD *thd, TABLE_LIST *table_ref,
inspite of errors.
*/
if (error == HA_ERR_WRONG_COMMAND ||
- table_ref->table->file->has_transactions())
+ table_ref->table->file->has_transactions_and_rollback())
DBUG_RETURN(TRUNCATE_FAILED_SKIP_BINLOG);
else
DBUG_RETURN(TRUNCATE_FAILED_BUT_BINLOG);
diff --git a/sql/sql_update.cc b/sql/sql_update.cc
index 43f33184fe3..8cd3b9f0411 100644
--- a/sql/sql_update.cc
+++ b/sql/sql_update.cc
@@ -931,7 +931,7 @@ update_begin:
thd->count_cuted_fields= CHECK_FIELD_WARN;
thd->cuted_fields=0L;
- transactional_table= table->file->has_transactions();
+ transactional_table= table->file->has_transactions_and_rollback();
thd->abort_on_warning= !ignore && thd->is_strict_mode();
if (do_direct_update)
@@ -2572,7 +2572,7 @@ int multi_update::send_data(List<Item> &not_used_values)
}
/* non-transactional or transactional table got modified */
/* either multi_update class' flag is raised in its branch */
- if (table->file->has_transactions())
+ if (table->file->has_transactions_and_rollback())
transactional_tables= TRUE;
else
{
@@ -2891,7 +2891,7 @@ int multi_update::do_updates()
if (updated != org_updated)
{
- if (table->file->has_transactions())
+ if (table->file->has_transactions_and_rollback())
transactional_tables= TRUE;
else
{
@@ -2928,7 +2928,7 @@ err2:
if (updated != org_updated)
{
- if (table->file->has_transactions())
+ if (table->file->has_transactions_and_rollback())
transactional_tables= TRUE;
else
{
diff --git a/sql/transaction.cc b/sql/transaction.cc
index 23d6ba1ecba..b61fa43e579 100644
--- a/sql/transaction.cc
+++ b/sql/transaction.cc
@@ -73,11 +73,10 @@ static bool trans_check(THD *thd)
if (unlikely(thd->in_sub_stmt))
my_error(ER_COMMIT_NOT_ALLOWED_IN_SF_OR_TRG, MYF(0));
- if (thd->transaction->xid_state.is_explicit_XA())
- thd->transaction->xid_state.er_xaer_rmfail();
- else
+ if (!thd->transaction->xid_state.is_explicit_XA())
DBUG_RETURN(FALSE);
+ thd->transaction->xid_state.er_xaer_rmfail();
DBUG_RETURN(TRUE);
}
@@ -261,10 +260,10 @@ bool trans_commit(THD *thd)
mysql_mutex_assert_not_owner(&LOCK_after_binlog_sync);
mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
- /*
- if res is non-zero, then ha_commit_trans has rolled back the
- transaction, so the hooks for rollback will be called.
- */
+ /*
+ if res is non-zero, then ha_commit_trans has rolled back the
+ transaction, so the hooks for rollback will be called.
+ */
#ifdef HAVE_REPLICATION
if (res)
repl_semisync_master.wait_after_rollback(thd, FALSE);
diff --git a/sql/tztime.cc b/sql/tztime.cc
index 72db37fde9c..36f527113ea 100644
--- a/sql/tztime.cc
+++ b/sql/tztime.cc
@@ -2330,16 +2330,21 @@ my_tz_find(THD *thd, const String *name)
else if (time_zone_tables_exist)
{
TABLE_LIST tz_tables[MY_TZ_TABLES_COUNT];
- Open_tables_backup open_tables_state_backup;
+ /*
+ Allocate start_new_trans with malloc as it's > 4000 bytes and this
+ function can be called deep inside a stored procedure
+ */
+ start_new_trans *new_trans= new start_new_trans(thd);
tz_init_table_list(tz_tables);
init_mdl_requests(tz_tables);
- if (!open_system_tables_for_read(thd, tz_tables,
- &open_tables_state_backup))
+ if (!open_system_tables_for_read(thd, tz_tables))
{
result_tz= tz_load_from_open_tables(name, tz_tables);
- close_system_tables(thd, &open_tables_state_backup);
+ thd->commit_whole_transaction_and_close_tables();
}
+ new_trans->restore_old_transaction();
+ delete new_trans;
}
}