2021-02-15 15:36:25 +00:00
# include <Coordination/Changelog.h>
# include <IO/WriteHelpers.h>
# include <IO/ReadHelpers.h>
2021-02-15 17:59:40 +00:00
# include <IO/ReadBufferFromFile.h>
2021-02-15 15:36:25 +00:00
# include <filesystem>
2021-02-15 17:59:40 +00:00
# include <boost/algorithm/string/split.hpp>
# include <boost/algorithm/string/join.hpp>
# include <boost/algorithm/string/trim.hpp>
2021-02-18 10:23:48 +00:00
# include <Common/Exception.h>
2021-02-20 15:36:56 +00:00
# include <Common/SipHash.h>
2021-02-18 10:23:48 +00:00
# include <common/logger_useful.h>
2021-02-15 15:36:25 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int CHECKSUM_DOESNT_MATCH ;
extern const int CORRUPTED_DATA ;
extern const int UNKNOWN_FORMAT_VERSION ;
extern const int LOGICAL_ERROR ;
}
namespace
{
2021-02-18 10:23:48 +00:00
constexpr auto DEFAULT_PREFIX = " changelog " ;
2021-02-15 15:36:25 +00:00
2021-02-17 09:00:12 +00:00
std : : string formatChangelogPath ( const std : : string & prefix , const ChangelogFileDescription & name )
2021-02-15 15:36:25 +00:00
{
std : : filesystem : : path path ( prefix ) ;
2021-02-20 11:04:38 +00:00
path / = std : : filesystem : : path ( name . prefix + " _ " + std : : to_string ( name . from_log_index ) + " _ " + std : : to_string ( name . to_log_index ) + " .bin " ) ;
2021-02-15 17:59:40 +00:00
return path ;
2021-02-15 15:36:25 +00:00
}
2021-02-17 09:00:12 +00:00
ChangelogFileDescription getChangelogFileDescription ( const std : : string & path_str )
2021-02-15 15:36:25 +00:00
{
std : : filesystem : : path path ( path_str ) ;
2021-02-15 17:59:40 +00:00
std : : string filename = path . stem ( ) ;
2021-02-15 15:36:25 +00:00
Strings filename_parts ;
boost : : split ( filename_parts , filename , boost : : is_any_of ( " _ " ) ) ;
2021-02-16 17:28:54 +00:00
if ( filename_parts . size ( ) < 3 )
2021-02-15 15:36:25 +00:00
throw Exception ( ErrorCodes : : CORRUPTED_DATA , " Invalid changelog {} " , path_str ) ;
2021-02-17 09:00:12 +00:00
ChangelogFileDescription result ;
2021-02-15 15:36:25 +00:00
result . prefix = filename_parts [ 0 ] ;
2021-04-08 14:17:57 +00:00
result . from_log_index = parse < uint64_t > ( filename_parts [ 1 ] ) ;
result . to_log_index = parse < uint64_t > ( filename_parts [ 2 ] ) ;
2021-02-17 09:00:12 +00:00
result . path = path_str ;
2021-02-15 15:36:25 +00:00
return result ;
}
2021-02-15 17:59:40 +00:00
LogEntryPtr makeClone ( const LogEntryPtr & entry )
{
return cs_new < nuraft : : log_entry > ( entry - > get_term ( ) , nuraft : : buffer : : clone ( entry - > get_buf ( ) ) , entry - > get_val_type ( ) ) ;
}
2021-02-20 11:04:38 +00:00
Checksum computeRecordChecksum ( const ChangelogRecord & record )
{
2021-02-20 15:36:56 +00:00
SipHash hash ;
hash . update ( record . header . version ) ;
hash . update ( record . header . index ) ;
hash . update ( record . header . term ) ;
hash . update ( record . header . value_type ) ;
hash . update ( record . header . blob_size ) ;
2021-02-20 11:04:38 +00:00
if ( record . header . blob_size ! = 0 )
2021-02-20 15:36:56 +00:00
hash . update ( reinterpret_cast < char * > ( record . blob - > data_begin ( ) ) , record . blob - > size ( ) ) ;
return hash . get64 ( ) ;
2021-02-20 11:04:38 +00:00
}
2021-02-15 15:36:25 +00:00
}
class ChangelogWriter
{
public :
2021-04-08 14:17:57 +00:00
ChangelogWriter ( const std : : string & filepath_ , WriteMode mode , uint64_t start_index_ )
2021-02-15 15:36:25 +00:00
: filepath ( filepath_ )
, plain_buf ( filepath , DBMS_DEFAULT_BUFFER_SIZE , mode = = WriteMode : : Rewrite ? - 1 : ( O_APPEND | O_CREAT | O_WRONLY ) )
, start_index ( start_index_ )
{ }
2021-04-16 13:50:09 +00:00
off_t appendRecord ( ChangelogRecord & & record )
2021-02-15 15:36:25 +00:00
{
off_t result = plain_buf . count ( ) ;
2021-02-20 11:04:38 +00:00
writeIntBinary ( computeRecordChecksum ( record ) , plain_buf ) ;
2021-02-20 15:36:56 +00:00
writeIntBinary ( record . header . version , plain_buf ) ;
writeIntBinary ( record . header . index , plain_buf ) ;
writeIntBinary ( record . header . term , plain_buf ) ;
writeIntBinary ( record . header . value_type , plain_buf ) ;
writeIntBinary ( record . header . blob_size , plain_buf ) ;
2021-02-15 15:36:25 +00:00
2021-02-15 17:59:40 +00:00
if ( record . header . blob_size ! = 0 )
2021-02-15 15:36:25 +00:00
plain_buf . write ( reinterpret_cast < char * > ( record . blob - > data_begin ( ) ) , record . blob - > size ( ) ) ;
entries_written + + ;
2021-02-15 17:59:40 +00:00
return result ;
2021-02-15 15:36:25 +00:00
}
void truncateToLength ( off_t new_length )
{
2021-04-16 13:50:09 +00:00
plain_buf . next ( ) ;
2021-02-15 15:36:25 +00:00
plain_buf . truncate ( new_length ) ;
2021-02-16 17:28:54 +00:00
plain_buf . seek ( new_length , SEEK_SET ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-16 13:50:09 +00:00
void flush ( bool force_fsync )
2021-02-15 15:36:25 +00:00
{
2021-04-16 13:50:09 +00:00
plain_buf . next ( ) ;
if ( force_fsync )
plain_buf . sync ( ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-08 14:17:57 +00:00
uint64_t getEntriesWritten ( ) const
2021-02-15 15:36:25 +00:00
{
return entries_written ;
}
2021-04-08 14:17:57 +00:00
void setEntriesWritten ( uint64_t entries_written_ )
2021-02-15 15:36:25 +00:00
{
entries_written = entries_written_ ;
}
2021-04-08 14:17:57 +00:00
uint64_t getStartIndex ( ) const
2021-02-15 15:36:25 +00:00
{
return start_index ;
}
2021-04-08 14:17:57 +00:00
void setStartIndex ( uint64_t start_index_ )
2021-02-15 15:36:25 +00:00
{
start_index = start_index_ ;
}
private :
std : : string filepath ;
WriteBufferFromFile plain_buf ;
2021-04-08 14:17:57 +00:00
uint64_t entries_written = 0 ;
uint64_t start_index ;
2021-02-15 15:36:25 +00:00
} ;
2021-02-18 11:42:09 +00:00
struct ChangelogReadResult
{
2021-04-08 14:17:57 +00:00
uint64_t entries_read ;
uint64_t first_read_index ;
2021-02-18 11:42:09 +00:00
off_t last_position ;
bool error ;
} ;
2021-02-15 15:36:25 +00:00
class ChangelogReader
{
public :
explicit ChangelogReader ( const std : : string & filepath_ )
: filepath ( filepath_ )
, read_buf ( filepath )
{ }
2021-04-08 14:17:57 +00:00
ChangelogReadResult readChangelog ( IndexToLogEntry & logs , uint64_t start_log_index , IndexToOffset & index_to_offset , Poco : : Logger * log )
2021-02-15 15:36:25 +00:00
{
2021-04-08 14:17:57 +00:00
uint64_t previous_index = 0 ;
2021-02-18 11:42:09 +00:00
ChangelogReadResult result { } ;
2021-02-18 10:23:48 +00:00
try
2021-02-15 15:36:25 +00:00
{
2021-02-18 10:23:48 +00:00
while ( ! read_buf . eof ( ) )
2021-02-15 15:36:25 +00:00
{
2021-02-18 11:42:09 +00:00
result . last_position = read_buf . count ( ) ;
2021-02-20 11:04:38 +00:00
Checksum record_checksum ;
readIntBinary ( record_checksum , read_buf ) ;
2021-02-20 15:36:56 +00:00
/// Initialization is required, otherwise checksums may fail
2021-02-18 10:23:48 +00:00
ChangelogRecord record ;
2021-02-20 15:36:56 +00:00
readIntBinary ( record . header . version , read_buf ) ;
readIntBinary ( record . header . index , read_buf ) ;
readIntBinary ( record . header . term , read_buf ) ;
readIntBinary ( record . header . value_type , read_buf ) ;
readIntBinary ( record . header . blob_size , read_buf ) ;
2021-02-20 11:04:38 +00:00
if ( record . header . version > CURRENT_CHANGELOG_VERSION )
throw Exception ( ErrorCodes : : UNKNOWN_FORMAT_VERSION , " Unsupported changelog version {} on path {} " , record . header . version , filepath ) ;
if ( record . header . blob_size ! = 0 )
{
auto buffer = nuraft : : buffer : : alloc ( record . header . blob_size ) ;
auto * buffer_begin = reinterpret_cast < char * > ( buffer - > data_begin ( ) ) ;
read_buf . readStrict ( buffer_begin , record . header . blob_size ) ;
record . blob = buffer ;
}
else
record . blob = nullptr ;
2021-02-18 11:42:09 +00:00
if ( previous_index ! = 0 & & previous_index + 1 ! = record . header . index )
throw Exception ( ErrorCodes : : CORRUPTED_DATA , " Previous log entry {}, next log entry {}, seems like some entries skipped " , previous_index , record . header . index ) ;
previous_index = record . header . index ;
2021-02-18 10:23:48 +00:00
2021-02-20 11:04:38 +00:00
Checksum checksum = computeRecordChecksum ( record ) ;
if ( checksum ! = record_checksum )
2021-02-18 10:23:48 +00:00
{
throw Exception ( ErrorCodes : : CHECKSUM_DOESNT_MATCH ,
" Checksums doesn't match for log {} (version {}), index {}, blob_size {} " ,
filepath , record . header . version , record . header . index , record . header . blob_size ) ;
}
if ( logs . count ( record . header . index ) ! = 0 )
throw Exception ( ErrorCodes : : CORRUPTED_DATA , " Duplicated index id {} in log {} " , record . header . index , filepath ) ;
2021-02-18 11:42:09 +00:00
result . entries_read + = 1 ;
2021-02-18 10:23:48 +00:00
2021-02-20 11:04:38 +00:00
if ( record . header . index < start_log_index )
2021-03-02 15:58:02 +00:00
{
2021-02-18 10:23:48 +00:00
continue ;
2021-03-02 15:58:02 +00:00
}
2021-02-18 10:23:48 +00:00
2021-02-20 11:04:38 +00:00
auto log_entry = nuraft : : cs_new < nuraft : : log_entry > ( record . header . term , record . blob , record . header . value_type ) ;
2021-03-04 11:22:59 +00:00
if ( result . first_read_index = = 0 )
result . first_read_index = record . header . index ;
2021-02-18 10:23:48 +00:00
logs . emplace ( record . header . index , log_entry ) ;
2021-02-18 11:42:09 +00:00
index_to_offset [ record . header . index ] = result . last_position ;
2021-02-20 21:08:19 +00:00
if ( result . entries_read % 50000 = = 0 )
LOG_TRACE ( log , " Reading changelog from path {}, entries {} " , filepath , result . entries_read ) ;
2021-02-15 15:36:25 +00:00
}
2021-02-18 10:23:48 +00:00
}
catch ( const Exception & ex )
{
2021-02-20 11:04:38 +00:00
if ( ex . code ( ) = = ErrorCodes : : UNKNOWN_FORMAT_VERSION )
throw ex ;
2021-02-18 11:42:09 +00:00
result . error = true ;
2021-02-20 11:04:38 +00:00
LOG_WARNING ( log , " Cannot completely read changelog on path {}, error: {} " , filepath , ex . message ( ) ) ;
2021-02-18 10:23:48 +00:00
}
catch ( . . . )
{
2021-02-18 11:42:09 +00:00
result . error = true ;
2021-02-20 11:04:38 +00:00
tryLogCurrentException ( log ) ;
2021-02-15 15:36:25 +00:00
}
2021-02-20 21:08:19 +00:00
LOG_TRACE ( log , " Totally read from changelog {} {} entries " , filepath , result . entries_read ) ;
2021-02-16 17:28:54 +00:00
2021-02-18 11:42:09 +00:00
return result ;
2021-02-15 15:36:25 +00:00
}
2021-02-18 10:23:48 +00:00
2021-02-15 15:36:25 +00:00
private :
std : : string filepath ;
ReadBufferFromFile read_buf ;
} ;
2021-04-16 13:50:09 +00:00
Changelog : : Changelog (
const std : : string & changelogs_dir_ ,
uint64_t rotate_interval_ ,
bool force_sync_ ,
Poco : : Logger * log_ )
2021-02-15 17:59:40 +00:00
: changelogs_dir ( changelogs_dir_ )
2021-02-15 15:36:25 +00:00
, rotate_interval ( rotate_interval_ )
2021-04-16 13:50:09 +00:00
, force_sync ( force_sync_ )
2021-02-20 11:04:38 +00:00
, log ( log_ )
2021-02-15 15:36:25 +00:00
{
namespace fs = std : : filesystem ;
2021-02-16 19:02:18 +00:00
if ( ! fs : : exists ( changelogs_dir ) )
fs : : create_directories ( changelogs_dir ) ;
for ( const auto & p : fs : : directory_iterator ( changelogs_dir ) )
2021-02-16 17:28:54 +00:00
{
2021-02-17 09:00:12 +00:00
auto file_description = getChangelogFileDescription ( p . path ( ) ) ;
2021-02-20 11:04:38 +00:00
existing_changelogs [ file_description . from_log_index ] = file_description ;
2021-02-16 17:28:54 +00:00
}
2021-02-15 15:36:25 +00:00
}
2021-04-08 14:17:57 +00:00
void Changelog : : readChangelogAndInitWriter ( uint64_t last_commited_log_index , uint64_t logs_to_keep )
2021-02-15 15:36:25 +00:00
{
2021-04-08 14:17:57 +00:00
uint64_t total_read = 0 ;
uint64_t entries_in_last = 0 ;
2021-08-25 18:11:52 +00:00
int64_t incomplete_log_index = - 1 ;
2021-02-18 11:42:09 +00:00
ChangelogReadResult result { } ;
2021-08-25 18:11:52 +00:00
/// First read log_id
2021-04-08 14:17:57 +00:00
uint64_t first_read_index = 0 ;
2021-03-04 11:22:59 +00:00
2021-08-25 18:11:52 +00:00
/// We must start to read from this log index
2021-04-08 14:17:57 +00:00
uint64_t start_to_read_from = last_commited_log_index ;
2021-08-25 18:11:52 +00:00
/// If we need to have some reserved log read additional `logs_to_keep` logs
2021-03-04 11:22:59 +00:00
if ( start_to_read_from > logs_to_keep )
start_to_read_from - = logs_to_keep ;
else
start_to_read_from = 1 ;
2021-02-20 11:04:38 +00:00
2021-02-22 20:51:58 +00:00
bool started = false ;
2021-08-25 18:11:52 +00:00
/// Got through changelog files in order op start_index
2021-02-20 15:36:56 +00:00
for ( const auto & [ changelog_start_index , changelog_description ] : existing_changelogs )
2021-02-15 15:36:25 +00:00
{
2021-08-25 18:11:52 +00:00
/// How many entries we have in the last changelog
entries_in_last = changelog_description . expectedEntriesCountInLog ( ) ;
2021-02-17 09:00:12 +00:00
2021-08-25 18:11:52 +00:00
/// [from_log_index.>=.......start_to_read_from.....<=.to_log_index]
2021-03-04 11:22:59 +00:00
if ( changelog_description . to_log_index > = start_to_read_from )
2021-02-15 15:36:25 +00:00
{
2021-08-25 18:11:52 +00:00
if ( ! started ) /// still nothing was read
2021-02-22 20:51:58 +00:00
{
2021-08-25 18:11:52 +00:00
/// Our first log starts from the more fresh log_id than we required to read and this changelog is not empty log.
/// So we are missing something in our logs, but it's not dataloss, we will receive snapshot and required
/// entries fro leader.
2021-03-04 12:01:56 +00:00
if ( changelog_description . from_log_index > last_commited_log_index & & ( changelog_description . from_log_index - last_commited_log_index ) > 1 )
{
2021-03-04 12:31:05 +00:00
LOG_ERROR ( log , " Some records was lost, last committed log index {}, smallest available log index on disk {}. Hopefully will receive missing records from leader. " , last_commited_log_index , changelog_description . from_log_index ) ;
2021-03-04 12:01:56 +00:00
incomplete_log_index = changelog_start_index ;
break ;
}
else if ( changelog_description . from_log_index > start_to_read_from )
2021-08-25 18:11:52 +00:00
{
/// We don't have required amount of reserved logs, but nothing was lost.
2021-05-28 11:52:19 +00:00
LOG_WARNING ( log , " Don't have required amount of reserved log records. Need to read from {}, smallest available log index on disk {}. " , start_to_read_from , changelog_description . from_log_index ) ;
2021-08-25 18:11:52 +00:00
}
2021-02-22 20:51:58 +00:00
}
2021-03-04 12:01:56 +00:00
2021-02-17 09:00:12 +00:00
ChangelogReader reader ( changelog_description . path ) ;
2021-03-04 11:22:59 +00:00
result = reader . readChangelog ( logs , start_to_read_from , index_to_start_pos , log ) ;
2021-08-25 18:11:52 +00:00
started = true ;
/// Otherwise we have already initialized it
2021-03-04 11:22:59 +00:00
if ( first_read_index = = 0 )
first_read_index = result . first_read_index ;
2021-02-18 11:42:09 +00:00
total_read + = result . entries_read ;
2021-02-17 09:00:12 +00:00
2021-02-20 11:04:38 +00:00
/// May happen after truncate, crash or simply unfinished log
2021-02-18 11:42:09 +00:00
if ( result . entries_read < entries_in_last )
2021-02-17 09:00:12 +00:00
{
2021-02-20 15:36:56 +00:00
incomplete_log_index = changelog_start_index ;
2021-02-17 09:00:12 +00:00
break ;
}
2021-02-15 15:36:25 +00:00
}
}
2021-02-15 17:59:40 +00:00
2021-03-04 11:22:59 +00:00
if ( first_read_index ! = 0 )
start_index = first_read_index ;
2021-08-25 18:11:52 +00:00
else /// We just may have no logs (only snapshot)
2021-03-04 11:22:59 +00:00
start_index = last_commited_log_index ;
2021-03-03 11:10:24 +00:00
2021-08-25 18:11:52 +00:00
if ( incomplete_log_index ! = - 1 ) /// otherwise all logs completed so just start a new one
2021-02-15 15:36:25 +00:00
{
2021-03-04 12:01:56 +00:00
auto start_remove_from = existing_changelogs . begin ( ) ;
if ( started )
start_remove_from = existing_changelogs . upper_bound ( incomplete_log_index ) ;
2021-02-20 11:04:38 +00:00
/// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them.
2021-03-04 12:01:56 +00:00
for ( auto itr = start_remove_from ; itr ! = existing_changelogs . end ( ) ; )
2021-02-17 09:00:12 +00:00
{
2021-02-20 11:28:39 +00:00
LOG_WARNING ( log , " Removing changelog {}, because it's goes after broken changelog entry " , itr - > second . path ) ;
2021-02-17 09:00:12 +00:00
std : : filesystem : : remove ( itr - > second . path ) ;
itr = existing_changelogs . erase ( itr ) ;
}
2021-02-20 11:04:38 +00:00
/// Continue to write into existing log
if ( ! existing_changelogs . empty ( ) )
{
auto description = existing_changelogs . rbegin ( ) - > second ;
2021-08-25 18:11:52 +00:00
if ( description . expectedEntriesCountInLog ( ) ! = rotate_interval )
LOG_TRACE ( log , " Looks like rotate_logs_interval was changed, current {}, expected entries in last log {} " , rotate_interval , description . expectedEntriesCountInLog ( ) ) ;
2021-02-20 11:04:38 +00:00
LOG_TRACE ( log , " Continue to write into {} " , description . path ) ;
current_writer = std : : make_unique < ChangelogWriter > ( description . path , WriteMode : : Append , description . from_log_index ) ;
current_writer - > setEntriesWritten ( result . entries_read ) ;
/// Truncate all broken entries from log
if ( result . error )
{
LOG_WARNING ( log , " Read finished with error, truncating all broken log entries " ) ;
current_writer - > truncateToLength ( result . last_position ) ;
}
}
2021-02-15 15:36:25 +00:00
}
2021-02-20 11:04:38 +00:00
/// Start new log if we don't initialize writer from previous log
if ( ! current_writer )
2021-02-17 08:00:17 +00:00
rotate ( start_index + total_read ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-08 14:17:57 +00:00
void Changelog : : rotate ( uint64_t new_start_log_index )
2021-02-15 15:36:25 +00:00
{
2021-04-16 13:50:09 +00:00
/// Flush previous log
flush ( ) ;
2021-02-17 09:00:12 +00:00
ChangelogFileDescription new_description ;
new_description . prefix = DEFAULT_PREFIX ;
2021-02-20 11:04:38 +00:00
new_description . from_log_index = new_start_log_index ;
new_description . to_log_index = new_start_log_index + rotate_interval - 1 ;
2021-02-15 15:36:25 +00:00
2021-02-17 09:00:12 +00:00
new_description . path = formatChangelogPath ( changelogs_dir , new_description ) ;
2021-02-20 11:04:38 +00:00
LOG_TRACE ( log , " Starting new changelog {} " , new_description . path ) ;
existing_changelogs [ new_start_log_index ] = new_description ;
current_writer = std : : make_unique < ChangelogWriter > ( new_description . path , WriteMode : : Rewrite , new_start_log_index ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-08 14:17:57 +00:00
ChangelogRecord Changelog : : buildRecord ( uint64_t index , const LogEntryPtr & log_entry )
2021-02-15 15:36:25 +00:00
{
2021-02-20 15:36:56 +00:00
ChangelogRecord record ;
2021-08-25 18:11:52 +00:00
record . header . version = ChangelogVersion : : V1 ;
2021-02-20 15:36:56 +00:00
record . header . index = index ;
record . header . term = log_entry - > get_term ( ) ;
record . header . value_type = log_entry - > get_val_type ( ) ;
2021-02-15 15:36:25 +00:00
auto buffer = log_entry - > get_buf_ptr ( ) ;
if ( buffer )
2021-02-20 15:36:56 +00:00
record . header . blob_size = buffer - > size ( ) ;
2021-02-15 15:36:25 +00:00
else
2021-02-20 15:36:56 +00:00
record . header . blob_size = 0 ;
record . blob = buffer ;
2021-02-15 15:36:25 +00:00
2021-02-20 15:36:56 +00:00
return record ;
2021-02-15 15:36:25 +00:00
}
2021-04-16 13:50:09 +00:00
void Changelog : : appendEntry ( uint64_t index , const LogEntryPtr & log_entry )
2021-02-15 15:36:25 +00:00
{
if ( ! current_writer )
2021-02-15 17:59:40 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Changelog must be initialized before appending records " ) ;
2021-02-15 15:36:25 +00:00
2021-02-16 17:28:54 +00:00
if ( logs . empty ( ) )
start_index = index ;
2021-08-25 18:11:52 +00:00
const auto & current_changelog_description = existing_changelogs [ current_writer - > getStartIndex ( ) ] ;
const bool log_is_complete = current_writer - > getEntriesWritten ( ) = = current_changelog_description . expectedEntriesCountInLog ( ) ;
if ( log_is_complete )
2021-02-15 15:36:25 +00:00
rotate ( index ) ;
2021-08-25 18:11:52 +00:00
const auto offset = current_writer - > appendRecord ( buildRecord ( index , log_entry ) ) ;
2021-02-15 15:36:25 +00:00
if ( ! index_to_start_pos . try_emplace ( index , offset ) . second )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Record with index {} already exists " , index ) ;
2021-02-16 17:28:54 +00:00
2021-02-15 17:59:40 +00:00
logs [ index ] = makeClone ( log_entry ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-16 13:50:09 +00:00
void Changelog : : writeAt ( uint64_t index , const LogEntryPtr & log_entry )
2021-02-15 15:36:25 +00:00
{
2021-02-15 17:59:40 +00:00
if ( index_to_start_pos . count ( index ) = = 0 )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot write at index {} because changelog doesn't contain it " , index ) ;
2021-08-25 18:11:52 +00:00
/// This write_at require to overwrite everything in this file and also in previous file(s)
const bool go_to_previous_file = index < current_writer - > getStartIndex ( ) ;
2021-02-20 11:04:38 +00:00
if ( go_to_previous_file )
2021-02-16 17:28:54 +00:00
{
auto index_changelog = existing_changelogs . lower_bound ( index ) ;
2021-08-25 18:11:52 +00:00
2021-02-17 09:00:12 +00:00
ChangelogFileDescription description ;
2021-08-25 18:11:52 +00:00
if ( index_changelog - > first = = index ) /// exactly this file starts from index
2021-02-17 09:00:12 +00:00
description = index_changelog - > second ;
2021-02-16 17:28:54 +00:00
else
2021-02-17 09:00:12 +00:00
description = std : : prev ( index_changelog ) - > second ;
2021-02-16 17:28:54 +00:00
2021-08-25 18:11:52 +00:00
/// Initialize writer from this log file
2021-02-17 09:00:12 +00:00
current_writer = std : : make_unique < ChangelogWriter > ( description . path , WriteMode : : Append , index_changelog - > first ) ;
2021-02-20 11:04:38 +00:00
current_writer - > setEntriesWritten ( description . to_log_index - description . from_log_index + 1 ) ;
2021-02-16 17:28:54 +00:00
}
2021-08-25 18:11:52 +00:00
/// Truncate current file
2021-02-15 17:59:40 +00:00
current_writer - > truncateToLength ( index_to_start_pos [ index ] ) ;
2021-02-16 17:28:54 +00:00
2021-02-20 11:04:38 +00:00
if ( go_to_previous_file )
2021-02-15 15:36:25 +00:00
{
2021-08-25 18:11:52 +00:00
/// Remove all subsequent files if overwritten something in previous one
2021-02-16 17:28:54 +00:00
auto to_remove_itr = existing_changelogs . upper_bound ( index ) ;
for ( auto itr = to_remove_itr ; itr ! = existing_changelogs . end ( ) ; )
2021-02-15 15:36:25 +00:00
{
2021-02-17 09:00:12 +00:00
std : : filesystem : : remove ( itr - > second . path ) ;
2021-02-16 17:28:54 +00:00
itr = existing_changelogs . erase ( itr ) ;
2021-02-15 15:36:25 +00:00
}
}
2021-08-25 18:11:52 +00:00
auto entries_written = current_writer - > getEntriesWritten ( ) ;
2021-02-20 11:04:38 +00:00
/// Remove redundant logs from memory
2021-08-25 18:11:52 +00:00
/// Everything >= index must be removed
2021-04-08 14:17:57 +00:00
for ( uint64_t i = index ; ; + + i )
2021-02-16 17:28:54 +00:00
{
2021-02-20 11:04:38 +00:00
auto log_itr = logs . find ( i ) ;
if ( log_itr = = logs . end ( ) )
break ;
logs . erase ( log_itr ) ;
index_to_start_pos . erase ( i ) ;
2021-02-16 17:28:54 +00:00
entries_written - - ;
}
current_writer - > setEntriesWritten ( entries_written ) ;
2021-02-15 17:59:40 +00:00
2021-08-25 18:11:52 +00:00
/// Now we can actually override entry at index
2021-04-16 13:50:09 +00:00
appendEntry ( index , log_entry ) ;
2021-02-15 15:36:25 +00:00
}
2021-04-08 14:17:57 +00:00
void Changelog : : compact ( uint64_t up_to_log_index )
2021-02-15 15:36:25 +00:00
{
for ( auto itr = existing_changelogs . begin ( ) ; itr ! = existing_changelogs . end ( ) ; )
{
2021-02-20 11:04:38 +00:00
/// Remove all completely outdated changelog files
if ( itr - > second . to_log_index < = up_to_log_index )
2021-02-15 15:36:25 +00:00
{
2021-02-20 11:04:38 +00:00
LOG_INFO ( log , " Removing changelog {} because of compaction " , itr - > second . path ) ;
std : : erase_if ( index_to_start_pos , [ right_index = itr - > second . to_log_index ] ( const auto & item ) { return item . first < = right_index ; } ) ;
2021-02-17 09:00:12 +00:00
std : : filesystem : : remove ( itr - > second . path ) ;
2021-02-16 17:28:54 +00:00
itr = existing_changelogs . erase ( itr ) ;
2021-02-15 15:36:25 +00:00
}
2021-02-20 11:04:38 +00:00
else /// Files are ordered, so all subsequent should exist
2021-02-16 17:28:54 +00:00
break ;
2021-02-15 15:36:25 +00:00
}
2021-02-20 11:04:38 +00:00
start_index = up_to_log_index + 1 ;
std : : erase_if ( logs , [ up_to_log_index ] ( const auto & item ) { return item . first < = up_to_log_index ; } ) ;
2021-02-15 15:36:25 +00:00
}
2021-02-15 17:59:40 +00:00
LogEntryPtr Changelog : : getLastEntry ( ) const
{
2021-04-08 14:17:57 +00:00
static LogEntryPtr fake_entry = nuraft : : cs_new < nuraft : : log_entry > ( 0 , nuraft : : buffer : : alloc ( sizeof ( uint64_t ) ) ) ;
2021-02-15 17:59:40 +00:00
2021-08-25 18:11:52 +00:00
const uint64_t next_index = getNextEntryIndex ( ) - 1 ;
2021-02-20 11:04:38 +00:00
auto entry = logs . find ( next_index ) ;
2021-02-15 17:59:40 +00:00
if ( entry = = logs . end ( ) )
return fake_entry ;
2021-02-21 17:42:19 +00:00
return entry - > second ;
2021-02-15 17:59:40 +00:00
}
2021-04-08 14:17:57 +00:00
LogEntriesPtr Changelog : : getLogEntriesBetween ( uint64_t start , uint64_t end )
2021-02-15 17:59:40 +00:00
{
LogEntriesPtr ret = nuraft : : cs_new < std : : vector < nuraft : : ptr < nuraft : : log_entry > > > ( ) ;
ret - > resize ( end - start ) ;
2021-04-08 14:17:57 +00:00
uint64_t result_pos = 0 ;
for ( uint64_t i = start ; i < end ; + + i )
2021-02-15 17:59:40 +00:00
{
( * ret ) [ result_pos ] = entryAt ( i ) ;
result_pos + + ;
}
return ret ;
}
2021-04-08 14:17:57 +00:00
LogEntryPtr Changelog : : entryAt ( uint64_t index )
2021-02-15 17:59:40 +00:00
{
nuraft : : ptr < nuraft : : log_entry > src = nullptr ;
2021-02-20 11:04:38 +00:00
auto entry = logs . find ( index ) ;
2021-02-15 17:59:40 +00:00
if ( entry = = logs . end ( ) )
return nullptr ;
src = entry - > second ;
2021-02-21 17:42:19 +00:00
return src ;
2021-02-15 17:59:40 +00:00
}
2021-04-08 14:17:57 +00:00
nuraft : : ptr < nuraft : : buffer > Changelog : : serializeEntriesToBuffer ( uint64_t index , int32_t count )
2021-02-15 17:59:40 +00:00
{
std : : vector < nuraft : : ptr < nuraft : : buffer > > returned_logs ;
2021-04-08 14:24:05 +00:00
uint64_t size_total = 0 ;
2021-04-08 14:17:57 +00:00
for ( uint64_t i = index ; i < index + count ; + + i )
2021-02-15 17:59:40 +00:00
{
auto entry = logs . find ( i ) ;
if ( entry = = logs . end ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Don't have log entry {} " , i ) ;
nuraft : : ptr < nuraft : : buffer > buf = entry - > second - > serialize ( ) ;
2021-04-08 14:24:05 +00:00
size_total + = buf - > size ( ) ;
2021-02-15 17:59:40 +00:00
returned_logs . push_back ( buf ) ;
}
2021-04-08 14:24:05 +00:00
nuraft : : ptr < nuraft : : buffer > buf_out = nuraft : : buffer : : alloc ( sizeof ( int32_t ) + count * sizeof ( int32_t ) + size_total ) ;
2021-02-15 17:59:40 +00:00
buf_out - > pos ( 0 ) ;
2021-02-20 11:04:38 +00:00
buf_out - > put ( static_cast < int32_t > ( count ) ) ;
2021-02-15 17:59:40 +00:00
for ( auto & entry : returned_logs )
{
nuraft : : ptr < nuraft : : buffer > & bb = entry ;
buf_out - > put ( static_cast < int32_t > ( bb - > size ( ) ) ) ;
buf_out - > put ( * bb ) ;
}
return buf_out ;
}
2021-04-16 13:50:09 +00:00
void Changelog : : applyEntriesFromBuffer ( uint64_t index , nuraft : : buffer & buffer )
2021-02-15 17:59:40 +00:00
{
buffer . pos ( 0 ) ;
int num_logs = buffer . get_int ( ) ;
for ( int i = 0 ; i < num_logs ; + + i )
{
2021-04-08 14:17:57 +00:00
uint64_t cur_index = index + i ;
2021-02-15 17:59:40 +00:00
int buf_size = buffer . get_int ( ) ;
nuraft : : ptr < nuraft : : buffer > buf_local = nuraft : : buffer : : alloc ( buf_size ) ;
buffer . get ( buf_local ) ;
LogEntryPtr log_entry = nuraft : : log_entry : : deserialize ( * buf_local ) ;
2021-02-20 11:04:38 +00:00
if ( i = = 0 & & logs . count ( cur_index ) )
2021-04-16 13:50:09 +00:00
writeAt ( cur_index , log_entry ) ;
2021-02-15 17:59:40 +00:00
else
2021-04-16 13:50:09 +00:00
appendEntry ( cur_index , log_entry ) ;
2021-02-15 17:59:40 +00:00
}
}
void Changelog : : flush ( )
{
2021-04-16 13:50:09 +00:00
if ( current_writer )
current_writer - > flush ( force_sync ) ;
2021-02-15 17:59:40 +00:00
}
2021-02-17 20:36:25 +00:00
Changelog : : ~ Changelog ( )
{
try
{
2021-04-16 13:50:09 +00:00
flush ( ) ;
2021-02-17 20:36:25 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2021-02-15 17:59:40 +00:00
2021-02-15 15:36:25 +00:00
}