Commit 40e34d03 authored by Monty's avatar Monty

Added information_schema.slave_status

parent 5cb9e56a
......@@ -97,7 +97,7 @@ if ($all_slaves_status)
{
--die Bug in test case: Both $all_slaves_status and $slave_name are set.
}
--let $_show_query=SHOW ALL SLAVES STATUS
--let $_show_query=SELECT * from information_schema.status
}
if ($slave_name)
{
......
......@@ -14,6 +14,8 @@ GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, SHUTDOWN, PROCESS, F
#
SHOW SLAVE STATUS;
ERROR 42000: Access denied; you need (at least one of) the SUPER, SLAVE MONITOR privilege(s) for this operation
SELECT * from information_schema.slave_status;
ERROR 42000: Access denied; you need (at least one of) the SUPER, SLAVE MONITOR privilege(s) for this operation
#
# Verify that having REPLICATION SLAVE ADMIN doesn't allow SHOW RELAYLOG EVENTS
# Expected error: Access denied; you need (at least one of) the REPLICA MONITOR
......
......@@ -41,6 +41,8 @@ SHOW GRANTS;
--echo #
--error ER_SPECIFIC_ACCESS_DENIED_ERROR
SHOW SLAVE STATUS;
--error ER_SPECIFIC_ACCESS_DENIED_ERROR
SELECT * from information_schema.slave_status;
--echo #
--echo # Verify that having REPLICATION SLAVE ADMIN doesn't allow SHOW RELAYLOG EVENTS
......
......@@ -32,8 +32,8 @@ EXECUTE stmt;
EXECUTE stmt;
DEALLOCATE PREPARE stmt;
let $master_host= query_get_value(SHOW SLAVE STATUS, Master_Host, 1);
let $master_user= query_get_value(SHOW SLAVE STATUS, Master_User, 1);
let $master_host= `select Master_Host from information_schema.slave_status`;
let $master_user= `select Master_user from information_schema.slave_status`;
--echo # Master_Host : $master_host
--echo # Master_User : $master_user
......
......@@ -34,10 +34,11 @@ SET STATEMENT sql_log_bin=0 FOR create table t1 (a int) engine=innodb;
start slave io_thread;
--source include/sync_io_with_master.inc
select * from information_schema.slave_status;
# Ensure Master_last_event_time and Slave_last_event_time are not yet set
--let $master_time= query_get_value(SHOW ALL SLAVES STATUS, Master_last_event_time, 1)
--let $master_time= `select Master_last_event_time from information_schema.slave_status`
--echo master_time: $master_time (should be NULL)
--let $slave_time= query_get_value(SHOW ALL SLAVES STATUS, Slave_last_event_time, 1)
--let $slave_time=`select Slave_last_event_time from information_schema.slave_status`
--echo slave_time: $slave_time (should be NULL)
--echo # Sleep 2s to create a time gap between the header events (i.e.
......@@ -57,7 +58,7 @@ insert into t1 values (0);
--connection server_2
--source include/sync_io_with_master.inc
--let $mle_time= query_get_value(SHOW ALL SLAVES STATUS, Master_last_event_time, 1)
--let $mle_time= `select Master_last_event_time from information_schema.slave_status`
--let $mle_time_unix= `select truncate(unix_timestamp("$mle_time"),0)`
if (`SELECT ($mle_time_unix < $t1_time_begin)`)
{
......@@ -66,13 +67,14 @@ if (`SELECT ($mle_time_unix < $t1_time_begin)`)
--die Master_last_event_time did not correspond to time that the transaction was binlogged on primary
}
--let $slave_time= query_get_value(SHOW ALL SLAVES STATUS, Slave_last_event_time, 1)
--let $slave_time= `Slave_last_event_time from information_schema.slave_status`
if (`select strcmp("$slave_time", "NULL") != 0`)
{
--echo # Slave_last_event_time: $slave_time
--die SQL thread was never started, Slave_last_event_time should be NULL
}
# Check that we also get the values from show all slaves status
--let $time_diff= query_get_value(SHOW ALL SLAVES STATUS, Master_Slave_time_diff, 1)
if (`select strcmp("$time_diff", "NULL") != 0`)
{
......@@ -87,7 +89,6 @@ if (`select strcmp("$time_diff", "NULL") != 0`)
--echo # transaction from committing (and thereby progressing
--echo # Slave_last_event_time to represent commit-time).
--connect (server_2_2,127.0.0.1,root,,test,$SERVER_MYPORT_2,)
lock tables t1 write;
......
!include suite/rpl/my.cnf
[mysqld.1]
[mysqld.2]
log-slave-updates
binlog-checksum=CRC32
[mysqld.3]
log-slave-updates
binlog-checksum=CRC32
[ENV]
SERVER_MYPORT_3= @mysqld.3.port
SERVER_MYSOCK_3= @mysqld.3.socket
......@@ -1139,7 +1139,8 @@ enum enum_schema_tables
SCH_TABLE_PRIVILEGES,
SCH_TRIGGERS,
SCH_USER_PRIVILEGES,
SCH_VIEWS
SCH_VIEWS,
SCH_SLAVE_STATUS,
};
struct TABLE_SHARE;
......
......@@ -998,6 +998,18 @@ Log_event* Log_event::read_log_event(IO_CACHE* file,
DBUG_RETURN(res);
}
/*
Update thd->orig_exec_time
*/
inline void set_orig_exec_time_in_thd(ulong exec_time)
{
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
THD *thd= current_thd;
if (likely(thd))
thd->orig_exec_time= exec_time;
#endif
}
/**
Binlog format tolerance is in (buf, event_len, fdle)
......@@ -1137,24 +1149,18 @@ Log_event* Log_event::read_log_event(const uchar *buf, uint event_len,
switch(event_type) {
case QUERY_EVENT:
ev= new Query_log_event(buf, event_len, fdle, QUERY_EVENT);
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
current_thd->orig_exec_time= ((Query_log_event*) ev)->exec_time;
#endif
set_orig_exec_time_in_thd(((Query_log_event*) ev)->exec_time);
break;
case QUERY_COMPRESSED_EVENT:
ev= new Query_compressed_log_event(buf, event_len, fdle,
QUERY_COMPRESSED_EVENT);
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
current_thd->orig_exec_time= ((Query_compressed_log_event*) ev)->exec_time;
#endif
set_orig_exec_time_in_thd(((Query_compressed_log_event*) ev)->exec_time);
break;
case LOAD_EVENT:
case NEW_LOAD_EVENT:
/* This can only happen when reading old binary logs before MySQL 5.0 */
ev= new Load_log_event(buf, event_len, fdle);
#if !defined(MYSQL_CLIENT) && defined(HAVE_REPLICATION)
current_thd->orig_exec_time= ((Load_log_event*) ev)->exec_time;
#endif
set_orig_exec_time_in_thd(((Load_log_event*) ev)->exec_time);
break;
case ROTATE_EVENT:
ev= new Rotate_log_event(buf, event_len, fdle);
......
......@@ -1877,6 +1877,14 @@ void Domain_id_filter::store_ids(THD *thd)
}
}
void Domain_id_filter::store_ids(Field ***field)
{
for (int i= DO_DOMAIN_IDS; i <= IGNORE_DOMAIN_IDS; i ++)
{
field_store_ids(*((*field)++), &m_domain_ids[i]);
}
}
/**
Initialize the given domain_id list (DYNAMIC_ARRAY) with the
space-separated list of numbers from the specified IO_CACHE where
......@@ -1951,20 +1959,10 @@ void update_change_master_ids(DYNAMIC_ARRAY *new_ids, DYNAMIC_ARRAY *old_ids)
return;
}
/**
Serialize and store the ids from the given ids DYNAMIC_ARRAY into the thd's
protocol buffer.
@param thd [IN] thread handler
@param ids [IN] ids list
@retval void
*/
void prot_store_ids(THD *thd, DYNAMIC_ARRAY *ids)
static size_t store_ids(DYNAMIC_ARRAY *ids, char *buff, size_t buff_len)
{
char buff[FN_REFLEN];
uint i, cur_len;
uint i;
size_t cur_len;
for (i= 0, buff[0]= 0, cur_len= 0; i < ids->elements; i++)
{
......@@ -1972,7 +1970,7 @@ void prot_store_ids(THD *thd, DYNAMIC_ARRAY *ids)
char dbuff[FN_REFLEN];
get_dynamic(ids, (void *) &id, i);
len= sprintf(dbuff, (i == 0 ? "%lu" : ", %lu"), id);
if (cur_len + len + 4 > FN_REFLEN)
if (cur_len + len + 4 > buff_len)
{
/*
break the loop whenever remained space could not fit
......@@ -1983,8 +1981,33 @@ void prot_store_ids(THD *thd, DYNAMIC_ARRAY *ids)
}
cur_len+= sprintf(buff + cur_len, "%s", dbuff);
}
return cur_len;
}
/**
Serialize and store the ids from the given ids DYNAMIC_ARRAY into the thd's
protocol buffer.
@param thd [IN] thread handler
@param ids [IN] ids list
@retval void
*/
void prot_store_ids(THD *thd, DYNAMIC_ARRAY *ids)
{
char buff[FN_REFLEN];
uint cur_len= store_ids(ids, buff, sizeof(buff));
thd->protocol->store(buff, cur_len, &my_charset_bin);
return;
}
void field_store_ids(Field *field, DYNAMIC_ARRAY *ids)
{
char buff[FN_REFLEN];
size_t cur_len= store_ids(ids, buff, sizeof(buff));
field->store(buff, cur_len, &my_charset_bin);
}
......
......@@ -105,7 +105,8 @@ class Domain_id_filter
@retval void
*/
void store_ids(THD *thd);
/* Same as above, but store the id's into a group of fields */
void store_ids(Field ***field);
/*
Initialize the given domain id list (DYNAMIC_ARRAY) with the
space-separated list of numbers from the specified IO_CACHE where
......@@ -395,7 +396,7 @@ int flush_master_info(Master_info* mi,
void copy_filter_setting(Rpl_filter* dst_filter, Rpl_filter* src_filter);
void update_change_master_ids(DYNAMIC_ARRAY *new_ids, DYNAMIC_ARRAY *old_ids);
void prot_store_ids(THD *thd, DYNAMIC_ARRAY *ids);
void field_store_ids(Field *field, DYNAMIC_ARRAY *ids);
/*
Multi master are handled trough this struct.
Changes to this needs to be protected by LOCK_active_mi;
......
......@@ -3435,9 +3435,289 @@ static bool send_show_master_info_data(THD *thd, Master_info *mi, bool full,
}
inline void store_string_or_null(Field **field, const char *str)
{
if (str)
(*field)->store(str, strlen(str), &my_charset_bin);
else
(*field)->set_null();
}
inline void store_string(Field **field, const char *str)
{
(*field)->store(str, strlen(str), &my_charset_bin);
}
inline void store_string(Field **field, const LEX_CSTRING *str)
{
(*field)->store(str->str, str->length, &my_charset_bin);
}
void store_list(Field **field, I_List<i_string>* str_list)
{
char buf[256];
String tmp(buf, sizeof(buf), &my_charset_bin);
uint32 len;
I_List_iterator<i_string> it(*str_list);
i_string* s;
tmp.length(0);
while ((s=it++))
{
tmp.append(s->ptr, strlen(s->ptr));
tmp.append(',');
}
if ((len= tmp.length()))
len--; // Remove last ','
(*field)->store((char*) tmp.ptr(), len, tmp.charset());
}
/*
Store master info for information_schema_tables
*/
void store_master_info(THD *thd, Master_info *mi, TABLE *table,
String *gtid_pos)
{
Field **field= table->field;
uint i=0;
const char *msg;
Rpl_filter *rpl_filter= mi->rpl_filter;
StringBuffer<256> tmp;
DBUG_ENTER("store_master_info_data");
table->clear_null_bits();
(*field++)->store(mi->connection_name.str, mi->connection_name.length,
&my_charset_bin);
mysql_mutex_lock(&mi->run_lock);
msg= (mi->rli.sql_driver_thd ?
mi->rli.sql_driver_thd->get_proc_info() : "");
store_string_or_null(field++, msg);
msg= mi->io_thd ? mi->io_thd->get_proc_info() : "";
store_string_or_null(field++, msg);
mysql_mutex_unlock(&mi->run_lock);
mysql_mutex_lock(&mi->data_lock);
mysql_mutex_lock(&mi->rli.data_lock);
/* err_lock is to protect mi->last_error() */
mysql_mutex_lock(&mi->err_lock);
/* err_lock is to protect mi->rli.last_error() */
mysql_mutex_lock(&mi->rli.err_lock);
DBUG_EXECUTE_IF("hold_sss_with_err_lock", {
DBUG_ASSERT(!debug_sync_set_action(thd,
STRING_WITH_LEN("now SIGNAL sss_got_err_lock "
"WAIT_FOR sss_continue")));
DBUG_SET("-d,hold_sss_with_err_lock");
});
store_string_or_null(field++, mi->host);
store_string_or_null(field++, mi->user);
(*field++)->store((uint32) mi->port);
(*field++)->store((uint32) mi->connect_retry);
(*field++)->store(mi->master_log_name, strlen(mi->master_log_name),
&my_charset_bin);
(*field++)->store((ulonglong) mi->master_log_pos);
msg= (mi->rli.group_relay_log_name +
dirname_length(mi->rli.group_relay_log_name));
store_string(field++, msg);
(*field++)->store((ulonglong) mi->rli.group_relay_log_pos);
store_string(field++, mi->rli.group_master_log_name);
store_string(field++, &slave_running[mi->slave_running]);
store_string(field++, mi->rli.slave_running ? &msg_yes : &msg_no);
store_list(field++, rpl_filter->get_do_db());
store_list(field++, rpl_filter->get_ignore_db());
rpl_filter->get_do_table(&tmp);
(*field++)->store(tmp.ptr(), tmp.length(), &my_charset_bin);
rpl_filter->get_ignore_table(&tmp);
(*field++)->store(tmp.ptr(), tmp.length(), &my_charset_bin);
rpl_filter->get_wild_do_table(&tmp);
(*field++)->store(tmp.ptr(), tmp.length(), &my_charset_bin);
rpl_filter->get_wild_ignore_table(&tmp);
(*field++)->store(tmp.ptr(), tmp.length(), &my_charset_bin);
(*field++)->store(mi->rli.last_error().number);
store_string_or_null(field++, mi->rli.last_error().message);
(*field++)->store((uint32) mi->rli.slave_skip_counter);
(*field++)->store((ulonglong) mi->rli.group_master_log_pos);
(*field++)->store((ulonglong) mi->rli.log_space_total);
msg= (mi->rli.until_condition==Relay_log_info::UNTIL_NONE ? "None" :
(mi->rli.until_condition==Relay_log_info::UNTIL_MASTER_POS? "Master":
(mi->rli.until_condition==Relay_log_info::UNTIL_RELAY_POS? "Relay":
"Gtid")));
(*field++)->store(msg, strlen(msg), &my_charset_bin);
store_string_or_null(field++, mi->rli.until_log_name);
(*field++)->store((ulonglong) mi->rli.until_log_pos);
#ifdef HAVE_OPENSSL
(*field++)->store(mi->ssl ? &msg_yes : &msg_no, &my_charset_bin);
#else
(*field++)->store(mi->ssl ? &msg_ignored: &msg_no, &my_charset_bin);
#endif
store_string_or_null(field++, mi->ssl_ca);
store_string_or_null(field++, mi->ssl_capath);
store_string_or_null(field++, mi->ssl_cert);
store_string_or_null(field++, mi->ssl_cipher);
store_string_or_null(field++, mi->ssl_key);
/*
Seconds_Behind_Master: if SQL thread is running and I/O thread is
connected, we can compute it otherwise show NULL (i.e. unknown).
*/
if ((mi->slave_running == MYSQL_SLAVE_RUN_READING) &&
mi->rli.slave_running)
{
long time_diff;
bool idle;
time_t stamp= mi->rli.last_master_timestamp;
if (!stamp)
idle= true;
else
{
idle= mi->rli.sql_thread_caught_up;
/*
The idleness of the SQL thread is needed for the parallel slave
because events can be ignored before distribution to a worker thread.
That is, Seconds_Behind_Master should still be calculated and visible
while the slave is processing ignored events, such as those skipped
due to slave_skip_counter.
*/
if (mi->using_parallel() && idle &&
!rpl_parallel::workers_idle(&mi->rli))
idle= false;
}
if (idle)
time_diff= 0;
else
{
time_diff= ((long)(time(0) - stamp) - mi->clock_diff_with_master);
/*
Apparently on some systems time_diff can be <0. Here are possible
reasons related to MySQL:
- the master is itself a slave of another master whose time is ahead.
- somebody used an explicit SET TIMESTAMP on the master.
Possible reason related to granularity-to-second of time functions
(nothing to do with MySQL), which can explain a value of -1:
assume the master's and slave's time are perfectly synchronized, and
that at slave's connection time, when the master's timestamp is read,
it is at the very end of second 1, and (a very short time later) when
the slave's timestamp is read it is at the very beginning of second
2. Then the recorded value for master is 1 and the recorded value for
slave is 2. At SHOW SLAVE STATUS time, assume that the difference
between timestamp of slave and rli->last_master_timestamp is 0
(i.e. they are in the same second), then we get 0-(2-1)=-1 as a result.
This confuses users, so we don't go below 0.
last_master_timestamp == 0 (an "impossible" timestamp 1970) is a
special marker to say "consider we have caught up".
*/
if (time_diff < 0)
time_diff= 0;
}
(*field++)->store((longlong)time_diff);
}
else
(*field++)->set_null();
(*field++)->store(mi->ssl_verify_server_cert? &msg_yes : &msg_no,
&my_charset_bin);
// Last_IO_Errno
(*field++)->store(mi->last_error().number);
// Last_IO_Error
store_string_or_null(field++, mi->last_error().message);
// Last_SQL_Errno
(*field++)->store(mi->rli.last_error().number);
// Last_SQL_Error
store_string_or_null(field++, mi->rli.last_error().message);
// Replicate_Ignore_Server_Ids
field_store_ids((*field++), &mi->ignore_server_ids);
// Master_Server_id
(*field++)->store((uint32) mi->master_id);
// SQL_Delay
// Master_Ssl_Crl
store_string_or_null(field++, mi->ssl_crl);
// Master_Ssl_Crlpath
store_string_or_null(field++, mi->ssl_crlpath);
// Using_Gtid
store_string_or_null(field++, mi->using_gtid_astext(mi->using_gtid));
// Gtid_IO_Pos
{
mi->gtid_current_pos.to_string(&tmp);
(*field++)->store(tmp.ptr(), tmp.length(), &my_charset_bin);
}
// Replicate_Do_Domain_Ids & Replicate_Ignore_Domain_Ids
mi->domain_id_filter.store_ids(&field);
// Parallel_Mode
{
const char *mode_name= get_type(&slave_parallel_mode_typelib,
mi->parallel_mode);
(*field++)->store(mode_name, strlen(mode_name), &my_charset_bin);
}
(*field++)->store((uint32) mi->rli.get_sql_delay());
// SQL_Remaining_Delay
// THD::proc_info is not protected by any lock, so we read it once
// to ensure that we use the same value throughout this function.
const char *slave_sql_running_state=
mi->rli.sql_driver_thd ? mi->rli.sql_driver_thd->proc_info : "";
if (slave_sql_running_state == stage_sql_thd_waiting_until_delay.m_name)
{
time_t t= my_time(0), sql_delay_end= mi->rli.get_sql_delay_end();
(*field++)->store((uint32)(t < sql_delay_end ? sql_delay_end - t : 0));
}
else
(*field++)->set_null();
// Slave_SQL_Running_State
store_string_or_null(field++, slave_sql_running_state);
(*field++)->store(mi->total_ddl_groups);
(*field++)->store(mi->total_non_trans_groups);
(*field++)->store(mi->total_trans_groups);
(*field++)->store((uint32) mi->rli.retried_trans);
(*field++)->store((ulonglong) mi->rli.max_relay_log_size);
(*field++)->store(mi->rli.executed_entries);
(*field++)->store((uint) mi->received_heartbeats);
(*field++)->store((double) mi->heartbeat_period);
(*field++)->store(gtid_pos->ptr(), gtid_pos->length(), &my_charset_bin);
if (mi->rli.newest_master_timestamp)
(*field++)->store_timestamp((my_time_t) mi->rli.newest_master_timestamp, 0);
else
field[i++]->set_null();
if (mi->rli.slave_timestamp)
{
DBUG_ASSERT(mi->rli.newest_master_timestamp);
(*field++)->store_timestamp((my_time_t) mi->rli.slave_timestamp, 0);
(*field++)->store((uint) (mi->rli.newest_master_timestamp -
mi->rli.slave_timestamp));
}
else
field[i++]->set_null();
mysql_mutex_unlock(&mi->rli.err_lock);
mysql_mutex_unlock(&mi->err_lock);
mysql_mutex_unlock(&mi->rli.data_lock);
mysql_mutex_unlock(&mi->data_lock);
DBUG_VOID_RETURN;
}
/* Used to sort connections by name */
static int cmp_mi_by_name(const Master_info **arg1,
int cmp_mi_by_name(const Master_info **arg1,
const Master_info **arg2)
{
return my_strcasecmp(system_charset_info, (*arg1)->connection_name.str,
......@@ -3501,7 +3781,8 @@ bool show_all_master_info(THD* thd)
for (i= 0; i < elements; i++)
{
if (send_show_master_info_data(thd, tmp[i], 1, &gtid_pos))
if (tmp[i]->host[0] &&
send_show_master_info_data(thd, tmp[i], 1, &gtid_pos))
DBUG_RETURN(TRUE);
}
......
......@@ -279,7 +279,9 @@ void slave_background_kill_request(THD *to_kill);
void slave_background_gtid_pos_create_request
(rpl_slave_state::gtid_pos_table *table_entry);
void slave_background_gtid_pending_delete_request(void);
void store_master_info(THD *thd, Master_info *mi, TABLE *table,
String *gtid_pos);
int cmp_mi_by_name(const Master_info **arg1, const Master_info **arg2);
extern Master_info *active_mi; /* active_mi for multi-master */
extern Master_info *default_master_info; /* To replace active_mi */
extern Master_info_index *master_info_index;
......
......@@ -47,6 +47,8 @@
#include "sql_derived.h"
#include "sql_statistics.h"
#include "sql_connect.h"
#include "sql_repl.h" // rpl_load_gtid_state
#include "rpl_mi.h" // master_info_index
#include "authors.h"
#include "contributors.h"
#include "sql_partition.h"
......@@ -8423,6 +8425,66 @@ TABLE *create_schema_table(THD *thd, TABLE_LIST *table_list)
}
#ifdef HAVE_REPLICATION
int fill_slave_status(THD *thd, TABLE_LIST *tables, COND *cond)
{
String gtid_pos;
Master_info **tmp;
TABLE *table= tables->table;
uint elements, i;
DBUG_ENTER("show_all_master_info");
if (check_global_access(thd, PRIV_STMT_SHOW_SLAVE_STATUS))
DBUG_RETURN(TRUE);
gtid_pos.length(0);
if (rpl_append_gtid_state(&gtid_pos, true))
{
my_error(ER_OUT_OF_RESOURCES, MYF(0));
DBUG_RETURN(TRUE);
}
if (!master_info_index ||
!(elements= master_info_index->master_info_hash.records))
{
/* No registered slaves */
return 0;
}
mysql_mutex_lock(&LOCK_active_mi);
/*
Sort lines to get them into a predicted order
(needed for test cases and to not confuse users)
*/
if (!(tmp= (Master_info**) thd->alloc(sizeof(Master_info*) * elements)))
goto error;
for (i= 0; i < elements; i++)
{
tmp[i]= (Master_info *) my_hash_element(&master_info_index->
master_info_hash, i);
}
my_qsort(tmp, elements, sizeof(Master_info*), (qsort_cmp) cmp_mi_by_name);
for (i= 0; i < elements; i++)
{
if (tmp[i]->host[0])
{
store_master_info(thd, tmp[i], table, &gtid_pos);
if (schema_table_store_record(thd, table))
goto error;
}
}
mysql_mutex_unlock(&LOCK_active_mi);
DBUG_RETURN(0);
error:
mysql_mutex_unlock(&LOCK_active_mi);
DBUG_RETURN(1);
}
#endif
/*
For old SHOW compatibility. It is used when
old SHOW doesn't have generated column names
......@@ -9730,10 +9792,7 @@ ST_FIELD_INFO files_fields_info[]=
void init_fill_schema_files_row(TABLE* table)
{
int i;
for(i=0; !Show::files_fields_info[i].end_marker(); i++)
table->field[i]->set_null();
table->set_null_bits();
table->field[IS_FILES_STATUS]->set_notnull();
table->field[IS_FILES_STATUS]->store("NORMAL", 6, system_charset_info);
}
......@@ -9841,8 +9900,76 @@ ST_FIELD_INFO check_constraints_fields_info[]=
CEnd()
};
}; // namespace Show
ST_FIELD_INFO slave_status_info[]=
{
Column("Connection_name", Name(), NOT_NULL),
Column("Slave_SQL_State", Varchar(64), NULLABLE),
Column("Slave_IO_State", Varchar(64), NULLABLE),
Column("Master_Host", Varchar(HOSTNAME_LENGTH), NULLABLE),
Column("Master_User", Varchar(USERNAME_LENGTH), NULLABLE),
Column("Master_Port", ULong(7), NOT_NULL),
Column("Connect_Retry", SLong(10), NOT_NULL),
Column("Master_Log_File", Varchar(FN_REFLEN), NOT_NULL),
Column("Read_Master_Log_Pos", ULonglong(10), NOT_NULL),
Column("Relay_Log_File", Varchar(FN_REFLEN), NOT_NULL),
Column("Relay_Log_Pos", ULonglong(10), NOT_NULL),
Column("Relay_Master_Log_File", Varchar(FN_REFLEN), NOT_NULL),
Column("Slave_IO_Running", Varchar(3), NOT_NULL),
Column("Slave_SQL_Running", Varchar(3), NOT_NULL),
Column("Replicate_Do_DB", Name(), NOT_NULL),
Column("Replicate_Ignore_DB", Name(), NOT_NULL),
Column("Replicate_Do_Table", Name(), NOT_NULL),
Column("Replicate_Ignore_Table", Name(), NOT_NULL),
Column("Replicate_Wild_Do_Table", Name(), NOT_NULL),
Column("Replicate_Wild_Ignore_Table", Name(), NOT_NULL),
Column("Last_Errno", SLong(4), NOT_NULL),
Column("Last_Error", Varchar(20), NULLABLE),
Column("Skip_Counter", ULong(10), NOT_NULL),
Column("Exec_Master_Log_Pos", ULonglong(10), NOT_NULL),
Column("Relay_Log_Space", ULonglong(10), NOT_NULL),
Column("Until_Condition", Varchar(6), NOT_NULL),
Column("Until_Log_File", Varchar(FN_REFLEN), NULLABLE),
Column("Until_Log_Pos", ULonglong(10), NOT_NULL),
Column("Master_SSL_Allowed", Varchar(7), NULLABLE),
Column("Master_SSL_CA_File", Varchar(FN_REFLEN), NULLABLE),
Column("Master_SSL_CA_Path", Varchar(FN_REFLEN), NULLABLE),
Column("Master_SSL_Cert", Varchar(FN_REFLEN), NULLABLE),
Column("Master_SSL_Cipher", Varchar(FN_REFLEN), NULLABLE),
Column("Master_SSL_Key", Varchar(FN_REFLEN), NULLABLE),
Column("Seconds_Behind_Master", SLonglong(10), NULLABLE),
Column("Master_SSL_Verify_Server_Cert", Varchar(3), NOT_NULL),
Column("Last_IO_Errno", SLong(4), NOT_NULL),
Column("Last_IO_Error", Varchar(20), NULLABLE),
Column("Last_SQL_Errno", SLong(4), NOT_NULL),
Column("Last_SQL_Error", Varchar(20), NULLABLE),
Column("Replicate_Ignore_Server_Ids", Varchar(FN_REFLEN), NOT_NULL),
Column("Master_Server_Id", ULong(10), NOT_NULL),
Column("Master_SSL_Crl", Varchar(FN_REFLEN), NULLABLE),
Column("Master_SSL_Crlpath", Varchar(FN_REFLEN), NULLABLE),
Column("Using_Gtid", Varchar(15), NULLABLE),
Column("Gtid_IO_Pos", Varchar(30), NOT_NULL),
Column("Replicate_Do_Domain_Ids", Varchar(FN_REFLEN), NOT_NULL),
Column("Replicate_Ignore_Domain_Ids", Varchar(FN_REFLEN), NOT_NULL),
Column("Parallel_Mode", Varchar(15), NOT_NULL),
Column("SQL_Delay", ULong(10), NOT_NULL),
Column("SQL_Remaining_Delay", ULong(10), NULLABLE),
Column("Slave_SQL_Running_State", Varchar(64), NULLABLE),
Column("Slave_DDL_Groups", ULonglong(20), NOT_NULL),
Column("Slave_Non_Transactional_Groups", ULonglong(20), NOT_NULL),
Column("Slave_Transactional_Groups", ULonglong(20), NOT_NULL),
Column("Retried_transactions", ULong(10), NOT_NULL),
Column("Max_relay_log_size", ULonglong(10), NOT_NULL),
Column("Executed_log_entries", ULong(10), NOT_NULL),
Column("Slave_received_heartbeats", ULong(10), NOT_NULL),
Column("Slave_heartbeat_period", Float(310), NOT_NULL), // 3 decimals
Column("Gtid_Slave_Pos", Varchar(FN_REFLEN), NOT_NULL),
Column("Master_last_event_time", Datetime(0), NULLABLE),
Column("Slave_state_time", Datetime(0), NULLABLE),
Column("Master_Slave_time_diff", SLonglong(10), NULLABLE),
CEnd()
};
}; // namespace Show
namespace Show {
......@@ -9960,6 +10087,10 @@ ST_SCHEMA_TABLE schema_tables[]=
{"VIEWS", Show::view_fields_info, 0,
get_all_tables, 0, get_schema_views_record, 1, 2, 0,
OPEN_VIEW_ONLY|OPTIMIZE_I_S_TABLE},
#ifdef HAVE_REPLICATION
{"SLAVE_STATUS", Show::slave_status_info, 0,
fill_slave_status, make_old_format, 0, 1, 0, 0, 0 },
#endif
{0, 0, 0, 0, 0, 0, 0, 0, 0, 0}
};
......
......@@ -1560,6 +1560,16 @@ struct TABLE
bool fill_item_list(List<Item> *item_list) const;
void reset_item_list(List<Item> *item_list, uint skip) const;
void clear_column_bitmaps(void);
inline void clear_null_bits()
{
if (s->null_bytes)
bzero(null_flags, s->null_bytes);
}
inline void set_null_bits()
{
if (s->null_bytes)
bfill(null_flags, s->null_bytes, 255);
}
void prepare_for_position(void);
MY_BITMAP *prepare_for_keyread(uint index, MY_BITMAP *map);
MY_BITMAP *prepare_for_keyread(uint index)
......@@ -3314,8 +3324,7 @@ inline void mark_as_null_row(TABLE *table)
{
table->null_row=1;
table->status|=STATUS_NULL_ROW;
if (table->s->null_bytes)
bfill(table->null_flags,table->s->null_bytes,255);
table->set_null_bits();
}
/*
......
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