Commit cec8ac3e authored by unknown's avatar unknown

WL#3071 Maria checkpoint

Finally this is the real checkpoint code.
It however exhibits unstabilities when a checkpoint runs concurrently
with data-modifying clients (table corruption, transaction log's
assertions) so for now a checkpoint is taken only at startup after
recovery and at shutdown, i.e. not in concurrent situations. Later
we will let it run periodically, as well as flush dirty pages
periodically (almost all needed code is there already, only pagecache
code is written but not committed).
WL#3072 Maria recovery
* replacing UNDO_ROW_PURGE with CLR_END; testing of those CLR_END via
ma_test2 which has INSERTs failing with duplicate keys.
* replaying of REDO_RENAME_TABLE
Now, off to test Recovery in ha_maria :)


BitKeeper/deleted/.del-ma_least_recently_dirtied.c:
  Delete: storage/maria/ma_least_recently_dirtied.c
BitKeeper/deleted/.del-ma_least_recently_dirtied.h:
  Delete: storage/maria/ma_least_recently_dirtied.h
storage/maria/Makefile.am:
  compile Checkpoint module
storage/maria/ha_maria.cc:
  When ha_maria starts, do a recovery from last checkpoint.
  Take a checkpoint when that recovery has ended and when ha_maria
  shuts down cleanly.
storage/maria/ma_blockrec.c:
  * even if my_sync() fails we have to my_close() (otherwise we leak
  a descriptor)
  * UNDO_ROW_PURGE is replaced by a simple CLR_END for UNDO_ROW_INSERT,
  as promised in the old comment; it gives us skipping during the
  UNDO phase.
storage/maria/ma_check.c:
  All REDOs before create_rename_lsn are ignored by Recovery. So
  create_rename_lsn must be set only after all data/index has been
  flushed and forced to disk. We thus move write_log_record_for_repair()
  to after _ma_flush_tables_files_after_repair().
storage/maria/ma_checkpoint.c:
  Checkpoint module.
storage/maria/ma_checkpoint.h:
  optional argument if caller wants a thread to periodically take
  checkpoints and flush dirty pages.
storage/maria/ma_create.c:
  * no need to init some vars as the initial bzero(share) takes care of this.
  * update to new function's name
  * even if we fail in my_sync() we have to my_close()
storage/maria/ma_extra.c:
  Checkpoint reads share->last_version under intern_lock, so we make
  maria_extra() update it under intern_lock. THR_LOCK_maria still needed
  because of _ma_test_if_reopen().
storage/maria/ma_init.c:
  destroy checkpoint module when Maria shuts down.
storage/maria/ma_loghandler.c:
  * UNDO_ROW_PURGE gone (see ma_blockrec.c)
  * we need to remember the LSN of the LOGREC_FILE_ID for a share,
  because this LSN is needed into the checkpoint record (Recovery wants
  to know the validity domain of an id->name mapping)
  * translog_get_horizon_no_lock() needed for Checkpoint
  * comment about failing assertion (Sanja knows)
  * translog_init_reader_data() thought that translog_read_record_header_scan()
  returns 0 in case of error, but 0 just means "0-length header".
  * translog_assign_id_to_share() now needs the MARIA_HA because
  LOGREC_FILE_ID uses a log-write hook.
  * Verify that (de)assignment of share->id happens only under intern_lock,
  as Checkpoint reads this id with intern_lock.
  * translog_purge() can accept TRANSLOG_ADDRESS, not necessarily
  a real LSN.
storage/maria/ma_loghandler.h:
  prototype updates
storage/maria/ma_open.c:
  no need to initialize "res"
storage/maria/ma_pagecache.c:
  When taking a checkpoint, we don't need to know the maximum rec_lsn
  of dirty pages; this LSN was intended to be used in the two-checkpoint
  rule, but last_checkpoint_lsn is as good.
  4 bytes for stored_list_size is enough as PAGECACHE::blocks (number
  of blocks which the pagecache can contain) is int.
storage/maria/ma_pagecache.h:
  new prototype
storage/maria/ma_recovery.c:
  * added replaying of REDO_RENAME_TABLE
  * UNDO_ROW_PURGE gone (see ma_blockrec.c), replaced by CLR_END
  * Recovery from the last checkpoint record now possible
  * In new_table() we skip the table if the id->name mapping is older than
  create_rename_lsn (mapping dates from lsn_of_file_id).
  * in get_MARIA_HA_from_REDO_record() we skip the record
  if the id->name mapping is newer than the record (can happen if processing
  a record which is before the checkpoint record).
  * parse_checkpoint_record() has to return a LSN, that's what caller expects
storage/maria/ma_rename.c:
  new function's name; log end zeroes of tables' names (ease recovery)
storage/maria/ma_test2.c:
  * equivalent of ma_test1's --test-undo added (named -u here).
  * -t=1 now stops right after creating the table, so that
  we can test undoing of INSERTs with duplicate keys (which tests the
  CLR_END logged by _ma_write_abort_block_record()).
storage/maria/ma_test_recovery.expected:
  Result of testing undoing of INSERTs with duplicate keys; there are
  some differences in maria_chk -dvv but they are normal (removing
  records does not shrink data/index file, does not put back the
  "analyzed, optimized keys"(etc) index state.
storage/maria/ma_test_recovery:
  Test undoing of INSERTs with duplicate keys, using ma_test2;
  when such INSERT happens, it logs REDO_INSERT, UNDO_INSERT, REDO_DELETE,
  CLR_END; we abort after that, and test that CLR_END causes recovery
  to jump over UNDO_INSERT.
storage/maria/ma_write.c:
  comment
storage/maria/maria_chk.c:
  comment
storage/maria/maria_def.h:
  * a new bit in MARIA_SHARE::in_checkpoint, used to build a list
  of unique shares during Checkpoint.
  * MARIA_SHARE::lsn_of_file_id added: the LSN of the last LOGREC_FILE_ID
  for this share; needed to know to which LSN domain the mappings
  found in the Checkpoint record apply (new mappings should not apply
  to old REDOs).
storage/maria/trnman.c:
  * small changes to how trnman_collect_transactions() fills its buffer;
  it also uses a non-dummy lsn_read_non_atomic() found in ma_checkpoint.h
parent cf9b8d33
......@@ -61,7 +61,8 @@ noinst_HEADERS = maria_def.h ma_rt_index.h ma_rt_key.h ma_rt_mbr.h \
ma_ft_eval.h trnman.h lockman.h tablockman.h \
ma_control_file.h ha_maria.h ma_blockrec.h \
ma_loghandler.h ma_loghandler_lsn.h ma_pagecache.h \
ma_recovery.h ma_commit.h trnman_public.h
ma_checkpoint.h ma_recovery.h ma_commit.h \
trnman_public.h
ma_test1_DEPENDENCIES= $(LIBRARIES)
ma_test1_LDADD= @CLIENT_EXTRA_LDFLAGS@ libmaria.a \
$(top_builddir)/storage/myisam/libmyisam.a \
......@@ -120,7 +121,7 @@ libmaria_a_SOURCES = ma_init.c ma_open.c ma_extra.c ma_info.c ma_rkey.c \
ma_rt_index.c ma_rt_key.c ma_rt_mbr.c ma_rt_split.c \
ma_sp_key.c ma_control_file.c ma_loghandler.c \
ma_pagecache.c ma_pagecaches.c \
ma_recovery.c ma_commit.c
ma_checkpoint.c ma_recovery.c ma_commit.c
CLEANFILES = test?.MA? FT?.MA? isam.log ma_test_all ma_rt_test.MA? sp_test.MA?
SUFFIXES = .sh
......
......@@ -31,6 +31,8 @@ C_MODE_START
#include "maria_def.h"
#include "ma_rt_index.h"
#include "ma_blockrec.h"
#include "ma_checkpoint.h"
#include "ma_recovery.h"
C_MODE_END
/*
......@@ -2344,6 +2346,7 @@ bool ha_maria::check_if_incompatible_data(HA_CREATE_INFO *info,
static int maria_hton_panic(handlerton *hton, ha_panic_function flag)
{
ma_checkpoint_execute(CHECKPOINT_FULL, FALSE); /* can't catch error */
return maria_panic(flag);
}
......@@ -2403,7 +2406,10 @@ static int ha_maria_init(void *p)
translog_init(maria_data_root, TRANSLOG_FILE_SIZE,
MYSQL_VERSION_ID, server_id, maria_log_pagecache,
TRANSLOG_DEFAULT_FLAGS) ||
trnman_init(0);
maria_recover() ||
ma_checkpoint_init(FALSE) ||
/* One checkpoint after Recovery */
ma_checkpoint_execute(CHECKPOINT_FULL, FALSE);
maria_multi_threaded= TRUE;
return res;
}
......
......@@ -402,9 +402,10 @@ my_bool _ma_once_end_block_record(MARIA_SHARE *share)
File must be synced as it is going out of the maria_open_list and so
becoming unknown to Checkpoint.
*/
if ((share->now_transactional &&
my_sync(share->bitmap.file.file, MYF(MY_WME))) ||
my_close(share->bitmap.file.file, MYF(MY_WME)))
if (share->now_transactional &&
my_sync(share->bitmap.file.file, MYF(MY_WME)))
res= 1;
if (my_close(share->bitmap.file.file, MYF(MY_WME)))
res= 1;
/*
Trivial assignment to guard against multiple invocations
......@@ -2587,7 +2588,8 @@ my_bool _ma_write_abort_block_record(MARIA_HA *info)
my_bool res= 0;
MARIA_BITMAP_BLOCKS *blocks= &info->cur_row.insert_blocks;
MARIA_BITMAP_BLOCK *block, *end;
DBUG_ENTER("_ma_abort_write_block_record");
LSN lsn= LSN_IMPOSSIBLE;
DBUG_ENTER("_ma_write_abort_block_record");
if (delete_head_or_tail(info,
ma_recordpos_to_page(info->cur_row.lastpos),
......@@ -2616,44 +2618,42 @@ my_bool _ma_write_abort_block_record(MARIA_HA *info)
if (info->s->now_transactional)
{
LSN lsn;
LSN previous_undo_lsn;
TRANSLOG_HEADER_BUFFER rec;
LEX_STRING log_array[TRANSLOG_INTERNAL_PARTS + 1];
uchar log_data[LSN_STORE_SIZE + FILEID_STORE_SIZE];
uchar log_data[LSN_STORE_SIZE + FILEID_STORE_SIZE + 1];
int len;
/*
Write UNDO record
This entry is just an end marker for the abort_insert as we will never
really undo a failed insert. Note that this UNDO will cause recover
to ignore the LOGREC_UNDO_ROW_INSERT that is the previous entry
in the UNDO chain.
*/
/**
@todo RECOVERY BUG
We do need the code above (delete_head_or_tail() etc) for
non-transactional tables.
For transactional tables we can either also use it or execute the
UNDO_INSERT. If we crash before this
_ma_write_abort_block_record(), Recovery will do the work of this
function by executing UNDO_INSERT.
For transactional tables, we will remove this LOGREC_UNDO_PURGE and
replace it with a LOGREC_CLR_END: we should go back the UNDO chain
until we reach the UNDO which inserted the row into the data file, and
use its previous_undo_lsn.
Same logic for when we remove inserted keys (in case of error in
maria_write(): we come to the present function only after removing the
inserted keys... as long as we unpin the key pages only after writing
the CLR_END, this would be recovery-safe...).
For transactional tables we could skip this code above and just execute
the UNDO_INSERT, but we try to have one code path.
Write CLR record, because we are somehow undoing UNDO_ROW_INSERT.
When we have logging for keys: as maria_write() first writes the row
then the keys, and if failure, deletes the keys then the rows,
info->trn->undo_lsn below will properly point to the UNDO of the
UNDO_ROW_INSERT for this row.
*/
lsn_store(log_data, info->trn->undo_lsn);
if ((len= translog_read_record_header(info->trn->undo_lsn, &rec)) ==
RECHEADER_READ_ERROR)
{
res= 1;
goto end;
}
DBUG_ASSERT(rec.type == LOGREC_UNDO_ROW_INSERT);
previous_undo_lsn= lsn_korr(rec.header);
lsn_store(log_data, previous_undo_lsn);
log_data[LSN_STORE_SIZE + FILEID_STORE_SIZE]= LOGREC_UNDO_ROW_INSERT;
log_array[TRANSLOG_INTERNAL_PARTS + 0].str= (char*) log_data;
log_array[TRANSLOG_INTERNAL_PARTS + 0].length= sizeof(log_data);
if (translog_write_record(&lsn, LOGREC_UNDO_ROW_PURGE,
if (translog_write_record(&lsn, LOGREC_CLR_END,
info->trn, info, sizeof(log_data),
TRANSLOG_INTERNAL_PARTS + 1, log_array,
log_data + LSN_STORE_SIZE))
res= 1;
}
_ma_unpin_all_pages_and_finalize_row(info, info->trn->undo_lsn);
end:
_ma_unpin_all_pages_and_finalize_row(info, lsn);
DBUG_RETURN(res);
}
......
......@@ -2264,7 +2264,7 @@ int maria_repair(HA_CHECK *param, register MARIA_HA *info,
llstr(sort_info.dupp,llbuff));
}
got_error= sync_dir ? write_log_record_for_repair(param, info) : 0;
got_error= 0;
/* If invoked by external program that uses thr_lock */
if (&share->state.state != info->state)
memcpy( &share->state.state, info->state, sizeof(*info->state));
......@@ -2309,6 +2309,14 @@ err:
}
maria_mark_crashed_on_repair(info);
}
else if (sync_dir)
{
/*
Now that we have flushed and forced everything, we can bump
create_rename_lsn:
*/
write_log_record_for_repair(param, info);
}
my_free(sort_param.rec_buff, MYF(MY_ALLOW_ZERO_PTR));
my_free(sort_param.record,MYF(MY_ALLOW_ZERO_PTR));
my_free(sort_info.buff,MYF(MY_ALLOW_ZERO_PTR));
......@@ -5551,7 +5559,7 @@ read_next_page:
/**
@brief Writes a LOGREC_REPAIR_TABLE record and updates create_rename_lsn
and is_of_lsn
and is_of_horizon
REPAIR/OPTIMIZE have replaced the data/index file with a new file
and so, in this scenario:
......@@ -5572,6 +5580,7 @@ read_next_page:
static int write_log_record_for_repair(const HA_CHECK *param, MARIA_HA *info)
{
MARIA_SHARE *share= info->s;
/* in case this is maria_chk or recovery... */
if (translog_inited && !maria_in_recovery)
{
......@@ -5613,16 +5622,12 @@ static int write_log_record_for_repair(const HA_CHECK *param, MARIA_HA *info)
return 1;
/*
The table's existence was made durable earlier (MY_SYNC_DIR passed to
maria_change_to_newfile()).
maria_change_to_newfile()). _ma_flush_table_files_after_repair() was
called earlier, flushed and forced data+index+state. Old REDOs should
not be applied to the table:
*/
if (_ma_update_create_rename_lsn_on_disk(info->s, lsn, FALSE))
if (_ma_update_create_rename_lsn(share, lsn, TRUE))
return 1;
/*
_ma_flush_table_files_after_repair() is later called by maria_repair(),
and makes sure to flush the data, index, update is_of_lsn, flush state
and sync, so create_rename_lsn reaches disk, thus we won't apply old
REDOs to the new table.
*/
}
return 0;
}
/* Copyright (C) 2006 MySQL AB & MySQL Finland AB & TCX DataKonsult AB
/* Copyright (C) 2006,2007 MySQL AB
This program is free software; you can redistribute it and/or modify
it under the terms of the GNU General Public License as published by
......@@ -16,413 +16,1084 @@
/*
WL#3071 Maria checkpoint
First version written by Guilhem Bichot on 2006-04-27.
Does not compile yet.
*/
/* Here is the implementation of this module */
/**
@todo RECOVERY BUG this is unreviewed code, but used in safe conditions:
ha_maria takes a checkpoint at end of recovery and one at clean shutdown,
that's all. So there never are open tables, dirty pages, transactions.
*/
/*
Summary:
- there are asynchronous checkpoints (a writer to the log notices that it's
been a long time since we last checkpoint-ed, so posts a request for a
background thread to do a checkpoint; does not care about the success of the
checkpoint). Then the checkpoint is done by the checkpoint thread, at an
unspecified moment ("later") (==soon, of course).
- there are synchronous checkpoints: a thread requests a checkpoint to
happen now and wants to know when it finishes and if it succeeded; then the
checkpoint is done by that same thread.
checkpoints are done either by a background thread (checkpoint every Nth
second) or by a client.
In ha_maria, it's not made available to clients, and will soon be done by a
background thread (periodically taking checkpoints and flushing dirty
pages).
*/
#include "page_cache.h"
#include "least_recently_dirtied.h"
#include "transaction.h"
#include "share.h"
#include "log.h"
#include "maria_def.h"
#include "ma_pagecache.h"
#include "trnman.h"
#include "ma_blockrec.h"
#include "ma_checkpoint.h"
#include "ma_loghandler_lsn.h"
#define LSN_MAX ((LSN)ULONGLONG_MAX)
/*
this transaction is used for any system work (purge, checkpoint writing
etc), that is, background threads. It will not be declared/initialized here
in the final version.
*/
st_transaction system_trans= {0 /* long trans id */, 0 /* short trans id */,0,...};
/* those three are protected by the log's mutex */
/*
The maximum rec_lsn in the LRD when last checkpoint was run, serves for the
MEDIUM checkpoint.
Checkpoints currently happen only at ha_maria's startup (after recovery) and
at shutdown, always when there is no open tables.
Background page flushing is not used.
So, needed pagecache functions for doing this flushing are not yet pushed.
*/
LSN max_rec_lsn_at_last_checkpoint= 0;
/* last submitted checkpoint request; cleared when starts */
CHECKPOINT_LEVEL next_asynchronous_checkpoint_to_do= NONE;
CHECKPOINT_LEVEL checkpoint_in_progress= NONE;
static inline ulonglong read_non_atomic(ulonglong volatile *x);
/*
Used by MySQL client threads requesting a checkpoint (like "ALTER MARIA
ENGINE DO CHECKPOINT"), and probably by maria_panic(), and at the end of the
UNDO recovery phase.
#define flush_pagecache_blocks_with_filter(A,B,C,D,E) (((int)D) * 0)
/**
filter has to return 0, 1 or 2: 0 means "don't flush this page", 1 means
"flush it", 2 means "don't flush this page and following pages".
Will move to ma_pagecache.h
*/
my_bool execute_synchronous_checkpoint(CHECKPOINT_LEVEL level)
{
my_bool result;
DBUG_ENTER("execute_synchronous_checkpoint");
DBUG_ASSERT(level > NONE);
typedef int (*PAGECACHE_FILTER)(enum pagecache_page_type type,
pgcache_page_no_t page,
LSN rec_lsn, void *arg);
lock(log_mutex);
while (checkpoint_in_progress != NONE)
wait_on_checkpoint_done_cond();
result= execute_checkpoint(level);
DBUG_RETURN(result);
}
/** @brief type of checkpoint currently running */
static CHECKPOINT_LEVEL checkpoint_in_progress= CHECKPOINT_NONE;
/** @brief protects checkpoint_in_progress */
static pthread_mutex_t LOCK_checkpoint;
/** @brief for killing the background checkpoint thread */
static pthread_cond_t COND_checkpoint;
/** @brief if checkpoint module was inited or not */
static my_bool checkpoint_inited= FALSE;
/** @brief 'kill' flag for the background checkpoint thread */
static int checkpoint_thread_die;
/* is ulong like pagecache->blocks_changed */
static ulong pages_to_flush_before_next_checkpoint;
static PAGECACHE_FILE *dfiles, /**< data files to flush in background */
*dfiles_end; /**< list of data files ends here */
static PAGECACHE_FILE *kfiles, /**< index files to flush in background */
*kfiles_end; /**< list of index files ends here */
/* those two statistics below could serve in SHOW GLOBAL STATUS */
static uint checkpoints_total= 0, /**< all checkpoint requests made */
checkpoints_ok_total= 0; /**< all checkpoints which succeeded */
/*
If no checkpoint is running, and there is a pending asynchronous checkpoint
request, executes it.
Is safe if multiple threads call it, though in first version only one will.
It's intended to be used by a thread which regularly calls this function;
this is why, if there is a request, it does not wait in a loop for
synchronous checkpoints to be finished, but just exits (because the thread
may want to do something useful meanwhile (flushing dirty pages for example)
instead of waiting).
*/
my_bool execute_asynchronous_checkpoint_if_any()
struct st_filter_param
{
my_bool result;
CHECKPOINT_LEVEL level;
DBUG_ENTER("execute_asynchronous_checkpoint");
my_bool is_data_file; /**< is the file about data or index */
LSN up_to_lsn; /**< only pages with rec_lsn < this LSN */
ulong pages_covered_by_bitmap; /**< to know which page is a bitmap page */
uint max_pages; /**< stop after flushing this number pages */
}; /**< information to determine which dirty pages should be flushed */
/* first check without mutex, ok to see old data */
if (likely((next_asynchronous_checkpoint_to_do == NONE) ||
(checkpoint_in_progress != NONE)))
DBUG_RETURN(FALSE);
static int filter_flush_data_file_medium(enum pagecache_page_type type,
pgcache_page_no_t page,
LSN rec_lsn, void *arg);
static int filter_flush_data_file_full(enum pagecache_page_type type,
pgcache_page_no_t page,
LSN rec_lsn, void *arg);
static int filter_flush_data_file_indirect(enum pagecache_page_type type,
pgcache_page_no_t page,
LSN rec_lsn, void *arg);
static int filter_flush_data_file_evenly(enum pagecache_page_type type,
pgcache_page_no_t pageno,
LSN rec_lsn, void *arg);
static int really_execute_checkpoint();
pthread_handler_t ma_checkpoint_background(void *arg);
static int collect_tables();
lock(log_mutex);
if (likely((next_asynchronous_checkpoint_to_do == NONE) ||
(checkpoint_in_progress != NONE)))
{
unlock(log_mutex);
DBUG_RETURN(FALSE);
}
/**
@brief Does a checkpoint
result= execute_checkpoint(next_asynchronous_checkpoint_to_do);
DBUG_RETURN(result);
}
@param level what level of checkpoint to do
@param no_wait if another checkpoint of same or stronger level
is already running, consider our job done
@note In ha_maria, there can never be two threads trying a checkpoint at
the same time.
/*
Does the actual checkpointing. Called by
execute_synchronous_checkpoint() and
execute_asynchronous_checkpoint_if_any().
@return Operation status
@retval 0 ok
@retval !=0 error
*/
my_bool execute_checkpoint(CHECKPOINT_LEVEL level)
int ma_checkpoint_execute(CHECKPOINT_LEVEL level, my_bool no_wait)
{
my_bool result;
DBUG_ENTER("execute_checkpoint");
int result= 0;
DBUG_ENTER("ma_checkpoint_execute");
safemutex_assert_owner(log_mutex);
if (next_asynchronous_checkpoint_to_do <= level)
next_asynchronous_checkpoint_to_do= NONE;
checkpoint_in_progress= level;
DBUG_ASSERT(checkpoint_inited);
DBUG_ASSERT(level > CHECKPOINT_NONE);
if (unlikely(level > INDIRECT))
/* look for already running checkpoints */
pthread_mutex_lock(&LOCK_checkpoint);
while (checkpoint_in_progress != CHECKPOINT_NONE)
{
LSN copy_of_max_rec_lsn_at_last_checkpoint=
max_rec_lsn_at_last_checkpoint;
/* much I/O work to do, release log mutex */
unlock(log_mutex);
switch (level)
if (no_wait && (checkpoint_in_progress >= level))
{
case FULL:
/* flush all pages up to the current end of the LRD */
flush_all_LRD_to_lsn(LSN_MAX);
/* this will go full speed (normal scheduling, no sleep) */
break;
case MEDIUM:
/*
flush all pages which were already dirty at last checkpoint:
ensures that recovery will never start from before the next-to-last
checkpoint (two-checkpoint rule).
If we are the checkpoint background thread, we don't wait (it's
smarter to flush pages instead of waiting here while the other thread
finishes its checkpoint).
*/
flush_all_LRD_to_lsn(copy_of_max_rec_lsn_at_last_checkpoint);
/* this will go full speed (normal scheduling, no sleep) */
break;
pthread_mutex_unlock(&LOCK_checkpoint);
goto end;
}
lock(log_mutex);
pthread_cond_wait(&COND_checkpoint, &LOCK_checkpoint);
}
result= execute_checkpoint_indirect();
checkpoint_in_progress= NONE;
unlock(log_mutex);
broadcast(checkpoint_done_cond);
checkpoint_in_progress= level;
pthread_mutex_unlock(&LOCK_checkpoint);
/* from then on, we are sure to be and stay the only checkpointer */
result= really_execute_checkpoint();
pthread_cond_broadcast(&COND_checkpoint);
end:
DBUG_RETURN(result);
}
/*
Does an indirect checpoint (collects data from data structures, writes into
a checkpoint log record).
Starts and ends while having log's mutex (released in the middle).
/**
@brief Does a checkpoint, really; expects no other checkpoints
running.
Checkpoint level requested is read from checkpoint_in_progress.
@return Operation status
@retval 0 ok
@retval !=0 error
*/
my_bool execute_checkpoint_indirect()
static int really_execute_checkpoint()
{
int error= 0, i;
/* checkpoint record data: */
LSN checkpoint_start_lsn;
char checkpoint_start_lsn_char[8];
LEX_STRING strings[6]=
{checkpoint_start_lsn_char, 8}, {0,0}, {0,0}, {0,0}, {0,0}, {0,0} };
uint i, error= 0;
/** @brief checkpoint_start_log_horizon will be stored there */
char *ptr;
LSN checkpoint_lsn;
LSN candidate_max_rec_lsn_at_last_checkpoint;
DBUG_ENTER("execute_checkpoint_indirect");
LEX_STRING record_pieces[4]; /**< only malloc-ed pieces */
LSN min_page_rec_lsn, min_trn_rec_lsn, min_first_undo_lsn;
TRANSLOG_ADDRESS checkpoint_start_log_horizon;
uchar checkpoint_start_log_horizon_char[LSN_STORE_SIZE];
DBUG_ENTER("really_execute_checkpoint");
bzero(&record_pieces, sizeof(record_pieces));
DBUG_ASSERT(sizeof(uchar *) <= 8);
DBUG_ASSERT(sizeof(LSN) <= 8);
/*
STEP 1: record current end-of-log position using log's lock. It is
critical for the correctness of Checkpoint (related to memory visibility
rules, the log's lock is a mutex).
"Horizon" is a lower bound of the LSN of the next log record.
*/
/**
@todo RECOVERY BUG
this is an horizon, but it is used as a LSN (REDO phase may start from
there! probably log handler would refuse to read then;
Sanja proposed to make a loghandler's function which finds the LSN after
this horizon.
*/
checkpoint_start_log_horizon= translog_get_horizon();
#define LSN_IN_HEX(L) (ulong)LSN_FILE_NO(L),(ulong)LSN_OFFSET(L)
DBUG_PRINT("info",("checkpoint_start_log_horizon (%lu,0x%lx)",
LSN_IN_HEX(checkpoint_start_log_horizon)));
lsn_store(checkpoint_start_log_horizon_char, checkpoint_start_log_horizon);
safemutex_assert_owner(log_mutex);
/* STEP 1: record current end-of-log LSN */
checkpoint_start_lsn= log_read_end_lsn();
if (LSN_IMPOSSIBLE == checkpoint_start_lsn) /* error */
DBUG_RETURN(TRUE);
unlock(log_mutex);
/*
STEP 2: fetch information about transactions.
We must fetch transactions before dirty pages. Indeed, a transaction
first sets its rec_lsn then sets the page's rec_lsn then sets its rec_lsn
to 0. If we fetched pages first, we may see no dirty page yet, then we
fetch transactions but the transaction has already reset its rec_lsn to 0
so we miss rec_lsn again.
For a similar reason (over-allocated bitmap pages) we have to fetch
transactions before flushing bitmap pages.
DBUG_PRINT("info",("checkpoint_start_lsn %lu", checkpoint_start_lsn));
int8store(strings[0].str, checkpoint_start_lsn);
min_trn_rec_lsn will serve to lower the starting point of the REDO phase
(down from checkpoint_start_log_horizon).
*/
if (unlikely(trnman_collect_transactions(&record_pieces[0],
&record_pieces[1],
&min_trn_rec_lsn,
&min_first_undo_lsn)))
goto err;
/* STEP 2: fetch information about dirty pages */
if (pagecache_collect_changed_blocks_with_LSN(pagecache, &strings[1],
&candidate_max_rec_lsn_at_last_checkpoint))
/* STEP 3: fetch information about table files */
if (unlikely(collect_tables(&record_pieces[2])))
goto err;
/* STEP 3: fetch information about transactions */
if (trnman_collect_transactions(&strings[2], &strings[3]))
/* STEP 4: fetch information about dirty pages */
/*
It's better to do it _after_ having flushed some data pages (which
collect_tables() may have done), because those are now non-dirty and so we
have a more up-to-date dirty pages list to put into the checkpoint record,
and thus we will have less work at Recovery.
*/
/* Using default pagecache for now */
if (unlikely(pagecache_collect_changed_blocks_with_lsn(maria_pagecache,
&record_pieces[3],
&min_page_rec_lsn)))
goto err;
/* STEP 4: fetch information about table files */
/* LAST STEP: now write the checkpoint log record */
{
/* This global mutex is in fact THR_LOCK_maria (see ma_open()) */
lock(global_share_list_mutex);
strings[4].length= 8+(8+8)*share_list->count;
if (NULL == (strings[4].str= my_malloc(strings[4].length)))
goto err;
ptr= string3.str;
LSN lsn;
uint total_rec_length;
/*
Note that maria_open_list is a list of MARIA_HA*, while we would prefer
a list of MARIA_SHARE* here (we are interested in the short id,
unique file name, members of MARIA_SHARE*, and in file descriptors,
which will in the end be in MARIA_SHARE*).
the log handler is allowed to modify "str" and "length" (but not "*str")
of its argument, so we must not pass it record_pieces directly,
otherwise we would later not know what memory pieces to my_free().
*/
for (iterate on the maria_open_list)
LEX_STRING log_array[TRANSLOG_INTERNAL_PARTS + 5];
log_array[TRANSLOG_INTERNAL_PARTS + 0].str=
checkpoint_start_log_horizon_char;
log_array[TRANSLOG_INTERNAL_PARTS + 0].length= total_rec_length=
sizeof(checkpoint_start_log_horizon_char);
for (i= 0; i < (sizeof(record_pieces)/sizeof(record_pieces[0])); i++)
{
/* latch each MARIA_SHARE, one by one, like this: */
pthread_mutex_lock(&share->intern_lock);
log_array[TRANSLOG_INTERNAL_PARTS + 1 + i]= record_pieces[i];
total_rec_length+= record_pieces[i].length;
}
if (unlikely(translog_write_record(&lsn, LOGREC_CHECKPOINT,
&dummy_transaction_object, NULL,
total_rec_length,
sizeof(log_array)/sizeof(log_array[0]),
log_array, NULL) ||
translog_flush(lsn)))
goto err;
translog_lock();
/*
TODO:
we need to prevent the share from going away while we later flush and
force it without holding THR_LOCK_maria. For example if the share is
free()d by maria_close() we'll have a problem. Or if the share's file
descriptor is closed by maria_close() we will not be able to my_sync()
it.
This cannot be done as a inwrite_rec_hook of LOGREC_CHECKPOINT, because
such hook would be called before translog_flush (and we must be sure
that log was flushed before we write to the control file).
*/
pthread_mutex_unlock(&share->intern_lock);
store the share pointer into a private array;
if (unlikely(ma_control_file_write_and_force(lsn, FILENO_IMPOSSIBLE,
CONTROL_FILE_UPDATE_ONLY_LSN)))
{
translog_unlock();
goto err;
}
translog_unlock();
}
unlock(global_share_list_mutex);
/* work on copy */
int8store(ptr, elements_in_array);
ptr+= 8;
for (el in array)
{
int8store(ptr, array[...].short_id);
ptr+= 8;
memcpy(ptr, array[...].unique_file_name[_length], ...);
ptr+= ...;
/* maybe we need to lock share->intern_lock here */
/*
these two are long ops (involving disk I/O) that's why we copied the
list, to not keep the list locked for long:
Note that we should not alter memory structures until we have successfully
written the checkpoint record and control file.
*/
flush_bitmap_pages(el);
/* TODO: and also autoinc counter, logical file end, free page list */
/* checkpoint succeeded */
ptr= record_pieces[3].str;
pages_to_flush_before_next_checkpoint= uint4korr(ptr);
DBUG_PRINT("info",("%u pages to flush before next checkpoint",
(uint)pages_to_flush_before_next_checkpoint));
/*
fsyncs the fd, that's the loooong operation (e.g. max 150 fsync per
second, so if you have touched 1000 files it's 7 seconds).
/* compute log's low-water mark */
TRANSLOG_ADDRESS log_low_water_mark= min_page_rec_lsn;
set_if_smaller(log_low_water_mark, min_trn_rec_lsn);
set_if_smaller(log_low_water_mark, min_first_undo_lsn);
set_if_smaller(log_low_water_mark, checkpoint_start_log_horizon);
/**
Now purge unneeded logs.
As some systems have an unreliable fsync (drive lying), we could try to
be robust against that: remember a few previous checkpoints in the
control file, and not purge logs immediately... Think about it.
*/
force_file(el);
#if 0 /* purging/keeping will be an option */
if (translog_purge(log_low_water_mark))
fprintf(stderr, "Maria engine: log purge failed\n"); /* not deadly */
#endif
goto end;
err:
error= 1;
fprintf(stderr, "Maria engine: checkpoint failed\n"); /* TODO: improve ;) */
/* we were possibly not able to determine what pages to flush */
pages_to_flush_before_next_checkpoint= 0;
end:
for (i= 0; i < (sizeof(record_pieces)/sizeof(record_pieces[0])); i++)
my_free(record_pieces[i].str, MYF(MY_ALLOW_ZERO_PTR));
pthread_mutex_lock(&LOCK_checkpoint);
checkpoint_in_progress= CHECKPOINT_NONE;
checkpoints_total++;
checkpoints_ok_total+= !error;
pthread_mutex_unlock(&LOCK_checkpoint);
DBUG_RETURN(error);
}
/**
@brief Initializes the checkpoint module
@param create_background_thread If one wants the module to now create a
thread which will periodically do
checkpoints, and flush dirty pages, in the
background.
@return Operation status
@retval 0 ok
@retval !=0 error
*/
int ma_checkpoint_init(my_bool create_background_thread)
{
pthread_t th;
int res= 0;
DBUG_ENTER("ma_checkpoint_init");
checkpoint_inited= TRUE;
checkpoint_thread_die= 2; /* not yet born == dead */
if (pthread_mutex_init(&LOCK_checkpoint, MY_MUTEX_INIT_SLOW) ||
pthread_cond_init(&COND_checkpoint, 0))
res= 1;
else if (create_background_thread)
{
if (!(res= pthread_create(&th, NULL, ma_checkpoint_background, NULL)))
checkpoint_thread_die= 0; /* thread lives, will have to be killed */
}
DBUG_RETURN(res);
}
/**
@brief Destroys the checkpoint module
*/
void ma_checkpoint_end()
{
DBUG_ENTER("ma_checkpoint_end");
if (checkpoint_inited)
{
pthread_mutex_lock(&LOCK_checkpoint);
if (checkpoint_thread_die != 2) /* thread was started ok */
{
DBUG_PRINT("info",("killing Maria background checkpoint thread"));
checkpoint_thread_die= 1; /* kill it */
do /* and wait for it to be dead */
{
/* wake it up if it was in a sleep */
pthread_cond_broadcast(&COND_checkpoint);
DBUG_PRINT("info",("waiting for Maria background checkpoint thread"
" to die"));
pthread_cond_wait(&COND_checkpoint, &LOCK_checkpoint);
}
while (checkpoint_thread_die != 2);
}
pthread_mutex_unlock(&LOCK_checkpoint);
my_free((uchar *)dfiles, MYF(MY_ALLOW_ZERO_PTR));
my_free((uchar *)kfiles, MYF(MY_ALLOW_ZERO_PTR));
pthread_mutex_destroy(&LOCK_checkpoint);
pthread_cond_destroy(&COND_checkpoint);
checkpoint_inited= FALSE;
}
DBUG_VOID_RETURN;
}
/* LAST STEP: now write the checkpoint log record */
checkpoint_lsn= log_write_record(LOGREC_CHECKPOINT,
&system_trans, strings);
/**
@brief dirty-page filtering criteria for MEDIUM checkpoint.
/*
Do nothing between the log write and the control file write, for the
"repair control file" tool to be possible one day.
*/
We flush data/index pages which have been dirty since the previous
checkpoint (this is the two-checkpoint rule: the REDO phase will not have
to start from earlier than the next-to-last checkpoint), and all dirty
bitmap pages.
if (LSN_IMPOSSIBLE == checkpoint_lsn)
goto err;
@param type Page's type
@param pageno Page's number
@param rec_lsn Page's rec_lsn
@param arg filter_param
if (0 != control_file_write_and_force(checkpoint_lsn, NULL))
goto err;
@return Operation status
@retval 0 don't flush the page
@retval 1 flush the page
*/
/*
Note that we should not alter memory structures until we have successfully
written the checkpoint record and control file.
Btw, a log write failure is serious:
- if we know how many bytes we managed to write, we should try to write
more, keeping the log's mutex (MY_FULL_IO)
- if we don't know, this log record is corrupted and we have no way to
"de-corrupt" it, so it will stay corrupted, and as the log is sequential,
any log record written after it will not be reachable (for example if we
would write UNDOs and crash, we would not be able to read the log and so
not be able to rollback), so we should stop the engine now (holding the
log's mutex) and do a recovery.
*/
goto end;
static int filter_flush_data_file_medium(enum pagecache_page_type type,
pgcache_page_no_t pageno,
LSN rec_lsn, void *arg)
{
struct st_filter_param *param= (struct st_filter_param *)arg;
return ((type == PAGECACHE_LSN_PAGE) &&
(cmp_translog_addr(rec_lsn, param->up_to_lsn) <= 0)) ||
(param->is_data_file &&
((pageno % param->pages_covered_by_bitmap) == 0));
}
err:
print_error_to_error_log(the_error_message);
candidate_max_rec_lsn_at_last_checkpoint= LSN_IMPOSSIBLE;
end:
/**
@brief dirty-page filtering criteria for FULL checkpoint.
We flush all dirty data/index pages and all dirty bitmap pages.
@param type Page's type
@param pageno Page's number
@param rec_lsn Page's rec_lsn
@param arg filter_param
@return Operation status
@retval 0 don't flush the page
@retval 1 flush the page
*/
static int filter_flush_data_file_full(enum pagecache_page_type type,
pgcache_page_no_t pageno,
LSN rec_lsn
__attribute__ ((unused)),
void *arg)
{
struct st_filter_param *param= (struct st_filter_param *)arg;
return (type == PAGECACHE_LSN_PAGE) ||
(param->is_data_file &&
((pageno % param->pages_covered_by_bitmap) == 0));
}
/**
@brief dirty-page filtering criteria for INDIRECT checkpoint.
We flush all dirty bitmap pages.
@param type Page's type
@param pageno Page's number
@param rec_lsn Page's rec_lsn
@param arg filter_param
@return Operation status
@retval 0 don't flush the page
@retval 1 flush the page
*/
static int filter_flush_data_file_indirect(enum pagecache_page_type type
__attribute__ ((unused)),
pgcache_page_no_t pageno,
LSN rec_lsn
__attribute__ ((unused)),
void *arg)
{
struct st_filter_param *param= (struct st_filter_param *)arg;
return
(param->is_data_file &&
((pageno % param->pages_covered_by_bitmap) == 0));
}
/**
@brief dirty-page filtering criteria for background flushing thread.
We flush data pages which have been dirty since the previous checkpoint
(this is the two-checkpoint rule: the REDO phase will not have to start
from earlier than the next-to-last checkpoint), and all dirty bitmap
pages. But we flush no more than a certain number of pages (to have an
even flushing, no write burst).
@param type Page's type
@param pageno Page's number
@param rec_lsn Page's rec_lsn
@param arg filter_param
@return Operation status
@retval 0 don't flush the page
@retval 1 flush the page
@retval 2 don't flush the page and following pages
*/
static int filter_flush_data_file_evenly(enum pagecache_page_type type,
pgcache_page_no_t pageno
__attribute__ ((unused)),
LSN rec_lsn, void *arg)
{
struct st_filter_param *param= (struct st_filter_param *)arg;
if (unlikely(param->max_pages == 0)) /* all flushed already */
return 2;
if ((type == PAGECACHE_LSN_PAGE) &&
(cmp_translog_addr(rec_lsn, param->up_to_lsn) <= 0))
{
param->max_pages--;
return 1;
}
return 0;
}
for (i= 1; i<6; i++)
my_free(strings[i].str, MYF(MY_ALLOW_ZERO_PTR));
/**
@brief Background thread which does checkpoints and flushes periodically.
Takes a checkpoint every 30th second. After taking a checkpoint, all pages
dirty at the time of that checkpoint are flushed evenly until it is time to
take another checkpoint (30 seconds later). This ensures that the REDO
phase starts at earliest (in LSN time) at the next-to-last checkpoint
record ("two-checkpoint rule").
@note MikaelR questioned why the same thread does two different jobs, the
risk could be that while a checkpoint happens no LRD flushing happens.
@note MikaelR noted that he observed that Linux's file cache may never
fsync to disk until this cache is full, at which point it decides to empty
the cache, making the machine very slow. A solution was to fsync after
writing 2 MB.
*/
pthread_handler_t ma_checkpoint_background(void *arg __attribute__((unused)))
{
const uint sleep_unit= 1 /* 1 second */,
time_between_checkpoints= 30; /* 30 sleep units */
uint sleeps= 0;
my_thread_init();
DBUG_PRINT("info",("Maria background checkpoint thread starts"));
for(;;)
{
#if 0 /* good for testing, to do a lot of checkpoints, finds a lot of bugs */
sleeps=0;
#endif
uint pages_bunch_size;
struct st_filter_param filter_param;
PAGECACHE_FILE *dfile; /**< data file currently being flushed */
PAGECACHE_FILE *kfile; /**< index file currently being flushed */
TRANSLOG_ADDRESS log_horizon_at_last_checkpoint= LSN_IMPOSSIBLE;
ulonglong pagecache_flushes_at_last_checkpoint= 0;
struct timespec abstime;
switch((sleeps++) % time_between_checkpoints)
{
case 0:
/*
With background flushing evenly distributed over the time
between two checkpoints, we should have only little flushing to do
in the checkpoint.
*/
/*
this portion cannot be done as a hook in write_log_record() for the
LOGREC_CHECKPOINT type because:
- at that moment we still have not written to the control file so cannot
mark the request as done; this could be solved by writing to the control
file in the hook but that would be an I/O under the log's mutex, bad.
- it would not be nice organisation of code (I tried it :).
No checkpoint if no work of interest for recovery was done
since last checkpoint. Such work includes log writing (lengthens
recovery, checkpoint would shorten it), page flushing (checkpoint
would decrease the amount of read pages in recovery).
*/
if (candidate_max_rec_lsn_at_last_checkpoint != LSN_IMPOSSIBLE)
if ((translog_get_horizon() == log_horizon_at_last_checkpoint) &&
(pagecache_flushes_at_last_checkpoint ==
maria_pagecache->global_cache_write))
{
/* checkpoint succeeded */
/* safety against errors during flush by this thread: */
pages_to_flush_before_next_checkpoint= 0;
break;
}
ma_checkpoint_execute(CHECKPOINT_MEDIUM, TRUE);
/*
Snapshot this kind of "state" of the engine. Note that the value below
is possibly greater than last_checkpoint_lsn.
*/
log_horizon_at_last_checkpoint= translog_get_horizon();
pagecache_flushes_at_last_checkpoint=
maria_pagecache->global_cache_write;
/*
TODO: compute log's low water mark (how to do that with our fuzzy
ARIES-like reads of data structures? TODO think about it :).
If the checkpoint above succeeded it has set d|kfiles and
d|kfiles_end. If is has failed, it has set
pages_to_flush_before_next_checkpoint to 0 so we will skip flushing
and sleep until the next checkpoint.
*/
lock(log_mutex);
/* That LSN is used for the "two-checkpoint rule" (MEDIUM checkpoints) */
maximum_rec_lsn_last_checkpoint= candidate_max_rec_lsn_at_last_checkpoint;
DBUG_RETURN(FALSE);
break;
case 1:
/* set up parameters for background page flushing */
filter_param.up_to_lsn= last_checkpoint_lsn;
pages_bunch_size= pages_to_flush_before_next_checkpoint /
time_between_checkpoints;
dfile= dfiles;
kfile= kfiles;
/* fall through */
default:
if (pages_bunch_size > 0)
{
/* flush a bunch of dirty pages */
filter_param.max_pages= pages_bunch_size;
filter_param.is_data_file= TRUE;
while (dfile != dfiles_end)
{
int res=
flush_pagecache_blocks_with_filter(maria_pagecache,
dfile, FLUSH_KEEP,
filter_flush_data_file_evenly,
&filter_param);
/* note that it may just be a pinned page */
if (unlikely(res))
fprintf(stderr, "Maria engine: warning - background page flush"
" failed\n");
if (filter_param.max_pages == 0) /* bunch all flushed, sleep */
break; /* and we will continue with the same file */
dfile++; /* otherwise all this file is flushed, move to next file */
}
filter_param.is_data_file= FALSE;
while (kfile != kfiles_end)
{
int res=
flush_pagecache_blocks_with_filter(maria_pagecache,
dfile, FLUSH_KEEP,
filter_flush_data_file_evenly,
&filter_param);
if (unlikely(res))
fprintf(stderr, "Maria engine: warning - background page flush"
" failed\n");
if (filter_param.max_pages == 0) /* bunch all flushed, sleep */
break; /* and we will continue with the same file */
kfile++; /* otherwise all this file is flushed, move to next file */
}
}
}
lock(log_mutex);
DBUG_RETURN(TRUE);
pthread_mutex_lock(&LOCK_checkpoint);
if (checkpoint_thread_die == 1)
break;
#if 0 /* good for testing, to do a lot of checkpoints, finds a lot of bugs */
pthread_mutex_unlock(&LOCK_checkpoint);
my_sleep(100000); // a tenth of a second
pthread_mutex_lock(&LOCK_checkpoint);
#else
/* To have a killable sleep, we use timedwait like our SQL GET_LOCK() */
set_timespec(abstime, sleep_unit);
pthread_cond_timedwait(&COND_checkpoint, &LOCK_checkpoint, &abstime);
#endif
if (checkpoint_thread_die == 1)
break;
pthread_mutex_unlock(&LOCK_checkpoint);
}
pthread_mutex_unlock(&LOCK_checkpoint);
DBUG_PRINT("info",("Maria background checkpoint thread ends"));
/*
keep mutex locked upon exit because callers will want to clear
mutex-protected status variables
A last checkpoint, now that all tables should be closed; to have instant
recovery later. We always do it, because the test above about number of
log records or flushed pages is only approximative. For example, some log
records may have been written while ma_checkpoint_execute() above was
running, or some pages may have been flushed during this time. Thus it
could be that, while nothing has changed since that checkpoint's *end*, if
we recovered from that checkpoint we would have a non-empty dirty pages
list, REDOs to execute, and we don't want that, we want a clean shutdown
to have an empty recovery (simplifies upgrade/backups: one can just do a
clean shutdown, copy its tables to another system without copying the log
or control file and it will work because recovery will not need those).
Another reason why it's approximative is that a log record may have been
written above between ma_checkpoint_execute() and the
tranlog_get_horizon() which follows.
So, we have at least two checkpoints per start/stop of the engine, and
only two if the engine stays idle.
*/
ma_checkpoint_execute(CHECKPOINT_FULL, FALSE);
pthread_mutex_lock(&LOCK_checkpoint);
checkpoint_thread_die= 2; /* indicate that we are dead */
/* wake up ma_checkpoint_end() which may be waiting for our death */
pthread_cond_broadcast(&COND_checkpoint);
/* broadcast was inside unlock because ma_checkpoint_end() destroys mutex */
pthread_mutex_unlock(&LOCK_checkpoint);
my_thread_end();
return 0;
}
/**
@brief Allocates buffer and stores in it some info about open tables,
does some flushing on those.
/*
Here's what should be put in log_write_record() in the log handler:
Does the allocation because the caller cannot know the size itself.
Memory freeing is to be done by the caller (if the "str" member of the
LEX_STRING is not NULL).
The caller is taking a checkpoint.
@param[out] str pointer to where the allocated buffer,
and its size, will be put; buffer will be filled
with info about open tables
@param checkpoint_start_log_horizon Of the in-progress checkpoint
record.
@return Operation status
@retval 0 OK
@retval 1 Error
*/
log_write_record(...)
static int collect_tables(LEX_STRING *str, LSN checkpoint_start_log_horizon)
{
...;
lock(log_mutex);
...;
write_to_log(length);
written_since_last_checkpoint+= length;
if (written_since_last_checkpoint >
MAX_LOG_BYTES_WRITTEN_BETWEEN_CHECKPOINTS)
MARIA_SHARE **distinct_shares= NULL;
char *ptr;
uint error= 1, sync_error= 0, nb, nb_stored, i;
my_bool unmark_tables= TRUE;
uint total_names_length;
LIST *pos; /**< to iterate over open tables */
struct st_state_copy {
uint index;
MARIA_STATE_INFO state;
};
struct st_state_copy *state_copies= NULL, /**< fixed-size cache of states */
*state_copies_end, /**< cache ends here */
*state_copy; /**< iterator in cache */
TRANSLOG_ADDRESS state_copies_horizon; /**< horizon of states' _copies_ */
DBUG_ENTER("collect_tables");
/* let's make a list of distinct shares */
pthread_mutex_lock(&THR_LOCK_maria);
for (nb= 0, pos= maria_open_list; pos; pos= pos->next)
{
MARIA_HA *info= (MARIA_HA*)pos->data;
MARIA_SHARE *share= info->s;
/* the first three variables below can never change */
if (share->base.born_transactional && !share->temporary &&
share->mode != O_RDONLY &&
!(share->in_checkpoint & MARIA_CHECKPOINT_SEEN_IN_LOOP))
{
/*
ask one system thread (the "LRD background flusher and checkpointer
thread" WL#3261) to do a checkpoint
Why we didn't take intern_lock above: table had in_checkpoint==0 so no
thread could set in_checkpoint. And no thread needs to know that we
are setting in_checkpoint, because only maria_close() needs it and
cannot run now as we hold THR_LOCK_maria.
*/
request_asynchronous_checkpoint(INDIRECT);
/* prevent similar redundant requests */
written_since_last_checkpoint= (my_off_t)0;
/*
This table is relevant for checkpoint and not already seen. Mark it,
so that it is not seen again in the loop.
*/
nb++;
DBUG_ASSERT(share->in_checkpoint == 0);
/* This flag ensures that we count only _distinct_ shares. */
share->in_checkpoint= MARIA_CHECKPOINT_SEEN_IN_LOOP;
}
}
if (unlikely((distinct_shares=
(MARIA_SHARE **)my_malloc(nb * sizeof(MARIA_SHARE *),
MYF(MY_WME))) == NULL))
goto err;
for (total_names_length= 0, i= 0, pos= maria_open_list; pos; pos= pos->next)
{
MARIA_HA *info= (MARIA_HA*)pos->data;
MARIA_SHARE *share= info->s;
if (share->in_checkpoint & MARIA_CHECKPOINT_SEEN_IN_LOOP)
{
distinct_shares[i++]= share;
/*
With this we prevent the share from going away while we later flush
and force it without holding THR_LOCK_maria. For example if the share
could be my_free()d by maria_close() we would have a problem when we
access it to flush the table. We "pin" the share pointer.
And we also take down MARIA_CHECKPOINT_SEEN_IN_LOOP, so that it is
not seen again in the loop.
*/
share->in_checkpoint= MARIA_CHECKPOINT_LOOKS_AT_ME;
/** @todo avoid strlen() */
total_names_length+= strlen(share->open_file_name);
}
}
...;
unlock(log_mutex);
...;
}
/*
Requests a checkpoint from the background thread, *asynchronously*
(requestor does not wait for completion, and does not even later check the
result).
In real life it will be called by log_write_record().
*/
void request_asynchronous_checkpoint(CHECKPOINT_LEVEL level);
{
safemutex_assert_owner(log_mutex);
DBUG_ASSERT(i == nb);
pthread_mutex_unlock(&THR_LOCK_maria);
DBUG_PRINT("info",("found %u table shares", nb));
str->length=
4 + /* number of tables */
(2 + /* short id */
4 + /* kfile */
4 + /* dfile */
LSN_STORE_SIZE + /* first_log_write_at_lsn */
1 /* end-of-name 0 */
) * nb + total_names_length;
if (unlikely((str->str= my_malloc(str->length, MYF(MY_WME))) == NULL))
goto err;
DBUG_ASSERT(level > NONE);
if ((next_asynchronous_checkpoint_to_do < level) &&
(checkpoint_in_progress < level))
ptr= str->str;
ptr+= 4; /* real number of stored tables is not yet know */
struct st_filter_param filter_param;
/* only possible checkpointer, so can do the read below without mutex */
filter_param.up_to_lsn= last_checkpoint_lsn;
PAGECACHE_FILTER filter;
switch(checkpoint_in_progress)
{
/* no equal or stronger running or to run, we post request */
case CHECKPOINT_MEDIUM:
filter= &filter_flush_data_file_medium;
break;
case CHECKPOINT_FULL:
filter= &filter_flush_data_file_full;
break;
case CHECKPOINT_INDIRECT:
filter= &filter_flush_data_file_indirect;
break;
default:
DBUG_ASSERT(0);
goto err;
}
/*
We just don't broacast a cond, the checkpoint thread
(see ma_least_recently_dirtied.c) will notice our request in max a few
seconds.
The principle of reading/writing the state below is explained in
ma_recovery.c, look for "Recovery of the state".
*/
next_asynchronous_checkpoint_to_do= level; /* post request */
#define STATE_COPIES 1024
state_copies= (struct st_state_copy *)
my_malloc(STATE_COPIES * sizeof(struct st_state_copy), MYF(MY_WME));
dfiles= (PAGECACHE_FILE *)my_realloc((uchar *)dfiles,
/* avoid size of 0 for my_realloc */
max(1, nb) * sizeof(PAGECACHE_FILE),
MYF(MY_WME));
kfiles= (PAGECACHE_FILE *)my_realloc((uchar *)kfiles,
/* avoid size of 0 for my_realloc */
max(1, nb) * sizeof(PAGECACHE_FILE),
MYF(MY_WME));
if (unlikely((state_copies == NULL) ||
(dfiles == NULL) || (kfiles == NULL)))
goto err;
state_copy= state_copies_end= NULL;
dfiles_end= dfiles;
kfiles_end= kfiles;
for (nb_stored= 0, i= 0; i < nb; i++)
{
MARIA_SHARE *share= distinct_shares[i];
PAGECACHE_FILE kfile, dfile;
if (!(share->in_checkpoint & MARIA_CHECKPOINT_LOOKS_AT_ME))
{
/* No need for a mutex to read the above, only us can write this flag */
continue;
}
DBUG_PRINT("info",("looking at table '%s'", share->open_file_name));
if (state_copy == state_copies_end) /* we have no more cached states */
{
/*
Collect and cache a bunch of states. We do this for many states at a
time, to not lock/unlock the log's lock too often.
*/
uint j, bound= min(nb, i + STATE_COPIES);
state_copy= state_copies;
/* part of the state is protected by log's lock */
translog_lock();
state_copies_horizon= translog_get_horizon_no_lock();
for (j= i; j < bound; j++)
{
MARIA_SHARE *share2= distinct_shares[j];
if (!(share2->in_checkpoint & MARIA_CHECKPOINT_LOOKS_AT_ME))
continue;
state_copy->index= j;
state_copy->state= share2->state; /* we copy the state */
state_copy++;
/*
data_file_length is not updated under log's lock by the bitmap
code, but writing a wrong data_file_length is ok: a next
maria_close() will correct it; if we crash before, Recovery will
set it to the true physical size.
*/
}
translog_unlock();
state_copies_end= state_copy;
state_copy= state_copies;
/* so now we have cached states */
}
/* locate our state among these cached ones */
for ( ; state_copy->index != i; state_copy++)
DBUG_ASSERT(state_copy < state_copies_end);
filter_param.pages_covered_by_bitmap= share->bitmap.pages_covered;
/* OS file descriptors are ints which we stored in 4 bytes */
compile_time_assert(sizeof(int) == 4);
pthread_mutex_lock(&share->intern_lock);
/*
If there was an error, only an error
message to the error log will say it; normal, for a checkpoint triggered
by a log write, we probably don't want the client's log write to throw an
error, as the log write succeeded and a checkpoint failure is not
critical: the failure in this case is more for the DBA to know than for
the end user.
Tables in a normal state have their two file descriptors open.
In some rare cases like REPAIR, some descriptor may be closed or even
-1. If that happened, the _ma_state_info_write() may fail. This is
prevented by enclosing all all places which close/change kfile.file with
intern_lock.
*/
}
kfile= share->kfile;
dfile= share->bitmap.file;
/*
Ignore table which has no logged writes (all its future log records will
be found naturally by Recovery). Ignore obsolete shares (_before_
setting themselves to last_version=0 they already did all flush and
sync; if we flush their state now we may be flushing an obsolete state
onto a newer one (assuming the table has been reopened with a different
share but of course same physical index file).
*/
if ((share->id != 0) && (share->last_version != 0))
{
/** @todo avoid strlen */
uint open_file_name_len= strlen(share->open_file_name) + 1;
/* remember the descriptors for background flush */
*(dfiles_end++)= dfile;
*(kfiles_end++)= kfile;
/* we will store this table in the record */
nb_stored++;
int2store(ptr, share->id);
ptr+= 2;
/*
We must store the OS file descriptors, because the pagecache, which
tells us the list of dirty pages, refers to these pages by OS file
descriptors. An alternative is to make the page cache aware of the
2-byte id and of the location of a page ("is it a data file page or an
index file page?").
If one descriptor is -1, normally there should be no dirty pages
collected for this file, it's ok to store -1, it will not be used.
*/
int4store(ptr, kfile.file);
ptr+= 4;
int4store(ptr, dfile.file);
ptr+= 4;
lsn_store(ptr, share->lsn_of_file_id);
ptr+= LSN_STORE_SIZE;
/*
first_bitmap_with_space is not updated under log's lock, and is
important. We would need the bitmap's lock to get it right. Recovery
of this is not clear, so we just play safe: write it out as
unknown: if crash, _ma_bitmap_init() at next open (for example in
Recovery) will convert it to 0 and thus the first insertion will
search for free space from the file's first bitmap (0) -
under-optimal but safe.
If no crash, maria_close() will write the exact value.
*/
state_copy->state.first_bitmap_with_space= ~(ulonglong)0;
memcpy(ptr, share->open_file_name, open_file_name_len);
ptr+= open_file_name_len;
if (cmp_translog_addr(share->state.is_of_horizon,
checkpoint_start_log_horizon) >= 0)
{
/*
State was flushed recently, it does not hold down the log's
low-water mark and will not give avoidable work to Recovery. So we
needn't flush it. Also, it is possible that while we copied the
state above (under log's lock, without intern_lock) it was being
modified in memory or flushed to disk (without log's lock, under
intern_lock, like in maria_extra()), so our copy may be incorrect
and we should not flush it.
It may also be a share which got last_version==0 since we checked
last_version; in this case, it flushed its state and the LSN test
above will catch it.
*/
}
else
{
/*
We could do the state flush only if share->changed, but it's
tricky.
Consider a maria_write() which has written REDO,UNDO, and before it
calls _ma_writeinfo() (setting share->changed=1), checkpoint
happens and sees share->changed=0, does not flush state. It is
possible that Recovery does not start from before the REDO and thus
the state is not recovered. A solution may be to set
share->changed=1 under log mutex when writing log records.
But as anyway we have another problem below, this optimization would
be of little use.
*/
/** @todo flush state only if changed since last checkpoint */
DBUG_ASSERT(share->last_version != 0);
state_copy->state.is_of_horizon= share->state.is_of_horizon=
state_copies_horizon;
if (kfile.file >= 0)
sync_error|=
_ma_state_info_write_sub(kfile.file, &state_copy->state, 1);
/*
We don't set share->changed=0 because it may interfere with a
concurrent _ma_writeinfo() doing share->changed=1 (cancel its
effect). The sad consequence is that we will flush the same state at
each checkpoint if the table was once written and then not anymore.
*/
}
sync_error|=
_ma_flush_bitmap(share); /* after that, all is in page cache */
DBUG_ASSERT(share->pagecache == maria_pagecache);
}
if (share->in_checkpoint & MARIA_CHECKPOINT_SHOULD_FREE_ME)
{
/* maria_close() left us to free the share */
pthread_mutex_unlock(&share->intern_lock);
pthread_mutex_destroy(&share->intern_lock);
my_free((uchar *)share, MYF(0));
}
else
{
/* share goes back to normal state */
share->in_checkpoint= 0;
pthread_mutex_unlock(&share->intern_lock);
}
/*
We do the big disk writes out of intern_lock to not block other
users of this table (intern_lock is taken at the start and end of
every statement). This means that file descriptors may be invalid
(files may have been closed for example by HA_EXTRA_PREPARE_FOR_*
under Windows, or REPAIR). This should not be a problem as we use
MY_IGNORE_BADFD. Descriptors may even point to other files but then
the old blocks (of before the close) must have been flushed for sure,
so our flush will flush new blocks (of after the latest open) and that
should do no harm.
*/
/*
If CHECKPOINT_MEDIUM, this big flush below may result in a
serious write burst. Realize that all pages dirtied between the
last checkpoint and the one we are doing now, will be flushed at
next checkpoint, except those evicted by LRU eviction (depending on
the size of the page cache compared to the size of the working data
set, eviction may be rare or frequent).
We avoid that burst by anticipating: those pages are flushed
in bunches spanned regularly over the time interval between now and
the next checkpoint, by a background thread. Thus the next checkpoint
will have only little flushing to do (CHECKPOINT_MEDIUM should thus be
only a little slower than CHECKPOINT_INDIRECT).
*/
/*
If a 64-bit variable transitions from both halves being zero to both halves
being non-zero, and never changes after that (like the transaction's
first_undo_lsn), this function can be used to do a read of it (without
mutex, without atomic load) which always produces a correct (though maybe
slightly old) value (even on 32-bit CPUs).
The prototype will change with Sanja's new LSN type.
*/
static inline ulonglong read_non_atomic(ulonglong volatile *x)
{
#if ( SIZEOF_CHARP >= 8 )
/* 64-bit CPU (right?), 64-bit reads are atomic */
return *x;
#else
/**
@todo we ignore the error because it may be just due a pinned page;
we should rather fix the function below to distinguish between
pinned page and write error. Then we can turn the warning into an
error.
*/
if (((filter_param.is_data_file= TRUE),
flush_pagecache_blocks_with_filter(maria_pagecache,
&dfile, FLUSH_KEEP,
filter, &filter_param)) ||
((filter_param.is_data_file= FALSE),
flush_pagecache_blocks_with_filter(maria_pagecache,
&kfile, FLUSH_KEEP,
filter, &filter_param)))
fprintf(stderr, "Maria engine: warning - checkpoint page flush"
" failed\n"); /** @todo improve */
/*
fsyncs the fd, that's the loooong operation (e.g. max 150 fsync
per second, so if you have touched 1000 files it's 7 seconds).
*/
sync_error|=
my_sync(dfile.file, MYF(MY_WME | MY_IGNORE_BADFD)) |
my_sync(kfile.file, MYF(MY_WME | MY_IGNORE_BADFD));
/*
32-bit CPU, 64-bit reads may give a mixed of old half and new half (old
low bits and new high bits, or the contrary).
As the variable we read transitions from both halves being zero to both
halves being non-zero, and never changes then, we can detect atomicity
problems:
*/
ulonglong y;
for (;;) /* loop until no atomicity problems */
{
y= *x;
if (likely(((0 == y) ||
((0 != (y >> 32)) && (0 != (y << 32)))))
return y;
/* Worth seeing it! */
DBUG_PRINT("info",("atomicity problem"));
in case of error, we continue because writing other tables to disk is
still useful.
*/
}
#endif
if (sync_error)
goto err;
/* We maybe over-estimated (due to share->id==0 or last_version==0) */
DBUG_ASSERT(str->length >= (uint)(ptr - str->str));
str->length= (uint)(ptr - str->str);
/*
As we support max 65k tables open at a time (2-byte short id), we
assume uint is enough for the cumulated length of table names; and
LEX_STRING::length is uint.
*/
int4store(str->str, nb_stored);
error= unmark_tables= 0;
err:
if (unlikely(unmark_tables))
{
/* maria_close() uses THR_LOCK_maria from start to end */
pthread_mutex_lock(&THR_LOCK_maria);
for (i= 0; i < nb; i++)
{
MARIA_SHARE *share= distinct_shares[i];
if (share->in_checkpoint & MARIA_CHECKPOINT_SHOULD_FREE_ME)
{
/* maria_close() left us to free the share */
pthread_mutex_destroy(&share->intern_lock);
my_free((uchar *)share, MYF(0));
}
else
{
/* share goes back to normal state */
share->in_checkpoint= 0;
}
}
pthread_mutex_unlock(&THR_LOCK_maria);
}
my_free((uchar *)distinct_shares, MYF(MY_ALLOW_ZERO_PTR));
my_free((uchar *)state_copies, MYF(MY_ALLOW_ZERO_PTR));
DBUG_RETURN(error);
}
......@@ -32,7 +32,7 @@ typedef enum enum_ma_checkpoint_level {
} CHECKPOINT_LEVEL;
C_MODE_START
int ma_checkpoint_init();
int ma_checkpoint_init(my_bool create_background_thread);
void ma_checkpoint_end();
int ma_checkpoint_execute(CHECKPOINT_LEVEL level, my_bool no_wait);
C_MODE_END
......
......@@ -636,7 +636,6 @@ int maria_create(const char *name, enum data_file_type datafile_type,
share.state.dellink = HA_OFFSET_ERROR;
share.state.first_bitmap_with_space= 0;
share.state.create_rename_lsn= share.state.is_of_lsn= LSN_IMPOSSIBLE;
share.state.process= (ulong) getpid();
share.state.unique= (ulong) 0;
share.state.update_count=(ulong) 0;
......@@ -1006,7 +1005,7 @@ int maria_create(const char *name, enum data_file_type datafile_type,
DROP+CREATE happened (applying REDOs to the wrong table).
*/
share.kfile.file= file;
if (_ma_update_create_rename_lsn_on_disk_sub(&share, lsn, FALSE))
if (_ma_update_create_rename_lsn_sub(&share, lsn, FALSE))
goto err;
my_free(log_data, MYF(0));
}
......@@ -1070,7 +1069,9 @@ int maria_create(const char *name, enum data_file_type datafile_type,
if (my_chsize(dfile,share.base.min_pack_length*ci->reloc_rows,0,MYF(0)))
goto err;
#endif
if ((sync_dir && my_sync(dfile, MYF(0))) || my_close(dfile,MYF(0)))
if (sync_dir && my_sync(dfile, MYF(0)))
goto err;
if (my_close(dfile,MYF(0)))
goto err;
}
pthread_mutex_unlock(&THR_LOCK_maria);
......@@ -1207,7 +1208,7 @@ int _ma_initialize_data_file(MARIA_SHARE *share, File dfile)
/**
@brief Writes create_rename_lsn and is_of_lsn to disk, optionally forces.
@brief Writes create_rename_lsn and is_of_horizon to disk, can force.
This is for special cases where:
- we don't want to write the full state to disk (so, not call
......@@ -1224,21 +1225,21 @@ int _ma_initialize_data_file(MARIA_SHARE *share, File dfile)
@retval 1 error (disk problem)
*/
int _ma_update_create_rename_lsn_on_disk(MARIA_SHARE *share,
int _ma_update_create_rename_lsn(MARIA_SHARE *share,
LSN lsn, my_bool do_sync)
{
int res;
pthread_mutex_lock(&share->intern_lock);
res= _ma_update_create_rename_lsn_on_disk_sub(share, lsn, do_sync);
res= _ma_update_create_rename_lsn_sub(share, lsn, do_sync);
pthread_mutex_unlock(&share->intern_lock);
return res;
}
/**
@brief Writes create_rename_lsn and is_of_lsn to disk, optionally forces.
@brief Writes create_rename_lsn and is_of_horizon to disk, can force.
Shortcut of _ma_update_create_rename_lsn_on_disk() when we know that
Shortcut of _ma_update_create_rename_lsn() when we know that
intern_lock is not needed (when creating a table or opening it for the
first time).
......@@ -1250,7 +1251,7 @@ int _ma_update_create_rename_lsn_on_disk(MARIA_SHARE *share,
@retval 1 error (disk problem)
*/
int _ma_update_create_rename_lsn_on_disk_sub(MARIA_SHARE *share,
int _ma_update_create_rename_lsn_sub(MARIA_SHARE *share,
LSN lsn, my_bool do_sync)
{
char buf[LSN_STORE_SIZE*2], *ptr;
......@@ -1258,7 +1259,20 @@ int _ma_update_create_rename_lsn_on_disk_sub(MARIA_SHARE *share,
DBUG_ASSERT(file >= 0);
for (ptr= buf; ptr < (buf + sizeof(buf)); ptr+= LSN_STORE_SIZE)
lsn_store(ptr, lsn);
share->state.is_of_lsn= share->state.create_rename_lsn= lsn;
share->state.is_of_horizon= share->state.create_rename_lsn= lsn;
if (share->id != 0)
{
/*
If OP is the operation which is calling us, if table is later written,
we could see in the log:
FILE_ID ... REDO_OP ... REDO_INSERT.
(that can happen in real life at least with OP=REPAIR).
As FILE_ID will be ignored by Recovery because it is <
create_rename_lsn, REDO_INSERT would be ignored too, wrongly.
To avoid that, we force a LOGREC_FILE_ID to be logged at next write:
*/
translog_deassign_id_from_share(share);
}
return my_pwrite(file, buf, sizeof(buf),
sizeof(share->state.header) + 2, MYF(MY_NABP)) ||
(do_sync && my_sync(file, MYF(0)));
......
......@@ -297,8 +297,10 @@ int maria_extra(MARIA_HA *info, enum ha_extra_function function,
HA_EXTRA_PREPARE_FOR_DROP|RENAME.
*/
pthread_mutex_lock(&THR_LOCK_maria);
pthread_mutex_lock(&share->intern_lock); /* protect against Checkpoint */
/* this makes the share not be re-used next time the table is opened */
share->last_version= 0L; /* Impossible version */
pthread_mutex_unlock(&share->intern_lock);
pthread_mutex_unlock(&THR_LOCK_maria);
break;
case HA_EXTRA_PREPARE_FOR_DROP:
......@@ -306,9 +308,8 @@ int maria_extra(MARIA_HA *info, enum ha_extra_function function,
{
my_bool do_flush= test(function != HA_EXTRA_PREPARE_FOR_DROP);
pthread_mutex_lock(&THR_LOCK_maria);
share->last_version= 0L; /* Impossible version */
/*
This share, having last_version=0, needs to save all its data/index
This share, to have last_version=0, needs to save all its data/index
blocks to disk if this is not for a DROP TABLE. Otherwise they would be
invisible to future openers; and they could even go to disk late and
cancel the work of future openers.
......@@ -396,6 +397,8 @@ int maria_extra(MARIA_HA *info, enum ha_extra_function function,
}
}
#endif
/* For protection against Checkpoint, we set under intern_lock: */
share->last_version= 0L; /* Impossible version */
pthread_mutex_unlock(&share->intern_lock);
pthread_mutex_unlock(&THR_LOCK_maria);
break;
......
......@@ -19,6 +19,7 @@
#include <ft_global.h>
#include "ma_blockrec.h"
#include "trnman_public.h"
#include "ma_checkpoint.h"
my_bool maria_inited= FALSE;
pthread_mutex_t THR_LOCK_maria;
......@@ -56,6 +57,7 @@ void maria_end(void)
{
maria_inited= maria_multi_threaded= FALSE;
ft_free_stopwords();
ma_checkpoint_end();
trnman_destroy();
translog_destroy();
end_pagecache(maria_log_pagecache, TRUE);
......
/* Copyright (C) 2006 MySQL AB & MySQL Finland AB & TCX DataKonsult AB
This program is free software; you can redistribute it and/or modify
it under the terms of the GNU General Public License as published by
the Free Software Foundation; version 2 of the License.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA */
/*
WL#3261 Maria - background flushing of the least-recently-dirtied pages
First version written by Guilhem Bichot on 2006-04-27.
Does not compile yet.
*/
/*
To be part of the page cache.
The pseudocode below is dependent on the page cache
which is being designed WL#3134. It is not clear if I need to do page
copies, as the page cache already keeps page copies.
So, this code will move to the page cache and take inspiration from its
methods. Below is just to give the idea of what could be done.
And I should compare my imaginations to WL#3134.
*/
/* Here is the implementation of this module */
#include "page_cache.h"
#include "least_recently_dirtied.h"
/*
This thread does background flush of pieces of the LRD, and serves
requests for asynchronous checkpoints.
Just launch it when engine starts.
MikaelR questioned why the same thread does two different jobs, the risk
could be that while a checkpoint happens no LRD flushing happens.
For now, we only do checkpoints - no LRD flushing (to be done when the
second version of the page cache is ready WL#3077).
Reasons to delay:
- Recovery will work (just slower)
- new page cache may be different, why do then re-do
- current pagecache probably has issues with flushing when somebody is
writing to the table being flushed - better avoid that.
*/
pthread_handler_decl background_flush_and_checkpoint_thread()
{
while (this_thread_not_killed)
{
/* note that we don't care of the checkpoint's success */
(void)execute_asynchronous_checkpoint_if_any();
sleep(5);
/*
in the final version, we will not sleep but call flush_pages_from_LRD()
repeatedly. If there are no dirty pages, we'll make sure to not have a
tight loop probing for checkpoint requests.
*/
}
}
/* The rest of this file will not serve in first version */
/*
flushes only the first pages of the LRD.
max_this_number could be FLUSH_CACHE (of mf_pagecache.c) for example.
*/
flush_pages_from_LRD(uint max_this_number, LSN max_this_lsn)
{
/*
One rule to better observe is "page must be flushed to disk before it is
removed from LRD" (otherwise checkpoint is incomplete info, corruption).
*/
/*
Build a list of pages to flush:
changed_blocks[i] is roughly sorted by descending rec_lsn,
so we could do a merge sort of changed_blocks[] lists, stopping after we
have the max_this_number first elements or after we have found a page with
rec_lsn > max_this_lsn.
Then do like pagecache_flush_blocks_int() does (beware! this time we are
not alone on the file! there may be dangers! TODO: sort this out).
*/
/*
MikaelR noted that he observed that Linux's file cache may never fsync to
disk until this cache is full, at which point it decides to empty the
cache, making the machine very slow. A solution was to fsync after writing
2 MB.
*/
}
/*
Note that when we flush all page from LRD up to rec_lsn>=max_lsn,
this is approximate because the LRD list may
not be exactly sorted by rec_lsn (because for a big row, all pages of the
row are inserted into the LRD with rec_lsn being the LSN of the REDO for the
first page, so if there are concurrent insertions, the last page of the big
row may have a smaller rec_lsn than the previous pages inserted by
concurrent inserters).
*/
/* Copyright (C) 2006 MySQL AB & MySQL Finland AB & TCX DataKonsult AB
This program is free software; you can redistribute it and/or modify
it under the terms of the GNU General Public License as published by
the Free Software Foundation; version 2 of the License.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA */
/*
WL#3261 Maria - background flushing of the least-recently-dirtied pages
First version written by Guilhem Bichot on 2006-04-27.
Does not compile yet.
*/
/* This is the interface of this module. */
/* flushes all page from LRD up to approximately rec_lsn>=max_lsn */
int flush_all_LRD_to_lsn(LSN max_lsn);
......@@ -225,13 +225,12 @@ static my_bool write_hook_for_undo_row_delete(enum translog_record_type type,
TRN *trn, MARIA_HA *tbl_info,
LSN *lsn,
struct st_translog_parts *parts);
static my_bool write_hook_for_undo_row_purge(enum translog_record_type type,
TRN *trn, MARIA_HA *tbl_info,
LSN *lsn,
struct st_translog_parts *parts);
static my_bool write_hook_for_clr_end(enum translog_record_type type,
TRN *trn, MARIA_HA *tbl_info, LSN *lsn,
struct st_translog_parts *parts);
static my_bool write_hook_for_file_id(enum translog_record_type type,
TRN *trn, MARIA_HA *tbl_info, LSN *lsn,
struct st_translog_parts *parts);
static my_bool translog_page_validator(uchar *page_addr, uchar* data_ptr);
......@@ -386,10 +385,12 @@ static LOG_DESC INIT_LOGREC_REDO_INSERT_ROW_TAIL=
write_hook_for_redo, NULL, 0,
"redo_insert_row_tail", LOGREC_NOT_LAST_IN_GROUP, NULL, NULL};
/** @todo RECOVERY BUG unused, remove? */
static LOG_DESC INIT_LOGREC_REDO_INSERT_ROW_BLOB=
{LOGRECTYPE_VARIABLE_LENGTH, 0, 8, NULL, write_hook_for_redo, NULL, 0,
"redo_insert_row_blob", LOGREC_NOT_LAST_IN_GROUP, NULL, NULL};
/** @todo RECOVERY BUG handle it in recovery */
/*QQQ:TODO:header???*/
static LOG_DESC INIT_LOGREC_REDO_INSERT_ROW_BLOBS=
{LOGRECTYPE_VARIABLE_LENGTH, 0, FILEID_STORE_SIZE, NULL,
......@@ -416,10 +417,12 @@ static LOG_DESC INIT_LOGREC_REDO_PURGE_BLOCKS=
NULL, write_hook_for_redo, NULL, 0,
"redo_purge_blocks", LOGREC_NOT_LAST_IN_GROUP, NULL, NULL};
/* not yet used; for when we have versioning */
static LOG_DESC INIT_LOGREC_REDO_DELETE_ROW=
{LOGRECTYPE_FIXEDLENGTH, 16, 16, NULL, write_hook_for_redo, NULL, 0,
"redo_delete_row", LOGREC_NOT_LAST_IN_GROUP, NULL, NULL};
/** @todo RECOVERY BUG unused, remove? */
static LOG_DESC INIT_LOGREC_REDO_UPDATE_ROW_HEAD=
{LOGRECTYPE_VARIABLE_LENGTH, 0, 9, NULL, write_hook_for_redo, NULL, 0,
"redo_update_row_head", LOGREC_NOT_LAST_IN_GROUP, NULL, NULL};
......@@ -460,12 +463,6 @@ static LOG_DESC INIT_LOGREC_UNDO_ROW_UPDATE=
NULL, write_hook_for_undo, NULL, 1,
"undo_row_update", LOGREC_LAST_IN_GROUP, NULL, NULL};
static LOG_DESC INIT_LOGREC_UNDO_ROW_PURGE=
{LOGRECTYPE_PSEUDOFIXEDLENGTH, LSN_STORE_SIZE + FILEID_STORE_SIZE,
LSN_STORE_SIZE + FILEID_STORE_SIZE,
NULL, write_hook_for_undo_row_purge, NULL, 1,
"undo_row_purge", LOGREC_LAST_IN_GROUP, NULL, NULL};
static LOG_DESC INIT_LOGREC_UNDO_KEY_INSERT=
{LOGRECTYPE_VARIABLE_LENGTH, 0, 10, NULL, write_hook_for_undo, NULL, 1,
"undo_key_insert", LOGREC_LAST_IN_GROUP, NULL, NULL};
......@@ -518,7 +515,7 @@ static LOG_DESC INIT_LOGREC_REDO_REPAIR_TABLE=
"redo_repair_table", LOGREC_IS_GROUP_ITSELF, NULL, NULL};
static LOG_DESC INIT_LOGREC_FILE_ID=
{LOGRECTYPE_VARIABLE_LENGTH, 0, 2, NULL, NULL, NULL, 0,
{LOGRECTYPE_VARIABLE_LENGTH, 0, 2, NULL, write_hook_for_file_id, NULL, 0,
"file_id", LOGREC_IS_GROUP_ITSELF, NULL, NULL};
static LOG_DESC INIT_LOGREC_LONG_TRANSACTION_ID=
......@@ -564,8 +561,6 @@ static void loghandler_init()
INIT_LOGREC_UNDO_ROW_DELETE;
log_record_type_descriptor[LOGREC_UNDO_ROW_UPDATE]=
INIT_LOGREC_UNDO_ROW_UPDATE;
log_record_type_descriptor[LOGREC_UNDO_ROW_PURGE]=
INIT_LOGREC_UNDO_ROW_PURGE;
log_record_type_descriptor[LOGREC_UNDO_KEY_INSERT]=
INIT_LOGREC_UNDO_KEY_INSERT;
log_record_type_descriptor[LOGREC_UNDO_KEY_DELETE]=
......@@ -4941,7 +4936,7 @@ my_bool translog_write_record(LSN *lsn,
log records are written; for example SELECT FOR UPDATE takes locks but
writes no log record.
*/
if (unlikely(translog_assign_id_to_share(share, trn)))
if (unlikely(translog_assign_id_to_share(tbl_info, trn)))
DBUG_RETURN(1);
}
fileid_store(store_share_id, share->id);
......@@ -5156,6 +5151,20 @@ TRANSLOG_ADDRESS translog_get_horizon()
}
/**
@brief Returns the current horizon at the end of the current log, caller is
assumed to already hold the lock
@return Horizon
*/
TRANSLOG_ADDRESS translog_get_horizon_no_lock()
{
translog_lock_assert_owner();
return log_descriptor.horizon;
}
/*
Set last page in the scanner data structure
......@@ -5616,7 +5625,9 @@ int translog_read_record_header_from_buffer(uchar *page,
res= translog_fixed_length_header(page, page_offset, buff);
break;
default:
DBUG_ASSERT(0);
#ifdef ASK_SANJA
DBUG_ASSERT(0); /* fails on empty log (Sanja knows) */
#endif
res= RECHEADER_READ_ERROR;
}
DBUG_RETURN(res);
......@@ -5877,13 +5888,14 @@ static my_bool translog_record_read_next_chunk(struct st_translog_reader_data
static my_bool translog_init_reader_data(LSN lsn,
struct st_translog_reader_data *data)
{
int read_header;
DBUG_ENTER("translog_init_reader_data");
if (translog_init_scanner(lsn, 1, &data->scanner) ||
!(data->read_header=
translog_read_record_header_scan(&data->scanner, &data->header, 1)))
{
((read_header=
translog_read_record_header_scan(&data->scanner, &data->header, 1))
== RECHEADER_READ_ERROR))
DBUG_RETURN(1);
}
data->read_header= read_header;
data->body_offset= data->header.non_header_data_start_offset;
data->chunk_size= data->header.non_header_data_len;
data->current_offset= data->read_header;
......@@ -6384,26 +6396,6 @@ static my_bool write_hook_for_undo_row_delete(enum translog_record_type type
}
/**
@brief Upates "records" and calls the generic UNDO hook
@todo we will get rid of this record soon.
@return Operation status, always 0 (success)
*/
static my_bool write_hook_for_undo_row_purge(enum translog_record_type type
__attribute__ ((unused)),
TRN *trn, MARIA_HA *tbl_info,
LSN *lsn,
struct st_translog_parts *parts
__attribute__ ((unused)))
{
tbl_info->s->state.state.records--;
return write_hook_for_undo(type, trn, tbl_info, lsn, parts);
}
/**
@brief Sets transaction's undo_lsn, first_undo_lsn if needed
......@@ -6425,7 +6417,6 @@ static my_bool write_hook_for_clr_end(enum translog_record_type type
ptr[LSN_STORE_SIZE + FILEID_STORE_SIZE];
DBUG_ASSERT(trn->trid != 0);
/** @todo depending on what we are undoing, update "records" or not */
trn->undo_lsn= lsn_korr(ptr);
switch (undone_record_type) {
case LOGREC_UNDO_ROW_DELETE:
......@@ -6445,6 +6436,30 @@ static my_bool write_hook_for_clr_end(enum translog_record_type type
}
/**
@brief Updates table's lsn_of_file_id.
@todo move it to a separate file
@return Operation status, always 0 (success)
*/
static my_bool write_hook_for_file_id(enum translog_record_type type
__attribute__ ((unused)),
TRN *trn
__attribute__ ((unused)),
MARIA_HA *tbl_info,
LSN *lsn
__attribute__ ((unused)),
struct st_translog_parts *parts
__attribute__ ((unused)))
{
DBUG_ASSERT(cmp_translog_addr(tbl_info->s->lsn_of_file_id, *lsn) < 0);
tbl_info->s->lsn_of_file_id= *lsn;
return 0;
}
/**
@brief Gives a 2-byte-id to MARIA_SHARE and logs this fact
......@@ -6452,7 +6467,7 @@ static my_bool write_hook_for_clr_end(enum translog_record_type type
open MARIA_SHAREs), give it one and record this assignment in the log
(LOGREC_FILE_ID log record).
@param share table
@param tbl_info table
@param trn calling transaction
@return Operation status
......@@ -6462,8 +6477,9 @@ static my_bool write_hook_for_clr_end(enum translog_record_type type
@note Can be called even if share already has an id (then will do nothing)
*/
int translog_assign_id_to_share(MARIA_SHARE *share, TRN *trn)
int translog_assign_id_to_share(MARIA_HA *tbl_info, TRN *trn)
{
MARIA_SHARE *share= tbl_info->s;
/*
If you give an id to a non-BLOCK_RECORD table, you also need to release
this id somewhere. Then you can change the assertion.
......@@ -6495,7 +6511,6 @@ int translog_assign_id_to_share(MARIA_SHARE *share, TRN *trn)
LSN lsn;
LEX_STRING log_array[TRANSLOG_INTERNAL_PARTS + 2];
uchar log_data[FILEID_STORE_SIZE];
fileid_store(log_data, share->id);
log_array[TRANSLOG_INTERNAL_PARTS + 0].str= (char*) log_data;
log_array[TRANSLOG_INTERNAL_PARTS + 0].length= sizeof(log_data);
/*
......@@ -6510,22 +6525,13 @@ int translog_assign_id_to_share(MARIA_SHARE *share, TRN *trn)
*/
log_array[TRANSLOG_INTERNAL_PARTS + 1].length=
strlen(share->open_file_name) + 1;
if (unlikely(translog_write_record(&lsn, LOGREC_FILE_ID, trn, NULL,
if (unlikely(translog_write_record(&lsn, LOGREC_FILE_ID, trn, tbl_info,
sizeof(log_data) +
log_array[TRANSLOG_INTERNAL_PARTS +
1].length,
sizeof(log_array)/sizeof(log_array[0]),
log_array, NULL)))
log_array, log_data)))
return 1;
/*
Note that we first set share->id then write the record. The checkpoint
record does not include any share with id==0; this is ok because:
checkpoint_start_log_horizon is either before or after the above
record. If before, ok to not include the share, as the record will be
seen for sure during the REDO phase. If after, Checkpoint will see all
data as it was after this record was written, including the id!=0, so
share will be included.
*/
}
pthread_mutex_unlock(&share->intern_lock);
return 0;
......@@ -6546,12 +6552,17 @@ void translog_deassign_id_from_share(MARIA_SHARE *share)
(ulong)share, share->id));
/*
We don't need any mutex as we are called only when closing the last
instance of the table: no writes can be happening.
instance of the table or at the end of REPAIR: no writes can be
happening. But a Checkpoint may be reading share->id, so we require this
mutex:
*/
safe_mutex_assert_owner(&share->intern_lock);
my_atomic_rwlock_rdlock(&LOCK_id_to_share);
my_atomic_storeptr((void **)&id_to_share[share->id], 0);
my_atomic_rwlock_rdunlock(&LOCK_id_to_share);
share->id= 0;
/* useless but safety: */
share->lsn_of_file_id= LSN_IMPOSSIBLE;
}
......@@ -6733,13 +6744,13 @@ LSN translog_first_theoretical_lsn()
/**
@brief Check given low water mark and purge files if it is need
@param low the last (minimum) LSN which is need
@param low the last (minimum) address which is need
@retval 0 OK
@retval 1 Error
*/
my_bool translog_purge(LSN low)
my_bool translog_purge(TRANSLOG_ADDRESS low)
{
uint32 last_need_file= LSN_FILE_NO(low);
TRANSLOG_ADDRESS horizon= translog_get_horizon();
......
......@@ -105,7 +105,6 @@ enum translog_record_type
LOGREC_UNDO_ROW_INSERT,
LOGREC_UNDO_ROW_DELETE,
LOGREC_UNDO_ROW_UPDATE,
LOGREC_UNDO_ROW_PURGE,
LOGREC_UNDO_KEY_INSERT,
LOGREC_UNDO_KEY_DELETE,
LOGREC_PREPARE,
......@@ -251,13 +250,14 @@ extern my_bool translog_init_scanner(LSN lsn,
extern int translog_read_next_record_header(TRANSLOG_SCANNER_DATA *scanner,
TRANSLOG_HEADER_BUFFER *buff);
extern LSN translog_get_file_max_lsn_stored(uint32 file);
extern my_bool translog_purge(LSN low);
extern my_bool translog_purge(TRANSLOG_ADDRESS low);
extern my_bool translog_is_file(uint file_no);
extern my_bool translog_lock();
extern my_bool translog_unlock();
extern void translog_lock_assert_owner();
extern TRANSLOG_ADDRESS translog_get_horizon();
extern int translog_assign_id_to_share(struct st_maria_share *share,
extern TRANSLOG_ADDRESS translog_get_horizon_no_lock();
extern int translog_assign_id_to_share(struct st_maria_info *tbl_info,
struct st_transaction *trn);
extern void translog_deassign_id_from_share(struct st_maria_share *share);
extern void
......
......@@ -618,14 +618,13 @@ MARIA_HA *maria_open(const char *name, int mode, uint open_flags)
view of the server, including server's recovery) now.
*/
if ((open_flags & HA_OPEN_FROM_SQL_LAYER) || maria_in_recovery)
_ma_update_create_rename_lsn_on_disk_sub(share,
translog_get_horizon(),
_ma_update_create_rename_lsn_sub(share, translog_get_horizon(),
TRUE);
}
else if ((!LSN_VALID(share->state.create_rename_lsn) ||
!LSN_VALID(share->state.is_of_lsn) ||
!LSN_VALID(share->state.is_of_horizon) ||
(cmp_translog_addr(share->state.create_rename_lsn,
share->state.is_of_lsn) > 0)) &&
share->state.is_of_horizon) > 0)) &&
!(open_flags & HA_OPEN_FOR_REPAIR))
{
/*
......@@ -981,7 +980,7 @@ static void setup_key_functions(register MARIA_KEYDEF *keyinfo)
@brief Function to save and store the header in the index file (.MYI)
Operates under MARIA_SHARE::intern_lock if requested.
Sets MARIA_SHARE::MARIA_STATE_INFO::is_of_lsn if table is transactional.
Sets MARIA_SHARE::MARIA_STATE_INFO::is_of_horizon if transactional table.
Then calls _ma_state_info_write_sub().
@param share table
......@@ -998,7 +997,7 @@ static void setup_key_functions(register MARIA_KEYDEF *keyinfo)
uint _ma_state_info_write(MARIA_SHARE *share, uint pWrite)
{
uint res= 0;
uint res;
if (pWrite & 4)
pthread_mutex_lock(&share->intern_lock);
else if (maria_multi_threaded)
......@@ -1007,11 +1006,11 @@ uint _ma_state_info_write(MARIA_SHARE *share, uint pWrite)
!maria_in_recovery)
{
/*
In a recovery, we want to set is_of_lsn to the LSN of the last
In a recovery, we want to set is_of_horizon to the LSN of the last
record executed by Recovery, not the current EOF of the log (which
is too new). Recovery does it by itself.
*/
share->state.is_of_lsn= translog_get_horizon();
share->state.is_of_horizon= translog_get_horizon();
}
res= _ma_state_info_write_sub(share->kfile.file, &share->state, pWrite);
if (pWrite & 4)
......@@ -1052,11 +1051,12 @@ uint _ma_state_info_write_sub(File file, MARIA_STATE_INFO *state, uint pWrite)
/* open_count must be first because of _ma_mark_file_changed ! */
mi_int2store(ptr,state->open_count); ptr+= 2;
/*
if you change the offset of create_rename_lsn/is_of_lsn inside the file,
fix ma_create + ma_rename + ma_delete_all + backward-compatibility.
if you change the offset of create_rename_lsn/is_of_horizon inside the
index file's header, fix ma_create + ma_rename + ma_delete_all +
backward-compatibility.
*/
lsn_store(ptr, state->create_rename_lsn); ptr+= LSN_STORE_SIZE;
lsn_store(ptr, state->is_of_lsn); ptr+= LSN_STORE_SIZE;
lsn_store(ptr, state->is_of_horizon); ptr+= LSN_STORE_SIZE;
*ptr++= (uchar)state->changed;
*ptr++= state->sortkey;
mi_rowstore(ptr,state->state.records); ptr+= 8;
......@@ -1119,7 +1119,7 @@ static uchar *_ma_state_info_read(uchar *ptr, MARIA_STATE_INFO *state)
state->open_count = mi_uint2korr(ptr); ptr+= 2;
state->create_rename_lsn= lsn_korr(ptr); ptr+= LSN_STORE_SIZE;
state->is_of_lsn= lsn_korr(ptr); ptr+= LSN_STORE_SIZE;
state->is_of_horizon= lsn_korr(ptr); ptr+= LSN_STORE_SIZE;
state->changed= (my_bool) *ptr++;
state->sortkey= (uint) *ptr++;
state->state.records= mi_rowkorr(ptr); ptr+= 8;
......
......@@ -3865,8 +3865,6 @@ int reset_pagecache_counters(const char *name, PAGECACHE *pagecache)
its size, will be put
@param[out] min_rec_lsn pointer to where the minimum rec_lsn of all
relevant dirty pages will be put
@param[out] max_rec_lsn pointer to where the maximum rec_lsn of all
relevant dirty pages will be put
@return Operation status
@retval 0 OK
@retval 1 Error
......@@ -3874,14 +3872,13 @@ int reset_pagecache_counters(const char *name, PAGECACHE *pagecache)
my_bool pagecache_collect_changed_blocks_with_lsn(PAGECACHE *pagecache,
LEX_STRING *str,
LSN *min_rec_lsn,
LSN *max_rec_lsn)
LSN *min_rec_lsn)
{
my_bool error= 0;
ulong stored_list_size= 0;
uint stored_list_size= 0;
uint file_hash;
char *ptr;
LSN minimum_rec_lsn= LSN_MAX, maximum_rec_lsn= 0;
LSN minimum_rec_lsn= LSN_MAX;
DBUG_ENTER("pagecache_collect_changed_blocks_with_LSN");
DBUG_ASSERT(NULL == str->str);
......@@ -3921,7 +3918,8 @@ my_bool pagecache_collect_changed_blocks_with_lsn(PAGECACHE *pagecache,
}
}
str->length= 8 + /* number of dirty pages */
compile_time_assert(sizeof(pagecache->blocks == 4));
str->length= 4 + /* number of dirty pages */
(4 + /* file */
4 + /* pageno */
LSN_STORE_SIZE /* rec_lsn */
......@@ -3929,8 +3927,8 @@ my_bool pagecache_collect_changed_blocks_with_lsn(PAGECACHE *pagecache,
if (NULL == (str->str= my_malloc(str->length, MYF(MY_WME))))
goto err;
ptr= str->str;
int8store(ptr, stored_list_size);
ptr+= 8;
int4store(ptr, stored_list_size);
ptr+= 4;
if (!stored_list_size)
goto end;
for (file_hash= 0; file_hash < PAGECACHE_CHANGED_BLOCKS_HASH; file_hash++)
......@@ -3955,15 +3953,12 @@ my_bool pagecache_collect_changed_blocks_with_lsn(PAGECACHE *pagecache,
DBUG_ASSERT(LSN_VALID(block->rec_lsn));
if (cmp_translog_addr(block->rec_lsn, minimum_rec_lsn) < 0)
minimum_rec_lsn= block->rec_lsn;
if (cmp_translog_addr(block->rec_lsn, maximum_rec_lsn) > 0)
maximum_rec_lsn= block->rec_lsn;
} /* otherwise, some trn->rec_lsn should hold the correct info */
}
}
end:
pagecache_pthread_mutex_unlock(&pagecache->cache_lock);
*min_rec_lsn= minimum_rec_lsn;
*max_rec_lsn= maximum_rec_lsn;
DBUG_RETURN(error);
err:
......
......@@ -247,8 +247,7 @@ extern my_bool pagecache_delete_pages(PAGECACHE *pagecache,
extern void end_pagecache(PAGECACHE *keycache, my_bool cleanup);
extern my_bool pagecache_collect_changed_blocks_with_lsn(PAGECACHE *pagecache,
LEX_STRING *str,
LSN *min_lsn,
LSN *max_lsn);
LSN *min_lsn);
extern int reset_pagecache_counters(const char *name, PAGECACHE *pagecache);
......
......@@ -63,6 +63,7 @@ static FILE *tracef; /**< trace file for debugging */
prototype_redo_exec_hook(LONG_TRANSACTION_ID);
prototype_redo_exec_hook_dummy(CHECKPOINT);
prototype_redo_exec_hook(REDO_CREATE_TABLE);
prototype_redo_exec_hook(REDO_RENAME_TABLE);
prototype_redo_exec_hook(REDO_DROP_TABLE);
prototype_redo_exec_hook(FILE_ID);
prototype_redo_exec_hook(REDO_INSERT_ROW_HEAD);
......@@ -74,7 +75,6 @@ prototype_redo_exec_hook(REDO_DELETE_ALL);
prototype_redo_exec_hook(UNDO_ROW_INSERT);
prototype_redo_exec_hook(UNDO_ROW_DELETE);
prototype_redo_exec_hook(UNDO_ROW_UPDATE);
prototype_redo_exec_hook(UNDO_ROW_PURGE);
prototype_redo_exec_hook(COMMIT);
prototype_redo_exec_hook(CLR_END);
prototype_undo_exec_hook(UNDO_ROW_INSERT);
......@@ -93,12 +93,13 @@ static MARIA_HA *get_MARIA_HA_from_REDO_record(const
TRANSLOG_HEADER_BUFFER *rec);
static MARIA_HA *get_MARIA_HA_from_UNDO_record(const
TRANSLOG_HEADER_BUFFER *rec);
static void prepare_table_for_close(MARIA_HA *info, LSN at_lsn);
static int parse_checkpoint_record(LSN lsn);
static void prepare_table_for_close(MARIA_HA *info, TRANSLOG_ADDRESS horizon);
static LSN parse_checkpoint_record(LSN lsn);
static void new_transaction(uint16 sid, TrID long_id, LSN undo_lsn,
LSN first_undo_lsn);
static int new_table(uint16 sid, const char *name,
File org_kfile, File org_dfile, LSN lsn);
File org_kfile, File org_dfile,
LSN lsn_of_file_id);
static int new_page(File fileid, pgcache_page_no_t pageid, LSN rec_lsn,
struct st_dirty_page *dirty_page);
static int close_all_tables();
......@@ -124,6 +125,10 @@ static LEX_STRING log_record_buffer;
of runtime: recreates transactions inside trnman, open tables with their
two-byte-id mapping; takes a checkpoint and runs the UNDO phase. Closes all
tables.
@return Operation status
@retval 0 OK
@retval !=0 Error
*/
int maria_recover()
......@@ -201,7 +206,6 @@ int maria_apply_log(LSN from_lsn, my_bool apply, FILE *trace_file,
}
else
{
DBUG_ASSERT(0); /* not yet implemented */
from_lsn= parse_checkpoint_record(last_checkpoint_lsn);
if (from_lsn == LSN_IMPOSSIBLE)
goto err;
......@@ -230,10 +234,7 @@ int maria_apply_log(LSN from_lsn, my_bool apply, FILE *trace_file,
if (close_all_tables())
goto err;
/*
At this stage, end of recovery, trnman is left initialized. This is for
the future, when we have an online UNDO phase or prepared transactions.
*/
/* If inside ha_maria, a checkpoint will soon be taken and save our work */
goto end;
err:
error= 1;
......@@ -426,7 +427,7 @@ prototype_redo_exec_hook(REDO_CREATE_TABLE)
ptr+= 2;
/* set create_rename_lsn (for maria_read_log to be idempotent) */
lsn_store(ptr + sizeof(info->s->state.header) + 2, rec->lsn);
/* we also set is_of_lsn, like maria_create() does */
/* we also set is_of_horizon, like maria_create() does */
lsn_store(ptr + sizeof(info->s->state.header) + 2 + LSN_STORE_SIZE,
rec->lsn);
if (my_pwrite(kfile, ptr,
......@@ -474,6 +475,77 @@ end:
}
prototype_redo_exec_hook(REDO_RENAME_TABLE)
{
char *old_name, *new_name;
int error= 1;
MARIA_HA *info= NULL;
enlarge_buffer(rec);
if (log_record_buffer.str == NULL ||
translog_read_record(rec->lsn, 0, rec->record_length,
log_record_buffer.str, NULL) !=
rec->record_length)
{
fprintf(tracef, "Failed to read record\n");
goto end;
}
old_name= log_record_buffer.str;
new_name= old_name + strlen(old_name) + 1;
fprintf(tracef, "Table '%s' to rename to '%s'", old_name, new_name);
info= maria_open(old_name, O_RDONLY, HA_OPEN_FOR_REPAIR);
if (info)
{
MARIA_SHARE *share= info->s;
/*
We may have open instances on this table. But it does not matter, the
maria_extra() below will take care of them.
*/
if (!share->base.born_transactional)
{
fprintf(tracef, ", is not transactional\n");
ALERT_USER();
error= 0;
goto end;
}
if (cmp_translog_addr(share->state.create_rename_lsn, rec->lsn) >= 0)
{
fprintf(tracef, ", has create_rename_lsn (%lu,0x%lx) more recent than"
" record, ignoring renaming",
LSN_IN_HEX(share->state.create_rename_lsn));
error= 0;
goto end;
}
if (maria_is_crashed(info))
{
fprintf(tracef, ", is crashed, can't rename it");
ALERT_USER();
goto end;
}
/*
This maria_extra() call serves to signal that old open instances of
this table should not be used anymore, and (only on Windows) to close
open files so they can be renamed
*/
if (maria_extra(info, HA_EXTRA_PREPARE_FOR_RENAME, NULL) ||
maria_close(info))
goto end;
info= NULL;
}
fprintf(tracef, ", renaming '%s'", old_name);
if (maria_rename(old_name, new_name))
{
fprintf(tracef, "Failed to rename table\n");
goto end;
}
error= 0;
end:
fprintf(tracef, "\n");
if (info != NULL)
error|= maria_close(info);
return error;
}
prototype_redo_exec_hook(REDO_DROP_TABLE)
{
char *name;
......@@ -553,6 +625,12 @@ prototype_redo_exec_hook(FILE_ID)
if (cmp_translog_addr(rec->lsn, checkpoint_start) < 0)
{
/*
If that mapping was still true at checkpoint time, it was found in
checkpoint record, no need to recreate it. If that mapping had ended at
checkpoint time (table was closed or repaired), a flush and force
happened and so mapping is not needed.
*/
fprintf(tracef, "ignoring because before checkpoint\n");
return 0;
}
......@@ -589,7 +667,8 @@ end:
static int new_table(uint16 sid, const char *name,
File org_kfile, File org_dfile, LSN lsn)
File org_kfile, File org_dfile,
LSN lsn_of_file_id)
{
/*
-1 (skip table): close table and return 0;
......@@ -628,11 +707,12 @@ static int new_table(uint16 sid, const char *name,
error= -1;
goto end;
}
if (cmp_translog_addr(lsn, share->state.create_rename_lsn) <= 0)
if (cmp_translog_addr(lsn_of_file_id, share->state.create_rename_lsn) <= 0)
{
fprintf(tracef, ", has create_rename_lsn (%lu,0x%lx) more recent than"
" record, ignoring open request",
LSN_IN_HEX(share->state.create_rename_lsn));
" LOGREC_FILE_ID's LSN (%lu,0x%lx), ignoring open request",
LSN_IN_HEX(share->state.create_rename_lsn),
LSN_IN_HEX(lsn_of_file_id));
error= -1;
goto end;
}
......@@ -655,6 +735,16 @@ static int new_table(uint16 sid, const char *name,
/* Recovery will fix this, no error */
ALERT_USER();
}
/*
This LSN serves in this situation; assume log is:
FILE_ID(6->"t2") REDO_INSERT(6) FILE_ID(6->"t1") CHECKPOINT(6->"t1")
then crash, checkpoint record is parsed and opens "t1" with id 6; assume
REDO phase starts from the REDO_INSERT above: it will wrongly try to
update a page of "t1". With this LSN below, REDO_INSERT can realize the
mapping is newer than itself, and not execute.
Same example is possible with UNDO_INSERT (update of the state).
*/
info->s->lsn_of_file_id= lsn_of_file_id;
all_tables[sid].info= info;
all_tables[sid].org_kfile= org_kfile;
all_tables[sid].org_dfile= org_dfile;
......@@ -848,7 +938,7 @@ prototype_redo_exec_hook(UNDO_ROW_INSERT)
if (info == NULL)
return 0;
set_undo_lsn_for_active_trans(rec->short_trid, rec->lsn);
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_lsn) > 0)
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_horizon) > 0)
{
fprintf(tracef, " state older than record, updating rows' count\n");
info->s->state.state.records++;
......@@ -871,7 +961,7 @@ prototype_redo_exec_hook(UNDO_ROW_DELETE)
if (info == NULL)
return 0;
set_undo_lsn_for_active_trans(rec->short_trid, rec->lsn);
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_lsn) > 0)
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_horizon) > 0)
{
fprintf(tracef, " state older than record, updating rows' count\n");
info->s->state.state.records--;
......@@ -889,7 +979,7 @@ prototype_redo_exec_hook(UNDO_ROW_UPDATE)
if (info == NULL)
return 0;
set_undo_lsn_for_active_trans(rec->short_trid, rec->lsn);
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_lsn) > 0)
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_horizon) > 0)
{
info->s->state.changed|= STATE_CHANGED | STATE_NOT_ANALYZED |
STATE_NOT_OPTIMIZED_KEYS | STATE_NOT_SORTED_PAGES;
......@@ -898,25 +988,6 @@ prototype_redo_exec_hook(UNDO_ROW_UPDATE)
}
prototype_redo_exec_hook(UNDO_ROW_PURGE)
{
MARIA_HA *info= get_MARIA_HA_from_UNDO_record(rec);
if (info == NULL)
return 0;
/* this a bit broken, but this log record type will be deleted soon */
set_undo_lsn_for_active_trans(rec->short_trid, rec->lsn);
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_lsn) > 0)
{
fprintf(tracef, " state older than record, updating rows' count\n");
info->s->state.state.records--;
info->s->state.changed|= STATE_CHANGED | STATE_NOT_ANALYZED |
STATE_NOT_OPTIMIZED_KEYS | STATE_NOT_SORTED_PAGES;
}
fprintf(tracef, " rows' count %lu\n", (ulong)info->s->state.state.records);
return 0;
}
prototype_redo_exec_hook(COMMIT)
{
uint16 sid= rec->short_trid;
......@@ -969,7 +1040,7 @@ prototype_redo_exec_hook(CLR_END)
set_undo_lsn_for_active_trans(rec->short_trid, previous_undo_lsn);
fprintf(tracef, " CLR_END was about %s, undo_lsn now LSN (%lu,0x%lx)\n",
log_desc->name, LSN_IN_HEX(previous_undo_lsn));
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_lsn) > 0)
if (cmp_translog_addr(rec->lsn, info->s->state.is_of_horizon) > 0)
{
fprintf(tracef, " state older than record, updating rows' count\n");
switch (undone_record_type) {
......@@ -1113,6 +1184,7 @@ static int run_redo_phase(LSN lsn, my_bool apply)
install_redo_exec_hook(LONG_TRANSACTION_ID);
install_redo_exec_hook(CHECKPOINT);
install_redo_exec_hook(REDO_CREATE_TABLE);
install_redo_exec_hook(REDO_RENAME_TABLE);
install_redo_exec_hook(REDO_DROP_TABLE);
install_redo_exec_hook(FILE_ID);
install_redo_exec_hook(REDO_INSERT_ROW_HEAD);
......@@ -1124,7 +1196,6 @@ static int run_redo_phase(LSN lsn, my_bool apply)
install_redo_exec_hook(UNDO_ROW_INSERT);
install_redo_exec_hook(UNDO_ROW_DELETE);
install_redo_exec_hook(UNDO_ROW_UPDATE);
install_redo_exec_hook(UNDO_ROW_PURGE);
install_redo_exec_hook(COMMIT);
install_redo_exec_hook(CLR_END);
install_undo_exec_hook(UNDO_ROW_INSERT);
......@@ -1134,17 +1205,14 @@ static int run_redo_phase(LSN lsn, my_bool apply)
current_group_end_lsn= LSN_IMPOSSIBLE;
TRANSLOG_HEADER_BUFFER rec;
/*
instead of this block below we will soon use
translog_first_lsn_in_log()...
*/
int len= translog_read_record_header(lsn, &rec);
/** @todo EOF should be detected */
if (len == RECHEADER_READ_ERROR)
{
fprintf(tracef, "Cannot find a first record\n");
return 1;
fprintf(tracef, "Cannot find a first record, empty log, nothing to do\n");
return 0;
}
struct st_translog_scanner_data scanner;
if (translog_init_scanner(lsn, 1, &scanner))
......@@ -1247,7 +1315,7 @@ static int run_redo_phase(LSN lsn, my_bool apply)
fprintf(tracef, "EOF on the log\n");
break;
case RECHEADER_READ_ERROR:
fprintf(stderr, "Error reading log\n");
fprintf(tracef, "Error reading log\n");
return 1;
}
break;
......@@ -1281,7 +1349,7 @@ static uint end_of_redo_phase(my_bool prepare_for_undo_phase)
dirty_pages_pool= NULL;
llstr(max_long_trid, llbuf);
printf("Maximum transaction long id seen: %s\n", llbuf);
fprintf(tracef, "Maximum transaction long id seen: %s\n", llbuf);
if (prepare_for_undo_phase && trnman_init(max_long_trid))
return -1;
......@@ -1400,22 +1468,30 @@ static int run_undo_phase(uint unfinished)
/**
@brief re-enables transactionality, updates is_of_lsn
@brief re-enables transactionality, updates is_of_horizon
@param info table
@param at_lsn LSN to set is_of_lsn
@param horizon address to set is_of_horizon
*/
static void prepare_table_for_close(MARIA_HA *info, LSN at_lsn)
static void prepare_table_for_close(MARIA_HA *info, TRANSLOG_ADDRESS horizon)
{
MARIA_SHARE *share= info->s;
/*
State is now at least as new as the LSN of the current record. It may be
In a fully-forward REDO phase (no checkpoint record),
state is now at least as new as the LSN of the current record. It may be
newer, in case we are seeing a LOGREC_FILE_ID which tells us to close a
table, but that table was later modified further in the log.
But if we parsed a checkpoint record, it may be this way in the log:
FILE_ID(6->t2)... FILE_ID(6->t1)... CHECKPOINT(6->t1)
Checkpoint parsing opened t1 with id 6; first FILE_ID above is going to
make t1 close; the first condition below is however false (when checkpoint
was taken it increased is_of_horizon) and so it works. For safety we
add the second condition.
*/
if (cmp_translog_addr(share->state.is_of_lsn, at_lsn) < 0)
share->state.is_of_lsn= at_lsn;
if (cmp_translog_addr(share->state.is_of_horizon, horizon) < 0 &&
cmp_translog_addr(share->lsn_of_file_id, horizon) < 0)
share->state.is_of_horizon= horizon;
_ma_reenable_logging_for_table(share);
}
......@@ -1446,6 +1522,22 @@ static MARIA_HA *get_MARIA_HA_from_REDO_record(const
return NULL;
}
fprintf(tracef, ", '%s'", info->s->open_file_name);
if (cmp_translog_addr(rec->lsn, info->s->lsn_of_file_id) <= 0)
{
/*
This can happen only if processing a record before the checkpoint
record.
id->name mapping is newer than REDO record: for sure the table subject
of the REDO has been flushed and forced (id re-assignment implies this);
REDO can be ignored (and must be, as we don't know what this subject
table was).
*/
DBUG_ASSERT(cmp_translog_addr(rec->lsn, checkpoint_start) < 0);
fprintf(tracef, ", table's LOGREC_FILE_ID has LSN (%lu,0x%lx) more recent"
" than record, skipping record",
LSN_IN_HEX(info->s->lsn_of_file_id));
return NULL;
}
/* detect if an open instance of a dropped table (internal bug) */
DBUG_ASSERT(info->s->last_version != 0);
if (cmp_translog_addr(rec->lsn, checkpoint_start) < 0)
......@@ -1491,27 +1583,45 @@ static MARIA_HA *get_MARIA_HA_from_UNDO_record(const
fprintf(tracef, ", table skipped, so skipping record\n");
return NULL;
}
_ma_writeinfo(info, WRITEINFO_UPDATE_KEYFILE); /* to flush state on close */
fprintf(tracef, ", '%s'", info->s->open_file_name);
if (cmp_translog_addr(rec->lsn, info->s->lsn_of_file_id) <= 0)
{
fprintf(tracef, ", table's LOGREC_FILE_ID has LSN (%lu,0x%lx) more recent"
" than record, skipping record",
LSN_IN_HEX(info->s->lsn_of_file_id));
return NULL;
}
DBUG_ASSERT(info->s->last_version != 0);
_ma_writeinfo(info, WRITEINFO_UPDATE_KEYFILE); /* to flush state on close */
fprintf(tracef, ", applying record\n");
return info;
}
static int parse_checkpoint_record(LSN lsn)
/**
@brief Parses checkpoint record.
Builds from it the dirty_pages list (a hash), opens tables and maps them to
their 2-byte IDs, recreates transactions (not real TRNs though).
@return From where in the log the REDO phase should start
@retval LSN_IMPOSSIBLE error
@retval other ok
*/
static LSN parse_checkpoint_record(LSN lsn)
{
uint i;
TRANSLOG_HEADER_BUFFER rec;
fprintf(tracef, "Loading data from checkpoint record\n");
fprintf(tracef, "Loading data from checkpoint record at LSN (%lu,0x%lx)\n",
LSN_IN_HEX(lsn));
int len= translog_read_record_header(lsn, &rec);
/** @todo EOF should be detected */
if (len == RECHEADER_READ_ERROR)
{
fprintf(tracef, "Cannot find checkpoint record where it should be\n");
return 1;
return LSN_IMPOSSIBLE;
}
enlarge_buffer(&rec);
......@@ -1521,7 +1631,7 @@ static int parse_checkpoint_record(LSN lsn)
rec.record_length)
{
fprintf(tracef, "Failed to read record\n");
return 1;
return LSN_IMPOSSIBLE;
}
char *ptr= log_record_buffer.str;
......@@ -1563,6 +1673,7 @@ static int parse_checkpoint_record(LSN lsn)
/* tables */
uint nb_tables= uint4korr(ptr);
ptr+= 4;
fprintf(tracef, "%u open tables\n", nb_tables);
for (i= 0; i< nb_tables; i++)
{
......@@ -1580,23 +1691,24 @@ static int parse_checkpoint_record(LSN lsn)
ptr+= name_len;
strnmov(name, ptr, sizeof(name));
if (new_table(sid, name, kfile, dfile, first_log_write_lsn))
return 1;
return LSN_IMPOSSIBLE;
}
/* dirty pages */
uint nb_dirty_pages= uint4korr(ptr);
ptr+= 4;
fprintf(tracef, "%u dirty pages\n", nb_dirty_pages);
if (hash_init(&all_dirty_pages, &my_charset_bin, nb_dirty_pages,
offsetof(struct st_dirty_page, file_and_page_id),
sizeof(((struct st_dirty_page *)NULL)->file_and_page_id),
NULL, NULL, 0))
return 1;
return LSN_IMPOSSIBLE;
dirty_pages_pool=
(struct st_dirty_page *)my_malloc(nb_dirty_pages *
sizeof(struct st_dirty_page),
MYF(MY_WME));
if (unlikely(dirty_pages_pool == NULL))
return 1;
return LSN_IMPOSSIBLE;
struct st_dirty_page *next_dirty_page_in_pool= dirty_pages_pool;
LSN minimum_rec_lsn_of_dirty_pages= LSN_MAX;
for (i= 0; i < nb_dirty_pages ; i++)
......@@ -1608,7 +1720,7 @@ static int parse_checkpoint_record(LSN lsn)
LSN rec_lsn= lsn_korr(ptr);
ptr+= LSN_STORE_SIZE;
if (new_page(fileid, pageid, rec_lsn, next_dirty_page_in_pool++))
return 1;
return LSN_IMPOSSIBLE;
set_if_smaller(minimum_rec_lsn_of_dirty_pages, rec_lsn);
}
/* after that, there will be no insert/delete into the hash */
......@@ -1627,11 +1739,11 @@ static int parse_checkpoint_record(LSN lsn)
if (ptr != (log_record_buffer.str + log_record_buffer.length))
{
fprintf(tracef, "checkpoint record corrupted\n");
return 1;
return LSN_IMPOSSIBLE;
}
set_if_smaller(checkpoint_start, minimum_rec_lsn_of_dirty_pages);
return 0;
return checkpoint_start;
}
static int new_page(File fileid, pgcache_page_no_t pageid, LSN rec_lsn,
......@@ -1656,9 +1768,9 @@ static int close_all_tables()
/*
Since the end of end_of_redo_phase(), we may have written new records
(if UNDO phase ran) and thus the state is newer than at
end_of_redo_phase(), we need to bump is_of_lsn again.
end_of_redo_phase(), we need to bump is_of_horizon again.
*/
LSN addr= translog_get_horizon();
TRANSLOG_ADDRESS addr= translog_get_horizon();
for (list_element= maria_open_list ; list_element ; list_element= next_open)
{
next_open= list_element->next;
......@@ -1674,14 +1786,14 @@ end:
}
#ifdef MARIA_EXTERNAL_LOCKING
#error Maria's Recovery is really not ready for it
#error Maria's Checkpoint and Recovery are really not ready for it
#endif
/*
Recovery of the state : how it works
=====================================
Ignoring Checkpoints for a start.
Here we ignore Checkpoints for a start.
The state (MARIA_HA::MARIA_SHARE::MARIA_STATE_INFO) is updated in
memory frequently (at least at every row write/update/delete) but goes
......@@ -1700,8 +1812,8 @@ the end of every row write/update/delete/delete_all. When Recovery sees the
sign of such row operation (UNDO or REDO), it may need to update the records'
count if that count does not reflect that operation (is older). How to know
the age of the state compared to the log record: every time the state
goes to disk at runtime, its member "is_of_lsn" is updated to the
current end-of-log LSN. So Recovery just needs to compare is_of_lsn
goes to disk at runtime, its member "is_of_horizon" is updated to the
current end-of-log horizon. So Recovery just needs to compare is_of_horizon
and the record's LSN to know if it should modify "records".
Other operations like ALTER TABLE DISABLE KEYS update the state but
......@@ -1738,7 +1850,7 @@ intern_lock (as Checkpoint needs it anyway to read MARIA_SHARE::kfile,
and as maria_close() takes it too). All state writes to disk are
changed to be protected with intern_lock.
So Checkpoint takes intern_lock, log's lock, reads "records" from
memory, releases log's lock, updates is_of_lsn and writes "records" to
memory, releases log's lock, updates is_of_horizon and writes "records" to
disk, release intern_lock.
In practice, not only "records" needs to be written but the full
state. So, Checkpoint reads the full state from memory. Some other
......@@ -1747,7 +1859,7 @@ state which are not protected by the lock's log (see ma_extra.c
HA_EXTRA_NO_KEYS), and Checkpoint would be reading a corrupted state
from memory; to guard against that we extend the intern_lock-zone to
changes done to the state in memory by HA_EXTRA_NO_KEYS et al, and
also any change made in memory to create_rename_lsn/state_is_of_lsn.
also any change made in memory to create_rename_lsn/state_is_of_horizon.
Last, we don't want in Checkpoint to do
log lock; read state from memory; release log lock;
for each table, it may hold the log's lock too much in total.
......@@ -1758,12 +1870,12 @@ But this re-introduces the problem that some other thread may be changing the
state in memory and on disk under intern_lock, without log's lock, like
HA_EXTRA_NO_KEYS, while we read the N states. However, when Checkpoint later
comes to handling the table under intern_lock, which is serialized with
HA_EXTRA_NO_KEYS, it can see that is_of_lsn is higher then when the state was
read from memory under log's lock, and thus can decide to not flush the
HA_EXTRA_NO_KEYS, it can see that is_of_horizon is higher then when the state
was read from memory under log's lock, and thus can decide to not flush the
obsolete state it has, knowing that the other thread flushed a more recent
state already. If on the other hand is_of_lsn is not higher, the read state is
current and can be flushed. So we have a per-table sequence:
lock intern_lock; test if is_of_lsn is higher than when we read the state
state already. If on the other hand is_of_horizon is not higher, the read
state is current and can be flushed. So we have a per-table sequence:
lock intern_lock; test if is_of_horizon is higher than when we read the state
under log's lock; if no then flush the read state to disk.
*/
......@@ -1794,7 +1906,6 @@ current and can be flushed. So we have a per-table sequence:
/**** UNDO PHASE *****/
print_information_to_error_log(nb of trans to roll back, nb of prepared trans
/*
Launch one or more threads to do the background rollback. Don't wait for
them to complete their rollback (background rollback; for debugging, we
......@@ -1815,33 +1926,4 @@ current and can be flushed. So we have a per-table sequence:
/*
mark that checkpoint requests are now allowed.
*/
}
pthread_handler_decl rollback_background_thread()
{
/*
execute the normal runtime-rollback code for a bunch of transactions.
*/
while (trans in list_of_trans_to_rollback_by_this_thread)
{
while (trans->undo_lsn != 0)
{
/* this is the normal runtime-rollback code: */
record= log_read_record(trans->undo_lsn);
execute_log_record_in_undo_phase(record);
trans->undo_lsn= record.prev_undo_lsn;
}
/* remove trans from list */
}
lock_mutex(rollback_threads); /* or atomic counter */
if (--total_of_rollback_threads == 0)
{
/*
All rollback threads are done. Print "rollback finished" to the error
log and take a full checkpoint.
*/
}
unlock_mutex(rollback_threads);
pthread_exit();
}
#endif
......@@ -67,17 +67,12 @@ int maria_rename(const char *old_name, const char *new_name)
if (sync_dir)
{
LSN lsn;
uchar log_data[2 + 2];
LEX_STRING log_array[TRANSLOG_INTERNAL_PARTS + 3];
uint old_name_len= strlen(old_name), new_name_len= strlen(new_name);
int2store(log_data, old_name_len);
int2store(log_data + 2, new_name_len);
log_array[TRANSLOG_INTERNAL_PARTS + 0].str= log_data;
log_array[TRANSLOG_INTERNAL_PARTS + 0].length= sizeof(log_data);
log_array[TRANSLOG_INTERNAL_PARTS + 1].str= (char *)old_name;
log_array[TRANSLOG_INTERNAL_PARTS + 1].length= old_name_len;
log_array[TRANSLOG_INTERNAL_PARTS + 2].str= (char *)new_name;
log_array[TRANSLOG_INTERNAL_PARTS + 2].length= new_name_len;
LEX_STRING log_array[TRANSLOG_INTERNAL_PARTS + 2];
uint old_name_len= strlen(old_name)+1, new_name_len= strlen(new_name)+1;
log_array[TRANSLOG_INTERNAL_PARTS + 0].str= (char *)old_name;
log_array[TRANSLOG_INTERNAL_PARTS + 0].length= old_name_len;
log_array[TRANSLOG_INTERNAL_PARTS + 1].str= (char *)new_name;
log_array[TRANSLOG_INTERNAL_PARTS + 1].length= new_name_len;
/*
For this record to be of any use for Recovery, we need the upper
MySQL layer to be crash-safe, which it is not now (that would require
......@@ -88,7 +83,7 @@ int maria_rename(const char *old_name, const char *new_name)
*/
if (unlikely(translog_write_record(&lsn, LOGREC_REDO_RENAME_TABLE,
&dummy_transaction_object, NULL,
2 + 2 + old_name_len + new_name_len,
old_name_len + new_name_len,
sizeof(log_array)/sizeof(log_array[0]),
log_array, NULL) ||
translog_flush(lsn)))
......@@ -100,7 +95,7 @@ int maria_rename(const char *old_name, const char *new_name)
store LSN into file, needed for Recovery to not be confused if a
RENAME happened (applying REDOs to the wrong table).
*/
if (_ma_update_create_rename_lsn_on_disk(share, lsn, TRUE))
if (_ma_update_create_rename_lsn(share, lsn, TRUE))
{
maria_close(info);
DBUG_RETURN(1);
......
......@@ -25,6 +25,7 @@
#define SAFEMALLOC
#endif
#include "maria_def.h"
#include "trnman.h"
#include <m_ctype.h>
#include <my_bit.h>
......@@ -47,7 +48,8 @@ static void copy_key(struct st_maria_info *info,uint inx,
static int verbose=0,testflag=0,
first_key=0,async_io=0,pagecacheing=0,write_cacheing=0,locking=0,
rec_pointer_size=0,pack_fields=1,silent=0,
opt_quick_mode=0, transactional= 0, skip_update= 0;
opt_quick_mode=0, transactional= 0, skip_update= 0,
die_in_middle_of_transaction= 0;
static int pack_seg=HA_SPACE_PACK,pack_type=HA_PACK_KEY,remove_count=-1;
static int create_flag= 0, srand_arg= 0;
static ulong pagecache_size=IO_SIZE*16;
......@@ -235,6 +237,9 @@ int main(int argc, char *argv[])
goto err;
if (!(file=maria_open(filename,2,HA_OPEN_ABORT_IF_LOCKED)))
goto err;
maria_begin(file);
if (testflag == 1)
goto end;
if (!silent)
printf("- Writing key:s\n");
if (locking)
......@@ -244,8 +249,6 @@ int main(int argc, char *argv[])
if (opt_quick_mode)
maria_extra(file,HA_EXTRA_QUICK,0);
maria_begin(file);
for (i=0 ; i < recant ; i++)
{
ulong blob_length;
......@@ -297,7 +300,7 @@ int main(int argc, char *argv[])
}
}
}
if (testflag == 1)
if (testflag == 2)
goto end;
if (write_cacheing)
......@@ -348,7 +351,7 @@ int main(int argc, char *argv[])
else
puts("Warning: Skipping delete test because no dupplicate keys");
}
if (testflag == 2)
if (testflag == 3)
goto end;
if (!silent)
......@@ -409,7 +412,7 @@ int main(int argc, char *argv[])
}
}
}
if (testflag == 3)
if (testflag == 4)
goto end;
for (i=999, dupp_keys=j=0 ; i>0 ; i--)
......@@ -814,7 +817,7 @@ int main(int argc, char *argv[])
goto err;
}
if (testflag == 4)
if (testflag == 5)
goto end;
if (!silent)
......@@ -892,6 +895,36 @@ int main(int argc, char *argv[])
goto err;
}
end:
if (die_in_middle_of_transaction)
{
/* As commit record is not done, UNDO entries needs to be rolled back */
switch (die_in_middle_of_transaction) {
case 1:
/*
Flush changed pages go to disk. That will also flush log. Recovery
will skip REDOs and apply UNDOs.
*/
_ma_flush_table_files(file, MARIA_FLUSH_DATA, FLUSH_RELEASE,
FLUSH_RELEASE);
break;
case 2:
/*
Just flush log. Pages are likely to not be on disk. Recovery will
then execute REDOs and UNDOs.
*/
if (translog_flush(file->trn->undo_lsn))
goto err;
break;
case 3:
/*
Flush nothing. Pages and log are likely to not be on disk. Recovery
will then do nothing.
*/
break;
}
printf("Dying on request without maria_commit()/maria_close()\n");
exit(0);
}
if (maria_commit(file))
goto err;
if (maria_close(file))
......@@ -998,9 +1031,9 @@ static void get_options(int argc, char **argv)
verbose=1;
break;
case 'm': /* records */
if ((recant=atoi(++pos)) < 10 && testflag > 1)
if ((recant=atoi(++pos)) < 10 && testflag > 2)
{
fprintf(stderr,"record count must be >= 10 (if testflag != 1)\n");
fprintf(stderr,"record count must be >= 10 (if testflag > 2)\n");
exit(1);
}
break;
......@@ -1048,6 +1081,9 @@ static void get_options(int argc, char **argv)
case 'T':
transactional= 1;
break;
case 'u':
die_in_middle_of_transaction= atoi(++pos);
break;
case 'q':
opt_quick_mode=1;
break;
......
......@@ -131,7 +131,7 @@ do
for test_undo in 1 2 3
do
# first iteration tests rollback of insert, second tests rollback of delete
set -- "ma_test1 $silent -M -T -c -N $blobs" "--testflag=1" "--testflag=2" "ma_test1 $silent -M -T -c -N --debug=d:t:i:o,/tmp/ma_test1.trace $blobs" "--testflag=3" "--testflag=4" "ma_test1 $silent -M -T -c -N --debug=d:t:i:o,/tmp/ma_test1.trace $blobs" "--testflag=2" "--testflag=3"
set -- "ma_test1 $silent -M -T -c -N $blobs" "--testflag=1" "--testflag=2 --test-undo=" "ma_test1 $silent -M -T -c -N --debug=d:t:i:o,/tmp/ma_test1.trace $blobs" "--testflag=3" "--testflag=4 --test-undo=" "ma_test1 $silent -M -T -c -N --debug=d:t:i:o,/tmp/ma_test1.trace $blobs" "--testflag=2" "--testflag=3 --test-undo=" "ma_test2 $silent -L -K -W -P -M -T -c $blobs" "-t1" "-t2 -u"
# -N (create NULL fields) is needed because --test-undo adds it anyway
while [ $# != 0 ]
do
......@@ -148,8 +148,8 @@ do
mv $table.MAD $tmp/$table.MAD.good
rm $table.MAI
rm maria_log.* maria_log_control
echo "TEST WITH $prog $abort_run_args --test-undo=$test_undo (additional aborted work)"
$maria_path/$prog $abort_run_args --test-undo=$test_undo
echo "TEST WITH $prog $abort_run_args$test_undo (additional aborted work)"
$maria_path/$prog $abort_run_args$test_undo
cp $table.MAD $tmp/$table.MAD.before_undo
if [ $test_undo -lt 3 ]
then
......@@ -174,7 +174,7 @@ do
echo "testing applying of CLRs to recreate table"
rm $table.MA?
apply_log "shouldnotchangelog"
# the cmp below fails with blobs! @todo RECOVERY BUG find out why.
# the cmp below fails with ma_test1+blobs! @todo RECOVERY BUG why?
# It is probably serious; REDOs shouldn't place rows in different
# positions from what the run-time code did. Indeed it may lead to
# more or less free space...
......@@ -189,12 +189,15 @@ do
check_table_is_same
shift 3
done
rm -f $table.* $tmp/$table* $tmp/maria_chk_*.txt $tmp/maria_read_log_$table.txt
done
done
rm -f $table.* $tmp/$table* $tmp/maria_chk_*.txt $tmp/maria_read_log_$table.txt
) 2>&1 > $tmp/ma_test_recovery.output
# also note that maria_chk -dvv shows differences for ma_test2 in UNDO phase,
# this is normal: removing records does not shrink the data/key file,
# does not put back the "analyzed,optimized keys"(etc) index state.
diff $maria_path/ma_test_recovery.expected $tmp/ma_test_recovery.output > /dev/null || diff_failed=1
if [ "$diff_failed" == "1" ]
then
......
......@@ -205,6 +205,47 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique number NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t2 -u1 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 8192
========DIFF END=======
TEST WITH ma_test1 -s -M -T -c -N --testflag=1 (commit at end)
TEST WITH ma_test1 -s -M -T -c -N --testflag=2 --test-undo=2 (additional aborted work)
Terminating after inserts
......@@ -339,6 +380,47 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique number NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t2 -u2 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 8192
========DIFF END=======
TEST WITH ma_test1 -s -M -T -c -N --testflag=1 (commit at end)
TEST WITH ma_test1 -s -M -T -c -N --testflag=2 --test-undo=3 (additional aborted work)
Terminating after inserts
......@@ -473,6 +555,47 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique number NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -t2 -u3 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 204800
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 90112 Keyfile length: 8192
========DIFF END=======
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=1 (commit at end)
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=2 --test-undo=1 (additional aborted work)
Terminating after inserts
......@@ -607,6 +730,47 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique varchar BLOB NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t2 -u1 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 8192
========DIFF END=======
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=1 (commit at end)
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=2 --test-undo=2 (additional aborted work)
Terminating after inserts
......@@ -741,6 +905,47 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique varchar BLOB NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t2 -u2 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 8192
========DIFF END=======
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=1 (commit at end)
TEST WITH ma_test1 -s -M -T -c -N -b --testflag=2 --test-undo=3 (additional aborted work)
Terminating after inserts
......@@ -875,3 +1080,44 @@ Differences in maria_chk -dvv, recovery not yet perfect !
---
> 1 2 6 unique varchar BLOB NULL 0 8192
========DIFF END=======
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t1 (commit at end)
TEST WITH ma_test2 -s -L -K -W -P -M -T -c -b -t2 -u3 (additional aborted work)
Dying on request without maria_commit()/maria_close()
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing idempotency
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 212992
========DIFF END=======
testing applying of CLRs to recreate table
applying log
Differences in maria_chk -dvv, recovery not yet perfect !
========DIFF START=======
6c6
< Status: checked,analyzed,optimized keys,sorted index pages
---
> Status: changed
11c11
< Datafile length: 8192 Keyfile length: 8192
---
> Datafile length: 81920 Keyfile length: 8192
========DIFF END=======
......@@ -222,6 +222,12 @@ err:
maria_flush_bulk_insert(info, j);
}
info->errkey= (int) i;
/*
We delete keys in the reverse order of insertion. This is the order that
a rollback would do and is important for CLR_ENDs generated by
_ma_ft|ck_delete() and write_record_abort() to work (with any other
order they would cause wrong jumps in the chain).
*/
while ( i-- > 0)
{
if (maria_is_key_active(share->state.key_map, i))
......@@ -231,6 +237,10 @@ err:
is_tree_inited(&info->bulk_insert[i])));
if (local_lock_tree)
rw_wrlock(&share->key_root_lock[i]);
/**
@todo RECOVERY BUG
The key deletes below should generate CLR_ENDs
*/
if (share->keyinfo[i].flag & HA_FULLTEXT)
{
if (_ma_ft_del(info,i,(char*) buff,record,filepos))
......
......@@ -1033,9 +1033,11 @@ static int maria_chk(HA_CHECK *param, char *filename)
Tell the server's Recovery to ignore old REDOs on this table; we don't
know what the log's end LSN is now, so we just let the server know
that it will have to find and store it.
This is the only case where create_rename_lsn can be a horizon and not
a LSN.
*/
if (share->base.born_transactional)
share->state.create_rename_lsn= share->state.is_of_lsn=
share->state.create_rename_lsn= share->state.is_of_horizon=
LSN_REPAIRED_BY_MARIA_CHK;
if ((param->testflag & (T_REP_BY_SORT | T_REP_PARALLEL)) &&
(maria_is_any_key_active(share->state.key_map) ||
......
......@@ -96,7 +96,8 @@ typedef struct st_maria_state_info
uint open_count;
uint8 changed; /* Changed since mariachk */
LSN create_rename_lsn; /**< LSN when table was last created/renamed */
LSN is_of_lsn; /**< LSN when state was last updated on disk */
/** @brief Log horizon when state was last updated on disk */
TRANSLOG_ADDRESS is_of_horizon;
/* the following isn't saved on disk */
uint state_diff_length; /* Should be 0 */
......@@ -218,6 +219,7 @@ typedef struct st_maria_file_bitmap
#define MARIA_CHECKPOINT_LOOKS_AT_ME 1
#define MARIA_CHECKPOINT_SHOULD_FREE_ME 2
#define MARIA_CHECKPOINT_SEEN_IN_LOOP 4
typedef struct st_maria_share
{ /* Shared between opens */
......@@ -331,6 +333,7 @@ typedef struct st_maria_share
non-mmaped area */
MARIA_FILE_BITMAP bitmap;
rw_lock_t mmap_lock;
LSN lsn_of_file_id; /**< LSN of its last LOGREC_FILE_ID */
} MARIA_SHARE;
......@@ -940,9 +943,9 @@ int _ma_create_index_by_sort(MARIA_SORT_PARAM *info, my_bool no_messages,
ulong);
int _ma_sync_table_files(const MARIA_HA *info);
int _ma_initialize_data_file(MARIA_SHARE *share, File dfile);
int _ma_update_create_rename_lsn_on_disk(MARIA_SHARE *share,
int _ma_update_create_rename_lsn(MARIA_SHARE *share,
LSN lsn, my_bool do_sync);
int _ma_update_create_rename_lsn_on_disk_sub(MARIA_SHARE *share,
int _ma_update_create_rename_lsn_sub(MARIA_SHARE *share,
LSN lsn, my_bool do_sync);
void _ma_unpin_all_pages(MARIA_HA *info, LSN undo_lsn);
......
......@@ -18,6 +18,7 @@
#include <my_sys.h>
#include <m_string.h>
#include "trnman.h"
#include "ma_checkpoint.h"
#include "ma_control_file.h"
/*
......@@ -587,27 +588,25 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
TRN *trn;
char *ptr;
uint stored_transactions= 0;
LSN minimum_rec_lsn= ULONGLONG_MAX, minimum_first_undo_lsn= ULONGLONG_MAX;
LSN minimum_rec_lsn= LSN_MAX, minimum_first_undo_lsn= LSN_MAX;
DBUG_ENTER("trnman_collect_transactions");
DBUG_ASSERT((NULL == str_act->str) && (NULL == str_com->str));
/* validate the use of read_non_atomic() in general: */
compile_time_assert((sizeof(LSN) == 8) && (sizeof(LSN_WITH_FLAGS) == 8));
DBUG_PRINT("info", ("pthread_mutex_lock LOCK_trn_list"));
pthread_mutex_lock(&LOCK_trn_list);
str_act->length= 2 + /* number of active transactions */
LSN_STORE_SIZE + /* minimum of their rec_lsn */
(6 + /* long id */
2 + /* short id */
(2 + /* short id */
6 + /* long id */
LSN_STORE_SIZE + /* undo_lsn */
#ifdef MARIA_VERSIONING /* not enabled yet */
LSN_STORE_SIZE + /* undo_purge_lsn */
#endif
LSN_STORE_SIZE /* first_undo_lsn */
) * trnman_active_transactions;
str_com->length= 8 + /* number of committed transactions */
str_com->length= 4 + /* number of committed transactions */
(6 + /* long id */
#ifdef MARIA_VERSIONING /* not enabled yet */
LSN_STORE_SIZE + /* undo_purge_lsn */
......@@ -638,13 +637,6 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
*/
continue;
}
#ifndef MARIA_CHECKPOINT
/*
in the checkpoint patch (not yet ready) we will have a real implementation
of lsn_read_non_atomic(); for now it's not needed
*/
#define lsn_read_non_atomic(A) (A)
#endif
/* needed for low-water mark calculation */
if (((rec_lsn= lsn_read_non_atomic(trn->rec_lsn)) > 0) &&
(cmp_translog_addr(rec_lsn, minimum_rec_lsn) < 0))
......@@ -656,23 +648,23 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
if ((undo_lsn= trn->undo_lsn) == 0) /* trn can be forgotten */
continue;
stored_transactions++;
int6store(ptr, trn->trid);
ptr+= 6;
int2store(ptr, sid);
ptr+= 2;
int6store(ptr, trn->trid);
ptr+= 6;
lsn_store(ptr, undo_lsn); /* needed for rollback */
ptr+= LSN_STORE_SIZE;
#ifdef MARIA_VERSIONING /* not enabled yet */
/* to know where purging should start (last delete of this trn) */
lsn_store(ptr, trn->undo_purge_lsn);
ptr+= LSN_STORE_SIZE;
#endif
/* needed for low-water mark calculation */
if (((first_undo_lsn= lsn_read_non_atomic(trn->first_undo_lsn)) > 0) &&
(cmp_translog_addr(first_undo_lsn, minimum_first_undo_lsn) < 0))
minimum_first_undo_lsn= first_undo_lsn;
lsn_store(ptr, first_undo_lsn);
ptr+= LSN_STORE_SIZE;
#ifdef MARIA_VERSIONING /* not enabled yet */
/* to know where purging should start (last delete of this trn) */
lsn_store(ptr, trn->undo_purge_lsn);
ptr+= LSN_STORE_SIZE;
#endif
/**
@todo RECOVERY: add a comment explaining why we can dirtily read some
vars, inspired by the text of "assumption 8" in WL#3072
......@@ -680,6 +672,8 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
}
str_act->length= ptr - str_act->str; /* as we maybe over-estimated */
ptr= str_act->str;
DBUG_PRINT("info",("collected %u active transactions",
(uint)stored_transactions));
int2store(ptr, stored_transactions);
ptr+= 2;
/* this LSN influences how REDOs for any page can be ignored by Recovery */
......@@ -687,8 +681,10 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
/* one day there will also be a list of prepared transactions */
/* do the same for committed ones */
ptr= str_com->str;
int8store(ptr, (ulonglong)trnman_committed_transactions);
ptr+= 8;
int4store(ptr, trnman_committed_transactions);
ptr+= 4;
DBUG_PRINT("info",("collected %u committed transactions",
(uint)trnman_committed_transactions));
for (trn= committed_list_min.next; trn != &committed_list_max;
trn= trn->next)
{
......@@ -716,7 +712,6 @@ my_bool trnman_collect_transactions(LEX_STRING *str_act, LEX_STRING *str_com,
err:
error= 1;
end:
DBUG_PRINT("info", ("pthread_mutex_unlock LOCK_trn_list"));
pthread_mutex_unlock(&LOCK_trn_list);
DBUG_RETURN(error);
}
......
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