2017-11-23 13:12:22 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeQueue.h>
2018-05-20 19:56:03 +00:00
# include <Storages/StorageReplicatedMergeTree.h>
2017-04-01 09:19:00 +00:00
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
2019-10-10 16:30:30 +00:00
# include <Storages/MergeTree/IMergeTreeDataPart.h>
2018-04-20 16:18:16 +00:00
# include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
2018-05-10 15:01:10 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
2018-01-15 19:07:47 +00:00
# include <Common/StringUtils/StringUtils.h>
2016-01-10 04:44:12 +00:00
namespace DB
{
2016-01-12 19:45:51 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR ;
2017-04-01 07:20:54 +00:00
extern const int UNEXPECTED_NODE_IN_ZOOKEEPER ;
2017-06-16 16:47:09 +00:00
extern const int UNFINISHED ;
2016-01-12 19:45:51 +00:00
}
2016-01-10 04:44:12 +00:00
2018-05-20 19:56:03 +00:00
ReplicatedMergeTreeQueue : : ReplicatedMergeTreeQueue ( StorageReplicatedMergeTree & storage_ )
: storage ( storage_ )
2019-05-03 02:00:57 +00:00
, format_version ( storage . format_version )
2018-06-18 12:17:46 +00:00
, current_parts ( format_version )
2018-05-20 19:56:03 +00:00
, virtual_parts ( format_version )
{ }
2019-08-20 08:38:02 +00:00
void ReplicatedMergeTreeQueue : : addVirtualParts ( const MergeTreeData : : DataParts & parts )
2016-01-10 04:44:12 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2016-01-10 04:44:12 +00:00
2019-08-20 08:38:02 +00:00
for ( auto part : parts )
2018-06-18 12:17:46 +00:00
{
current_parts . add ( part - > name ) ;
2017-04-01 07:20:54 +00:00
virtual_parts . add ( part - > name ) ;
2018-06-18 12:17:46 +00:00
}
2016-01-10 04:44:12 +00:00
}
2019-09-10 11:21:59 +00:00
bool ReplicatedMergeTreeQueue : : isVirtualPart ( const MergeTreeData : : DataPartPtr & data_part ) const
2019-08-20 08:38:02 +00:00
{
std : : lock_guard lock ( state_mutex ) ;
2019-08-20 17:16:32 +00:00
return virtual_parts . getContainingPart ( data_part - > info ) ! = data_part - > name ;
2019-08-20 08:38:02 +00:00
}
2017-11-28 14:07:17 +00:00
bool ReplicatedMergeTreeQueue : : load ( zkutil : : ZooKeeperPtr zookeeper )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
auto queue_path = replica_path + " /queue " ;
LOG_DEBUG ( log , " Loading queue from " < < queue_path ) ;
2016-01-12 19:45:51 +00:00
2017-11-28 14:07:17 +00:00
bool updated = false ;
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
2018-05-10 15:01:10 +00:00
2017-11-28 14:07:17 +00:00
{
2018-08-09 15:06:39 +00:00
std : : lock_guard pull_logs_lock ( pull_logs_to_queue_mutex ) ;
2016-01-10 04:44:12 +00:00
2018-05-10 15:01:10 +00:00
String log_pointer_str = zookeeper - > get ( replica_path + " /log_pointer " ) ;
log_pointer = log_pointer_str . empty ( ) ? 0 : parse < UInt64 > ( log_pointer_str ) ;
2018-08-27 19:06:32 +00:00
2018-08-20 13:31:24 +00:00
std : : unordered_set < String > already_loaded_paths ;
2018-08-22 14:01:54 +00:00
{
std : : lock_guard lock ( state_mutex ) ;
for ( const LogEntryPtr & log_entry : queue )
already_loaded_paths . insert ( log_entry - > znode_name ) ;
}
2016-01-12 19:45:51 +00:00
2017-11-28 14:07:17 +00:00
Strings children = zookeeper - > getChildren ( queue_path ) ;
2018-08-09 15:06:39 +00:00
auto to_remove_it = std : : remove_if (
2019-05-03 02:00:57 +00:00
children . begin ( ) , children . end ( ) , [ & ] ( const String & path )
{
return already_loaded_paths . count ( path ) ;
} ) ;
2018-08-09 15:06:39 +00:00
LOG_DEBUG ( log ,
2019-05-03 02:00:57 +00:00
" Having " < < ( to_remove_it - children . begin ( ) ) < < " queue entries to load, "
< < ( children . end ( ) - to_remove_it ) < < " entries already loaded. " ) ;
2018-08-09 15:06:39 +00:00
children . erase ( to_remove_it , children . end ( ) ) ;
2017-11-28 14:07:17 +00:00
std : : sort ( children . begin ( ) , children . end ( ) ) ;
2016-01-10 04:44:12 +00:00
2018-08-25 01:58:14 +00:00
zkutil : : AsyncResponses < Coordination : : GetResponse > futures ;
2017-11-28 14:07:17 +00:00
futures . reserve ( children . size ( ) ) ;
2016-01-10 04:44:12 +00:00
2017-11-28 14:07:17 +00:00
for ( const String & child : children )
futures . emplace_back ( child , zookeeper - > asyncGet ( queue_path + " / " + child ) ) ;
for ( auto & future : futures )
{
2018-08-25 01:58:14 +00:00
Coordination : : GetResponse res = future . second . get ( ) ;
2018-03-24 20:11:46 +00:00
LogEntryPtr entry = LogEntry : : parse ( res . data , res . stat ) ;
2017-11-28 14:07:17 +00:00
entry - > znode_name = future . first ;
2018-08-09 15:06:39 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-06-20 11:12:16 +00:00
insertUnlocked ( entry , min_unprocessed_insert_time_changed , lock ) ;
2018-05-10 15:01:10 +00:00
2017-11-28 14:07:17 +00:00
updated = true ;
}
2018-06-21 13:27:36 +00:00
zookeeper - > tryGet ( replica_path + " /mutation_pointer " , mutation_pointer ) ;
2017-04-01 07:20:54 +00:00
}
2016-01-12 19:45:51 +00:00
2018-03-03 16:46:32 +00:00
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , { } ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
LOG_TRACE ( log , " Loaded queue " ) ;
2017-11-28 14:07:17 +00:00
return updated ;
2016-01-10 04:44:12 +00:00
}
void ReplicatedMergeTreeQueue : : initialize (
2017-04-01 07:20:54 +00:00
const String & zookeeper_path_ , const String & replica_path_ , const String & logger_name_ ,
2018-08-09 16:24:03 +00:00
const MergeTreeData : : DataParts & parts )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
zookeeper_path = zookeeper_path_ ;
replica_path = replica_path_ ;
logger_name = logger_name_ ;
log = & Logger : : get ( logger_name ) ;
2016-01-10 04:44:12 +00:00
2018-05-21 13:49:54 +00:00
addVirtualParts ( parts ) ;
2016-01-10 04:44:12 +00:00
}
2018-05-10 15:01:10 +00:00
void ReplicatedMergeTreeQueue : : insertUnlocked (
const LogEntryPtr & entry , std : : optional < time_t > & min_unprocessed_insert_time_changed ,
2020-02-01 20:33:36 +00:00
std : : lock_guard < std : : mutex > & state_lock )
2016-01-10 04:44:12 +00:00
{
2018-06-18 12:17:46 +00:00
for ( const String & virtual_part_name : entry - > getVirtualPartNames ( ) )
{
2018-05-21 13:49:54 +00:00
virtual_parts . add ( virtual_part_name ) ;
2020-02-05 16:30:02 +00:00
addPartToMutations ( virtual_part_name ) ;
2018-06-18 12:17:46 +00:00
}
2018-03-28 12:48:40 +00:00
/// Put 'DROP PARTITION' entries at the beginning of the queue not to make superfluous fetches of parts that will be eventually deleted
if ( entry - > type ! = LogEntry : : DROP_RANGE )
queue . push_back ( entry ) ;
else
queue . push_front ( entry ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
if ( entry - > type = = LogEntry : : GET_PART )
{
inserts_by_time . insert ( entry ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
if ( entry - > create_time & & ( ! min_unprocessed_insert_time | | entry - > create_time < min_unprocessed_insert_time ) )
2018-03-03 16:46:32 +00:00
{
2017-04-01 07:20:54 +00:00
min_unprocessed_insert_time = entry - > create_time ;
2018-03-03 16:46:32 +00:00
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
}
2017-04-01 07:20:54 +00:00
}
2020-01-28 17:15:22 +00:00
if ( entry - > type = = LogEntry : : ALTER_METADATA )
2020-02-14 20:13:57 +00:00
{
LOG_TRACE ( log , " Adding alter metadata version " < < entry - > alter_version < < " to the queue " ) ;
2020-02-17 18:07:22 +00:00
alter_sequence . addMetadataAlter ( entry - > alter_version , entry - > have_mutation , state_lock ) ;
2020-02-14 20:13:57 +00:00
}
2016-01-10 04:44:12 +00:00
}
2016-01-17 13:00:42 +00:00
void ReplicatedMergeTreeQueue : : insert ( zkutil : : ZooKeeperPtr zookeeper , LogEntryPtr & entry )
2016-01-10 04:44:12 +00:00
{
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
insertUnlocked ( entry , min_unprocessed_insert_time_changed , lock ) ;
2017-04-01 07:20:54 +00:00
}
2016-01-17 13:00:42 +00:00
2018-03-03 16:48:24 +00:00
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , { } ) ;
2016-01-17 13:00:42 +00:00
}
2018-06-18 12:17:46 +00:00
void ReplicatedMergeTreeQueue : : updateStateOnQueueEntryRemoval (
2017-04-01 07:20:54 +00:00
const LogEntryPtr & entry ,
2018-06-18 12:17:46 +00:00
bool is_successful ,
2018-03-03 16:46:32 +00:00
std : : optional < time_t > & min_unprocessed_insert_time_changed ,
std : : optional < time_t > & max_processed_insert_time_changed ,
2020-02-01 20:33:36 +00:00
std : : unique_lock < std : : mutex > & state_lock )
2016-01-17 13:00:42 +00:00
{
2018-06-18 12:17:46 +00:00
/// Update insert times.
if ( entry - > type = = LogEntry : : GET_PART )
{
inserts_by_time . erase ( entry ) ;
2017-04-01 07:20:54 +00:00
2018-06-18 12:17:46 +00:00
if ( inserts_by_time . empty ( ) )
{
min_unprocessed_insert_time = 0 ;
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
}
else if ( ( * inserts_by_time . begin ( ) ) - > create_time > min_unprocessed_insert_time )
{
min_unprocessed_insert_time = ( * inserts_by_time . begin ( ) ) - > create_time ;
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
}
2017-04-01 07:20:54 +00:00
2018-06-18 12:17:46 +00:00
if ( entry - > create_time > max_processed_insert_time )
{
max_processed_insert_time = entry - > create_time ;
max_processed_insert_time_changed = max_processed_insert_time ;
}
2017-04-01 07:20:54 +00:00
}
2018-06-18 12:17:46 +00:00
if ( is_successful )
2017-04-01 07:20:54 +00:00
{
2020-02-19 12:50:11 +00:00
if ( ! entry - > actual_new_part_name . empty ( ) )
{
/// We don't add bigger fetched part to current_parts because we
/// have an invariant `virtual_parts` = `current_parts` + `queue`.
/// But we can remove it from mutations, because we actually have it.
removePartFromMutations ( entry - > actual_new_part_name ) ;
}
2018-06-18 12:17:46 +00:00
for ( const String & virtual_part_name : entry - > getVirtualPartNames ( ) )
{
2020-02-19 12:50:11 +00:00
current_parts . add ( virtual_part_name ) ;
/// Each processed part may be already mutated, so we try to remove
/// all current parts from mutations.
removePartFromMutations ( virtual_part_name ) ;
2018-06-18 12:17:46 +00:00
}
2017-04-01 07:20:54 +00:00
2018-06-18 12:17:46 +00:00
String drop_range_part_name ;
if ( entry - > type = = LogEntry : : DROP_RANGE )
drop_range_part_name = entry - > new_part_name ;
else if ( entry - > type = = LogEntry : : REPLACE_RANGE )
drop_range_part_name = entry - > replace_range_entry - > drop_range_part_name ;
if ( ! drop_range_part_name . empty ( ) )
{
current_parts . remove ( drop_range_part_name ) ;
virtual_parts . remove ( drop_range_part_name ) ;
}
2020-01-28 17:15:22 +00:00
if ( entry - > type = = LogEntry : : ALTER_METADATA )
2020-02-14 20:13:57 +00:00
{
LOG_TRACE ( log , " Finishing metadata alter with version " < < entry - > alter_version ) ;
2020-02-17 18:07:22 +00:00
alter_sequence . finishMetadataAlter ( entry - > alter_version , state_lock ) ;
2020-02-14 20:13:57 +00:00
}
2018-06-18 12:17:46 +00:00
}
else
2017-04-01 07:20:54 +00:00
{
2018-06-18 12:17:46 +00:00
for ( const String & virtual_part_name : entry - > getVirtualPartNames ( ) )
{
/// Because execution of the entry is unsuccessful, `virtual_part_name` will never appear
/// so we won't need to mutate it.
2020-02-05 16:30:02 +00:00
removePartFromMutations ( virtual_part_name ) ;
2018-06-21 15:54:01 +00:00
}
2020-02-10 13:32:59 +00:00
2018-06-21 15:54:01 +00:00
}
}
2018-06-18 12:17:46 +00:00
2020-02-05 16:30:02 +00:00
void ReplicatedMergeTreeQueue : : removePartFromMutations ( const String & part_name )
2018-06-21 15:54:01 +00:00
{
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
auto in_partition = mutations_by_partition . find ( part_info . partition_id ) ;
if ( in_partition = = mutations_by_partition . end ( ) )
return ;
bool some_mutations_are_probably_done = false ;
2020-02-05 16:30:02 +00:00
auto from_it = in_partition - > second . lower_bound ( part_info . getDataVersion ( ) ) ;
2018-06-21 15:54:01 +00:00
for ( auto it = from_it ; it ! = in_partition - > second . end ( ) ; + + it )
{
2019-02-05 13:03:52 +00:00
MutationStatus & status = * it - > second ;
2020-02-11 14:54:46 +00:00
2020-02-05 16:30:02 +00:00
status . parts_to_do . removePartAndCoveredParts ( part_name ) ;
2020-03-08 21:40:00 +00:00
if ( status . parts_to_do . empty ( ) )
2018-06-21 15:54:01 +00:00
some_mutations_are_probably_done = true ;
2019-02-05 13:03:52 +00:00
2020-02-05 16:30:02 +00:00
if ( ! status . latest_failed_part . empty ( ) & & part_info . contains ( status . latest_failed_part_info ) )
2019-02-05 13:03:52 +00:00
{
status . latest_failed_part . clear ( ) ;
status . latest_failed_part_info = MergeTreePartInfo ( ) ;
status . latest_fail_time = 0 ;
status . latest_fail_reason . clear ( ) ;
}
2017-04-01 07:20:54 +00:00
}
2018-06-18 12:17:46 +00:00
2018-06-21 13:27:36 +00:00
if ( some_mutations_are_probably_done )
storage . mutations_finalizing_task - > schedule ( ) ;
2016-01-17 13:00:42 +00:00
}
2020-02-05 16:30:02 +00:00
void ReplicatedMergeTreeQueue : : addPartToMutations ( const String & part_name )
{
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
auto in_partition = mutations_by_partition . find ( part_info . partition_id ) ;
if ( in_partition = = mutations_by_partition . end ( ) )
return ;
auto from_it = in_partition - > second . upper_bound ( part_info . getDataVersion ( ) ) ;
for ( auto it = from_it ; it ! = in_partition - > second . end ( ) ; + + it )
{
MutationStatus & status = * it - > second ;
status . parts_to_do . add ( part_name ) ;
}
}
2016-01-17 13:00:42 +00:00
void ReplicatedMergeTreeQueue : : updateTimesInZooKeeper (
2017-04-01 07:20:54 +00:00
zkutil : : ZooKeeperPtr zookeeper ,
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ,
std : : optional < time_t > max_processed_insert_time_changed ) const
2016-01-17 13:00:42 +00:00
{
2018-03-03 16:46:32 +00:00
/// Here there can be a race condition (with different remove at the same time)
/// because we update times in ZooKeeper with unlocked mutex, while these times may change.
2017-04-01 07:20:54 +00:00
/// Consider it unimportant (for a short time, ZK will have a slightly different time value).
2018-03-03 16:46:32 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2017-04-01 07:20:54 +00:00
if ( min_unprocessed_insert_time_changed )
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2018-03-03 16:46:32 +00:00
replica_path + " /min_unprocessed_insert_time " , toString ( * min_unprocessed_insert_time_changed ) , - 1 ) ) ;
2017-04-01 07:20:54 +00:00
if ( max_processed_insert_time_changed )
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2018-03-03 16:46:32 +00:00
replica_path + " /max_processed_insert_time " , toString ( * max_processed_insert_time_changed ) , - 1 ) ) ;
2017-04-01 07:20:54 +00:00
if ( ! ops . empty ( ) )
{
2018-08-25 01:58:14 +00:00
Coordination : : Responses responses ;
2018-03-25 00:15:52 +00:00
auto code = zookeeper - > tryMulti ( ops , responses ) ;
2017-04-01 07:20:54 +00:00
2018-03-24 00:45:04 +00:00
if ( code )
2017-04-01 07:20:54 +00:00
LOG_ERROR ( log , " Couldn't set value of nodes for insert times ( "
< < replica_path < < " /min_unprocessed_insert_time, max_processed_insert_time) " < < " : "
< < zkutil : : ZooKeeper : : error2string ( code ) + " . This shouldn't happen often. " ) ;
}
2016-01-10 04:44:12 +00:00
}
2018-06-06 19:15:10 +00:00
void ReplicatedMergeTreeQueue : : removeProcessedEntry ( zkutil : : ZooKeeperPtr zookeeper , LogEntryPtr & entry )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
auto code = zookeeper - > tryRemove ( replica_path + " /queue/ " + entry - > znode_name ) ;
2018-03-24 00:45:04 +00:00
if ( code )
2017-04-01 07:20:54 +00:00
LOG_ERROR ( log , " Couldn't remove " < < replica_path < < " /queue/ " < < entry - > znode_name < < " : "
< < zkutil : : ZooKeeper : : error2string ( code ) < < " . This shouldn't happen often. " ) ;
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
std : : optional < time_t > max_processed_insert_time_changed ;
2017-04-01 07:20:54 +00:00
2018-05-21 13:49:54 +00:00
bool found = false ;
size_t queue_size = 0 ;
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
/// Remove the job from the queue in the RAM.
/// You can not just refer to a pre-saved iterator, because someone else might be able to delete the task.
/// Why do we view the queue from the end?
/// - because the task for execution first is moved to the end of the queue, so that in case of failure it remains at the end.
for ( Queue : : iterator it = queue . end ( ) ; it ! = queue . begin ( ) ; )
{
- - it ;
if ( * it = = entry )
{
2018-05-21 13:49:54 +00:00
found = true ;
2018-06-18 12:17:46 +00:00
updateStateOnQueueEntryRemoval (
entry , /* is_successful = */ true ,
min_unprocessed_insert_time_changed , max_processed_insert_time_changed , lock ) ;
queue . erase ( it ) ;
2018-05-21 13:49:54 +00:00
queue_size = queue . size ( ) ;
2017-04-01 07:20:54 +00:00
break ;
}
}
}
2018-05-21 15:10:50 +00:00
if ( ! found )
throw Exception ( " Can't find " + entry - > znode_name + " in the memory queue. It is a bug " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-05-21 13:49:54 +00:00
notifySubscribers ( queue_size ) ;
2017-04-01 07:20:54 +00:00
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , max_processed_insert_time_changed ) ;
2016-01-10 04:44:12 +00:00
}
bool ReplicatedMergeTreeQueue : : remove ( zkutil : : ZooKeeperPtr zookeeper , const String & part_name )
{
2017-04-01 07:20:54 +00:00
LogEntryPtr found ;
2018-05-21 13:49:54 +00:00
size_t queue_size = 0 ;
2017-04-01 07:20:54 +00:00
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
std : : optional < time_t > max_processed_insert_time_changed ;
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( state_mutex ) ;
2018-05-10 15:01:10 +00:00
virtual_parts . remove ( part_name ) ;
2017-04-01 07:20:54 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; )
{
if ( ( * it ) - > new_part_name = = part_name )
{
found = * it ;
2018-06-18 12:17:46 +00:00
updateStateOnQueueEntryRemoval (
found , /* is_successful = */ false ,
2018-06-20 11:12:16 +00:00
min_unprocessed_insert_time_changed , max_processed_insert_time_changed , lock ) ;
2017-04-01 07:20:54 +00:00
queue . erase ( it + + ) ;
2018-05-21 13:49:54 +00:00
queue_size = queue . size ( ) ;
2017-04-01 07:20:54 +00:00
break ;
}
else
+ + it ;
}
}
if ( ! found )
return false ;
2018-05-21 13:49:54 +00:00
notifySubscribers ( queue_size ) ;
2017-04-01 07:20:54 +00:00
zookeeper - > tryRemove ( replica_path + " /queue/ " + found - > znode_name ) ;
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , max_processed_insert_time_changed ) ;
return true ;
2016-01-10 04:44:12 +00:00
}
2018-05-10 15:01:10 +00:00
bool ReplicatedMergeTreeQueue : : removeFromVirtualParts ( const MergeTreePartInfo & part_info )
{
2019-08-20 18:00:48 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-05-10 15:01:10 +00:00
return virtual_parts . remove ( part_info ) ;
}
2018-08-25 01:58:14 +00:00
void ReplicatedMergeTreeQueue : : pullLogsToQueue ( zkutil : : ZooKeeperPtr zookeeper , Coordination : : WatchCallback watch_callback )
2016-01-10 04:44:12 +00:00
{
2018-05-10 15:01:10 +00:00
std : : lock_guard lock ( pull_logs_to_queue_mutex ) ;
2016-01-13 02:36:11 +00:00
2017-11-28 14:07:17 +00:00
String index_str = zookeeper - > get ( replica_path + " /log_pointer " ) ;
2017-04-01 07:20:54 +00:00
UInt64 index ;
2016-01-10 04:44:12 +00:00
2018-05-28 15:37:30 +00:00
Strings log_entries = zookeeper - > getChildrenWatch ( zookeeper_path + " /log " , nullptr , watch_callback ) ;
2016-01-12 19:45:51 +00:00
2018-04-19 14:20:18 +00:00
/// We update mutations after we have loaded the list of log entries, but before we insert them
/// in the queue.
2018-05-20 19:56:03 +00:00
/// With this we ensure that if you read the log state L1 and then the state of mutations M1,
/// then L1 "happened-before" M1.
2018-05-31 13:05:05 +00:00
updateMutations ( zookeeper ) ;
2018-04-19 14:20:18 +00:00
2017-04-01 07:20:54 +00:00
if ( index_str . empty ( ) )
{
/// If we do not already have a pointer to the log, put a pointer to the first entry in it.
index = log_entries . empty ( ) ? 0 : parse < UInt64 > ( std : : min_element ( log_entries . begin ( ) , log_entries . end ( ) ) - > substr ( strlen ( " log- " ) ) ) ;
2016-01-10 04:44:12 +00:00
2017-11-28 14:07:17 +00:00
zookeeper - > set ( replica_path + " /log_pointer " , toString ( index ) ) ;
2017-04-01 07:20:54 +00:00
}
else
{
index = parse < UInt64 > ( index_str ) ;
}
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
String min_log_entry = " log- " + padIndex ( index ) ;
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
/// Multiple log entries that must be copied to the queue.
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
log_entries . erase (
2018-08-22 14:10:00 +00:00
std : : remove_if ( log_entries . begin ( ) , log_entries . end ( ) , [ & min_log_entry ] ( const String & entry ) { return entry < min_log_entry ; } ) ,
log_entries . end ( ) ) ;
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
if ( ! log_entries . empty ( ) )
{
std : : sort ( log_entries . begin ( ) , log_entries . end ( ) ) ;
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
/// ZK contains a limit on the number or total size of operations in a multi-request.
/// If the limit is exceeded, the connection is simply closed.
/// The constant is selected with a margin. The default limit in ZK is 1 MB of data in total.
/// The average size of the node value in this case is less than 10 kilobytes.
static constexpr auto MAX_MULTI_OPS = 100 ;
2016-01-10 04:44:12 +00:00
2019-01-04 12:10:00 +00:00
for ( size_t entry_idx = 0 , num_entries = log_entries . size ( ) ; entry_idx < num_entries ; entry_idx + = MAX_MULTI_OPS )
2017-04-01 07:20:54 +00:00
{
2019-01-04 12:10:00 +00:00
auto begin = log_entries . begin ( ) + entry_idx ;
auto end = entry_idx + MAX_MULTI_OPS > = log_entries . size ( )
2018-08-22 14:10:00 +00:00
? log_entries . end ( )
: ( begin + MAX_MULTI_OPS ) ;
2017-04-01 07:20:54 +00:00
auto last = end - 1 ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
String last_entry = * last ;
if ( ! startsWith ( last_entry , " log- " ) )
throw Exception ( " Error in zookeeper data: unexpected node " + last_entry + " in " + zookeeper_path + " /log " ,
2018-08-22 14:10:00 +00:00
ErrorCodes : : UNEXPECTED_NODE_IN_ZOOKEEPER ) ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
UInt64 last_entry_index = parse < UInt64 > ( last_entry . substr ( strlen ( " log- " ) ) ) ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
LOG_DEBUG ( log , " Pulling " < < ( end - begin ) < < " entries to queue: " < < * begin < < " - " < < * last ) ;
2016-01-12 19:45:51 +00:00
2018-08-25 01:58:14 +00:00
zkutil : : AsyncResponses < Coordination : : GetResponse > futures ;
2017-04-01 07:20:54 +00:00
futures . reserve ( end - begin ) ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
for ( auto it = begin ; it ! = end ; + + it )
futures . emplace_back ( * it , zookeeper - > asyncGet ( zookeeper_path + " /log/ " + * it ) ) ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
/// Simultaneously add all new entries to the queue and move the pointer to the log.
2016-01-17 13:00:42 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2017-04-01 07:20:54 +00:00
std : : vector < LogEntryPtr > copied_entries ;
copied_entries . reserve ( end - begin ) ;
2016-01-12 19:45:51 +00:00
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
for ( auto & future : futures )
{
2018-08-25 01:58:14 +00:00
Coordination : : GetResponse res = future . second . get ( ) ;
2018-03-24 20:11:46 +00:00
copied_entries . emplace_back ( LogEntry : : parse ( res . data , res . stat ) ) ;
2016-01-21 19:43:00 +00:00
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest (
2018-08-22 14:10:00 +00:00
replica_path + " /queue/queue- " , res . data , zkutil : : CreateMode : : PersistentSequential ) ) ;
2016-01-21 19:43:00 +00:00
2017-04-01 07:20:54 +00:00
const auto & entry = * copied_entries . back ( ) ;
if ( entry . type = = LogEntry : : GET_PART )
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
if ( entry . create_time & & ( ! min_unprocessed_insert_time | | entry . create_time < min_unprocessed_insert_time ) )
{
min_unprocessed_insert_time = entry . create_time ;
2018-03-03 16:46:32 +00:00
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
2017-04-01 07:20:54 +00:00
}
}
}
2016-01-12 19:45:51 +00:00
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2018-08-22 14:10:00 +00:00
replica_path + " /log_pointer " , toString ( last_entry_index + 1 ) , - 1 ) ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
if ( min_unprocessed_insert_time_changed )
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2018-08-22 14:10:00 +00:00
replica_path + " /min_unprocessed_insert_time " , toString ( * min_unprocessed_insert_time_changed ) , - 1 ) ) ;
2016-01-12 19:45:51 +00:00
2018-03-24 00:45:04 +00:00
auto responses = zookeeper - > multi ( ops ) ;
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
/// Now we have successfully updated the queue in ZooKeeper. Update it in RAM.
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
try
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2018-05-10 15:01:10 +00:00
log_pointer = last_entry_index + 1 ;
2019-01-04 12:10:00 +00:00
for ( size_t copied_entry_idx = 0 , num_copied_entries = copied_entries . size ( ) ; copied_entry_idx < num_copied_entries ; + + copied_entry_idx )
2017-04-01 07:20:54 +00:00
{
2019-01-04 12:10:00 +00:00
String path_created = dynamic_cast < const Coordination : : CreateResponse & > ( * responses [ copied_entry_idx ] ) . path_created ;
copied_entries [ copied_entry_idx ] - > znode_name = path_created . substr ( path_created . find_last_of ( ' / ' ) + 1 ) ;
2017-04-01 07:20:54 +00:00
2018-03-03 16:46:32 +00:00
std : : optional < time_t > unused = false ;
2019-01-04 12:10:00 +00:00
insertUnlocked ( copied_entries [ copied_entry_idx ] , unused , state_lock ) ;
2017-04-01 07:20:54 +00:00
}
2017-08-04 14:00:26 +00:00
last_queue_update = time ( nullptr ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
2020-01-15 13:00:08 +00:00
tryLogCurrentException ( log ) ;
2017-04-01 07:20:54 +00:00
/// If it fails, the data in RAM is incorrect. In order to avoid possible further corruption of data in ZK, we will kill ourselves.
/// This is possible only if there is an unknown logical error.
std : : terminate ( ) ;
}
2016-01-12 19:45:51 +00:00
2017-04-01 07:20:54 +00:00
if ( ! copied_entries . empty ( ) )
LOG_DEBUG ( log , " Pulled " < < copied_entries . size ( ) < < " entries to queue. " ) ;
}
2018-05-20 19:56:03 +00:00
if ( storage . queue_task_handle )
storage . queue_task_handle - > wake ( ) ;
}
2016-01-10 04:44:12 +00:00
}
2018-06-21 15:54:01 +00:00
2020-02-17 12:47:34 +00:00
namespace
{
2020-01-15 13:00:08 +00:00
Names getPartNamesToMutate (
2018-06-21 15:54:01 +00:00
const ReplicatedMergeTreeMutationEntry & mutation , const ActiveDataPartSet & parts )
{
2019-12-12 16:24:03 +00:00
Names result ;
2018-06-21 15:54:01 +00:00
for ( const auto & pair : mutation . block_numbers )
{
const String & partition_id = pair . first ;
Int64 block_num = pair . second ;
/// Note that we cannot simply count all parts to mutate using getPartsCoveredBy(appropriate part_info)
/// because they are not consecutive in `parts`.
MergeTreePartInfo covering_part_info (
partition_id , 0 , block_num , MergeTreePartInfo : : MAX_LEVEL , MergeTreePartInfo : : MAX_BLOCK_NUMBER ) ;
for ( const String & covered_part_name : parts . getPartsCoveredBy ( covering_part_info ) )
{
auto part_info = MergeTreePartInfo : : fromPartName ( covered_part_name , parts . getFormatVersion ( ) ) ;
if ( part_info . getDataVersion ( ) < block_num )
2019-12-12 16:24:03 +00:00
result . push_back ( covered_part_name ) ;
2018-06-21 15:54:01 +00:00
}
}
2019-12-12 16:24:03 +00:00
return result ;
2018-06-21 15:54:01 +00:00
}
2020-01-15 13:00:08 +00:00
}
2018-06-21 15:54:01 +00:00
2018-08-25 01:58:14 +00:00
void ReplicatedMergeTreeQueue : : updateMutations ( zkutil : : ZooKeeperPtr zookeeper , Coordination : : WatchCallback watch_callback )
2018-04-19 14:20:18 +00:00
{
std : : lock_guard lock ( update_mutations_mutex ) ;
2018-05-28 15:37:30 +00:00
Strings entries_in_zk = zookeeper - > getChildrenWatch ( zookeeper_path + " /mutations " , nullptr , watch_callback ) ;
2018-04-19 14:20:18 +00:00
StringSet entries_in_zk_set ( entries_in_zk . begin ( ) , entries_in_zk . end ( ) ) ;
/// Compare with the local state, delete obsolete entries and determine which new entries to load.
Strings entries_to_load ;
2019-02-05 17:22:23 +00:00
bool some_active_mutations_were_killed = false ;
2018-04-19 14:20:18 +00:00
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2018-04-19 14:20:18 +00:00
2018-11-24 01:48:06 +00:00
for ( auto it = mutations_by_znode . begin ( ) ; it ! = mutations_by_znode . end ( ) ; )
2018-04-19 14:20:18 +00:00
{
2018-06-18 12:17:46 +00:00
const ReplicatedMergeTreeMutationEntry & entry = * it - > second . entry ;
2018-04-19 14:20:18 +00:00
if ( ! entries_in_zk_set . count ( entry . znode_name ) )
{
2019-02-05 17:22:23 +00:00
if ( ! it - > second . is_done )
{
LOG_DEBUG ( log , " Removing killed mutation " + entry . znode_name + " from local state. " ) ;
some_active_mutations_were_killed = true ;
}
else
LOG_DEBUG ( log , " Removing obsolete mutation " + entry . znode_name + " from local state. " ) ;
2018-04-19 14:20:18 +00:00
for ( const auto & partition_and_block_num : entry . block_numbers )
{
auto & in_partition = mutations_by_partition [ partition_and_block_num . first ] ;
in_partition . erase ( partition_and_block_num . second ) ;
if ( in_partition . empty ( ) )
mutations_by_partition . erase ( partition_and_block_num . first ) ;
}
2018-06-18 12:17:46 +00:00
2018-04-19 14:20:18 +00:00
it = mutations_by_znode . erase ( it ) ;
}
else
+ + it ;
}
for ( const String & znode : entries_in_zk_set )
{
if ( ! mutations_by_znode . count ( znode ) )
entries_to_load . push_back ( znode ) ;
}
}
2019-02-05 17:22:23 +00:00
if ( some_active_mutations_were_killed )
storage . queue_task_handle - > wake ( ) ;
2018-04-19 14:20:18 +00:00
if ( ! entries_to_load . empty ( ) )
{
LOG_INFO ( log , " Loading " + toString ( entries_to_load . size ( ) ) + " mutation entries: "
+ entries_to_load . front ( ) + " - " + entries_to_load . back ( ) ) ;
2018-08-25 01:58:14 +00:00
std : : vector < std : : future < Coordination : : GetResponse > > futures ;
2018-04-19 14:20:18 +00:00
for ( const String & entry : entries_to_load )
futures . emplace_back ( zookeeper - > asyncGet ( zookeeper_path + " /mutations/ " + entry ) ) ;
2018-06-07 11:00:43 +00:00
std : : vector < ReplicatedMergeTreeMutationEntryPtr > new_mutations ;
2018-04-19 14:20:18 +00:00
for ( size_t i = 0 ; i < entries_to_load . size ( ) ; + + i )
{
2018-06-07 11:00:43 +00:00
new_mutations . push_back ( std : : make_shared < ReplicatedMergeTreeMutationEntry > (
ReplicatedMergeTreeMutationEntry : : parse ( futures [ i ] . get ( ) . data , entries_to_load [ i ] ) ) ) ;
2018-04-19 14:20:18 +00:00
}
2018-06-21 13:27:36 +00:00
bool some_mutations_are_probably_done = false ;
2018-04-19 14:20:18 +00:00
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2018-04-19 14:20:18 +00:00
2018-06-07 11:00:43 +00:00
for ( const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations )
2018-04-19 14:20:18 +00:00
{
2020-02-05 16:30:02 +00:00
auto & mutation = mutations_by_znode . emplace ( entry - > znode_name , MutationStatus ( entry , format_version ) )
2018-06-18 12:17:46 +00:00
. first - > second ;
for ( const auto & pair : entry - > block_numbers )
{
const String & partition_id = pair . first ;
Int64 block_num = pair . second ;
mutations_by_partition [ partition_id ] . emplace ( block_num , & mutation ) ;
}
2018-04-19 14:20:18 +00:00
2018-06-21 15:54:01 +00:00
/// Initialize `mutation.parts_to_do`. First we need to mutate all parts in `current_parts`.
2020-02-05 16:30:02 +00:00
Strings current_parts_to_mutate = getPartNamesToMutate ( * entry , current_parts ) ;
for ( const String & current_part_to_mutate : current_parts_to_mutate )
mutation . parts_to_do . add ( current_part_to_mutate ) ;
2018-06-21 15:54:01 +00:00
/// And next we would need to mutate all parts with getDataVersion() greater than
/// mutation block number that would appear as a result of executing the queue.
2018-06-18 12:17:46 +00:00
for ( const auto & queue_entry : queue )
{
for ( const String & produced_part_name : queue_entry - > getVirtualPartNames ( ) )
{
auto part_info = MergeTreePartInfo : : fromPartName ( produced_part_name , format_version ) ;
auto it = entry - > block_numbers . find ( part_info . partition_id ) ;
if ( it ! = entry - > block_numbers . end ( ) & & it - > second > part_info . getDataVersion ( ) )
2020-02-05 16:30:02 +00:00
mutation . parts_to_do . add ( produced_part_name ) ;
2018-06-18 12:17:46 +00:00
}
}
2018-06-21 13:27:36 +00:00
2020-03-08 21:40:00 +00:00
if ( mutation . parts_to_do . empty ( ) )
2020-02-14 20:13:57 +00:00
{
2018-06-21 13:27:36 +00:00
some_mutations_are_probably_done = true ;
2020-02-14 20:13:57 +00:00
}
2020-02-10 16:55:09 +00:00
2020-02-14 20:13:57 +00:00
/// otherwise it's already done
2020-02-17 16:33:05 +00:00
if ( entry - > isAlterMutation ( ) & & entry - > znode_name > mutation_pointer )
2020-02-14 20:13:57 +00:00
{
LOG_TRACE ( log , " Adding mutation " < < entry - > znode_name < < " with alter version " < < entry - > alter_version < < " to the queue " ) ;
2020-02-17 18:07:22 +00:00
alter_sequence . addMutationForAlter ( entry - > alter_version , state_lock ) ;
2020-02-14 20:13:57 +00:00
}
2018-04-19 14:20:18 +00:00
}
}
2018-04-20 16:18:16 +00:00
2018-05-31 13:05:05 +00:00
storage . merge_selecting_task - > schedule ( ) ;
2018-06-21 13:27:36 +00:00
if ( some_mutations_are_probably_done )
storage . mutations_finalizing_task - > schedule ( ) ;
2018-04-19 14:20:18 +00:00
}
}
2019-02-04 12:53:25 +00:00
ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue : : removeMutation (
zkutil : : ZooKeeperPtr zookeeper , const String & mutation_id )
{
std : : lock_guard lock ( update_mutations_mutex ) ;
auto rc = zookeeper - > tryRemove ( zookeeper_path + " /mutations/ " + mutation_id ) ;
if ( rc = = Coordination : : ZOK )
LOG_DEBUG ( log , " Removed mutation " + mutation_id + " from ZooKeeper. " ) ;
2019-02-05 17:22:23 +00:00
ReplicatedMergeTreeMutationEntryPtr entry ;
bool mutation_was_active = false ;
{
std : : lock_guard state_lock ( state_mutex ) ;
2019-02-04 12:53:25 +00:00
2019-02-05 17:22:23 +00:00
auto it = mutations_by_znode . find ( mutation_id ) ;
if ( it = = mutations_by_znode . end ( ) )
return nullptr ;
2019-02-04 12:53:25 +00:00
2019-02-05 17:22:23 +00:00
mutation_was_active = ! it - > second . is_done ;
entry = it - > second . entry ;
for ( const auto & partition_and_block_num : entry - > block_numbers )
{
auto & in_partition = mutations_by_partition [ partition_and_block_num . first ] ;
in_partition . erase ( partition_and_block_num . second ) ;
if ( in_partition . empty ( ) )
mutations_by_partition . erase ( partition_and_block_num . first ) ;
}
2020-02-17 16:33:05 +00:00
if ( entry - > isAlterMutation ( ) )
2020-02-17 13:20:09 +00:00
{
LOG_DEBUG ( log , " Removed alter " < < entry - > alter_version < < " because mutation " + entry - > znode_name + " were killed. " ) ;
2020-02-17 18:07:22 +00:00
alter_sequence . finishDataAlter ( entry - > alter_version , state_lock ) ;
2020-02-17 13:20:09 +00:00
}
2019-02-05 17:22:23 +00:00
mutations_by_znode . erase ( it ) ;
LOG_DEBUG ( log , " Removed mutation " + entry - > znode_name + " from local state. " ) ;
2019-02-04 12:53:25 +00:00
}
2019-02-05 17:22:23 +00:00
if ( mutation_was_active )
storage . queue_task_handle - > wake ( ) ;
2019-02-04 12:53:25 +00:00
return entry ;
}
2016-01-10 04:44:12 +00:00
ReplicatedMergeTreeQueue : : StringSet ReplicatedMergeTreeQueue : : moveSiblingPartsForMergeToEndOfQueue ( const String & part_name )
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
/// Let's find the action to merge this part with others. Let's remember others.
StringSet parts_for_merge ;
2019-01-09 15:44:20 +00:00
Queue : : iterator merge_entry = queue . end ( ) ;
2017-04-01 07:20:54 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; + + it )
{
2018-04-20 16:18:16 +00:00
if ( ( * it ) - > type = = LogEntry : : MERGE_PARTS | | ( * it ) - > type = = LogEntry : : MUTATE_PART )
2017-04-01 07:20:54 +00:00
{
2018-04-20 16:18:16 +00:00
if ( std : : find ( ( * it ) - > source_parts . begin ( ) , ( * it ) - > source_parts . end ( ) , part_name )
! = ( * it ) - > source_parts . end ( ) )
2017-04-01 07:20:54 +00:00
{
2018-04-20 16:18:16 +00:00
parts_for_merge = StringSet ( ( * it ) - > source_parts . begin ( ) , ( * it ) - > source_parts . end ( ) ) ;
2017-04-01 07:20:54 +00:00
merge_entry = it ;
break ;
}
}
}
if ( ! parts_for_merge . empty ( ) )
{
2018-05-10 15:01:10 +00:00
/// Move to the end of queue actions that result in one of the parts in `parts_for_merge`.
2017-04-01 07:20:54 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; )
{
auto it0 = it ;
+ + it ;
if ( it0 = = merge_entry )
break ;
2018-04-20 16:18:16 +00:00
if ( ( ( * it0 ) - > type = = LogEntry : : MERGE_PARTS | | ( * it0 ) - > type = = LogEntry : : GET_PART | | ( * it0 ) - > type = = LogEntry : : MUTATE_PART )
2017-04-01 07:20:54 +00:00
& & parts_for_merge . count ( ( * it0 ) - > new_part_name ) )
{
queue . splice ( queue . end ( ) , queue , it0 , it ) ;
}
}
}
return parts_for_merge ;
2016-01-10 04:44:12 +00:00
}
2018-08-06 12:59:51 +00:00
bool ReplicatedMergeTreeQueue : : checkReplaceRangeCanBeRemoved ( const MergeTreePartInfo & part_info , const LogEntryPtr entry_ptr , const ReplicatedMergeTreeLogEntryData & current ) const
{
if ( entry_ptr - > type ! = LogEntry : : REPLACE_RANGE )
return false ;
if ( current . type ! = LogEntry : : REPLACE_RANGE & & current . type ! = LogEntry : : DROP_RANGE )
return false ;
if ( entry_ptr - > replace_range_entry ! = nullptr & & entry_ptr - > replace_range_entry = = current . replace_range_entry ) /// same partition, don't want to drop ourselves
return false ;
for ( const String & new_part_name : entry_ptr - > replace_range_entry - > new_part_names )
if ( ! part_info . contains ( MergeTreePartInfo : : fromPartName ( new_part_name , format_version ) ) )
return false ;
2016-01-10 04:44:12 +00:00
2018-08-06 12:59:51 +00:00
return true ;
}
2016-01-10 04:44:12 +00:00
2019-08-16 15:57:19 +00:00
void ReplicatedMergeTreeQueue : : removePartProducingOpsInRange (
zkutil : : ZooKeeperPtr zookeeper ,
const MergeTreePartInfo & part_info ,
const ReplicatedMergeTreeLogEntryData & current )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
Queue to_wait ;
size_t removed_entries = 0 ;
2018-03-03 16:46:32 +00:00
std : : optional < time_t > min_unprocessed_insert_time_changed ;
std : : optional < time_t > max_processed_insert_time_changed ;
2017-04-01 07:20:54 +00:00
/// Remove operations with parts, contained in the range to be deleted, from the queue.
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; )
{
2018-05-21 13:49:54 +00:00
auto type = ( * it ) - > type ;
2018-08-06 12:59:51 +00:00
if ( ( ( type = = LogEntry : : GET_PART | | type = = LogEntry : : MERGE_PARTS | | type = = LogEntry : : MUTATE_PART )
& & part_info . contains ( MergeTreePartInfo : : fromPartName ( ( * it ) - > new_part_name , format_version ) ) )
| | checkReplaceRangeCanBeRemoved ( part_info , * it , current ) )
2017-04-01 07:20:54 +00:00
{
if ( ( * it ) - > currently_executing )
to_wait . push_back ( * it ) ;
auto code = zookeeper - > tryRemove ( replica_path + " /queue/ " + ( * it ) - > znode_name ) ;
2018-03-24 00:45:04 +00:00
if ( code )
2017-04-01 07:20:54 +00:00
LOG_INFO ( log , " Couldn't remove " < < replica_path + " /queue/ " + ( * it ) - > znode_name < < " : "
< < zkutil : : ZooKeeper : : error2string ( code ) ) ;
2018-06-18 12:17:46 +00:00
updateStateOnQueueEntryRemoval (
* it , /* is_successful = */ false ,
min_unprocessed_insert_time_changed , max_processed_insert_time_changed , lock ) ;
2017-04-01 07:20:54 +00:00
queue . erase ( it + + ) ;
+ + removed_entries ;
}
else
+ + it ;
}
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , max_processed_insert_time_changed ) ;
LOG_DEBUG ( log , " Removed " < < removed_entries < < " entries from queue. "
" Waiting for " < < to_wait . size ( ) < < " entries that are currently executing. " ) ;
/// Let's wait for the operations with the parts contained in the range to be deleted.
for ( LogEntryPtr & entry : to_wait )
entry - > execution_complete . wait ( lock , [ & entry ] { return ! entry - > currently_executing ; } ) ;
2016-01-10 04:44:12 +00:00
}
2018-05-28 15:37:30 +00:00
size_t ReplicatedMergeTreeQueue : : getConflictsCountForRange (
2018-05-31 15:52:42 +00:00
const MergeTreePartInfo & range , const LogEntry & entry ,
String * out_description , std : : lock_guard < std : : mutex > & /* queue_lock */ ) const
2017-06-16 16:47:09 +00:00
{
2018-05-23 14:33:55 +00:00
std : : vector < std : : pair < String , LogEntryPtr > > conflicts ;
2017-06-16 16:47:09 +00:00
2018-05-23 14:33:55 +00:00
for ( auto & future_part_elem : future_parts )
2017-06-16 16:47:09 +00:00
{
2018-05-23 14:33:55 +00:00
/// Do not check itself log entry
if ( future_part_elem . second - > znode_name = = entry . znode_name )
2018-05-21 13:49:54 +00:00
continue ;
2017-06-16 16:47:09 +00:00
2018-05-23 14:33:55 +00:00
if ( ! range . isDisjoint ( MergeTreePartInfo : : fromPartName ( future_part_elem . first , format_version ) ) )
2018-05-21 13:49:54 +00:00
{
2018-05-23 14:33:55 +00:00
conflicts . emplace_back ( future_part_elem . first , future_part_elem . second ) ;
continue ;
2017-06-16 16:47:09 +00:00
}
}
2018-05-23 14:33:55 +00:00
if ( out_description )
2017-06-16 16:47:09 +00:00
{
std : : stringstream ss ;
2018-05-23 14:33:55 +00:00
ss < < " Can't execute command for range " < < range . getPartName ( ) < < " (entry " < < entry . znode_name < < " ). " ;
2017-06-16 16:47:09 +00:00
ss < < " There are " < < conflicts . size ( ) < < " currently executing entries blocking it: " ;
for ( const auto & conflict : conflicts )
2018-05-23 14:33:55 +00:00
ss < < conflict . second - > typeToString ( ) < < " part " < < conflict . first < < " , " ;
2017-06-16 16:47:09 +00:00
2018-05-23 14:33:55 +00:00
* out_description = ss . str ( ) ;
2017-06-16 16:47:09 +00:00
}
2018-05-21 13:49:54 +00:00
return conflicts . size ( ) ;
2017-06-16 16:47:09 +00:00
}
2018-05-23 14:33:55 +00:00
void ReplicatedMergeTreeQueue : : checkThereAreNoConflictsInRange ( const MergeTreePartInfo & range , const LogEntry & entry )
2017-06-16 16:47:09 +00:00
{
String conflicts_description ;
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-06-16 16:47:09 +00:00
2018-05-23 14:33:55 +00:00
if ( 0 ! = getConflictsCountForRange ( range , entry , & conflicts_description , lock ) )
2017-06-16 16:47:09 +00:00
throw Exception ( conflicts_description , ErrorCodes : : UNFINISHED ) ;
}
2018-05-10 15:01:10 +00:00
bool ReplicatedMergeTreeQueue : : isNotCoveredByFuturePartsImpl ( const String & new_part_name , String & out_reason , std : : lock_guard < std : : mutex > & /* queue_lock */ ) const
2017-05-12 13:47:42 +00:00
{
/// Let's check if the same part is now being created by another action.
if ( future_parts . count ( new_part_name ) )
{
out_reason = " Not executing log entry for part " + new_part_name
+ " because another log entry for the same part is being processed. This shouldn't happen often. " ;
return false ;
/** When the corresponding action is completed, then `isNotCoveredByFuturePart` next time, will succeed,
* and queue element will be processed .
* Immediately in the ` executeLogEntry ` function it will be found that we already have a part ,
* and queue element will be immediately treated as processed .
*/
}
/// A more complex check is whether another part is currently created by other action that will cover this part.
/// NOTE The above is redundant, but left for a more convenient message in the log.
2017-11-28 14:07:17 +00:00
auto result_part = MergeTreePartInfo : : fromPartName ( new_part_name , format_version ) ;
2017-05-12 13:47:42 +00:00
/// It can slow down when the size of `future_parts` is large. But it can not be large, since `BackgroundProcessingPool` is limited.
2018-05-23 14:33:55 +00:00
for ( const auto & future_part_elem : future_parts )
2017-05-12 13:47:42 +00:00
{
2018-05-23 14:33:55 +00:00
auto future_part = MergeTreePartInfo : : fromPartName ( future_part_elem . first , format_version ) ;
2017-05-12 13:47:42 +00:00
if ( future_part . contains ( result_part ) )
{
2018-05-21 13:49:54 +00:00
out_reason = " Not executing log entry for part " + new_part_name + " because it is covered by part "
2018-05-23 14:33:55 +00:00
+ future_part_elem . first + " that is currently executing " ;
2017-05-12 13:47:42 +00:00
return false ;
}
}
return true ;
}
2018-05-23 14:33:55 +00:00
bool ReplicatedMergeTreeQueue : : addFuturePartIfNotCoveredByThem ( const String & part_name , LogEntry & entry , String & reject_reason )
2017-05-12 13:47:42 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-05-12 13:47:42 +00:00
2018-03-03 16:26:06 +00:00
if ( isNotCoveredByFuturePartsImpl ( part_name , reject_reason , lock ) )
2017-05-12 13:47:42 +00:00
{
CurrentlyExecuting : : setActualPartName ( entry , part_name , * this ) ;
return true ;
}
return false ;
}
2016-10-30 11:05:45 +00:00
bool ReplicatedMergeTreeQueue : : shouldExecuteLogEntry (
2017-04-01 07:20:54 +00:00
const LogEntry & entry ,
String & out_postpone_reason ,
2018-04-20 16:18:16 +00:00
MergeTreeDataMergerMutator & merger_mutator ,
2018-03-03 16:26:06 +00:00
MergeTreeData & data ,
2020-02-01 20:33:36 +00:00
std : : lock_guard < std : : mutex > & state_lock ) const
2016-01-10 04:44:12 +00:00
{
2020-02-17 16:33:05 +00:00
/// If our entry produce part which is alredy covered by
/// some other entry which is currently executing, then we can postpone this entry.
2018-04-20 16:18:16 +00:00
if ( entry . type = = LogEntry : : MERGE_PARTS
| | entry . type = = LogEntry : : GET_PART
| | entry . type = = LogEntry : : MUTATE_PART )
2017-04-01 07:20:54 +00:00
{
2018-05-23 12:29:36 +00:00
for ( const String & new_part_name : entry . getBlockingPartNames ( ) )
2017-04-01 07:20:54 +00:00
{
2020-02-01 20:33:36 +00:00
if ( ! isNotCoveredByFuturePartsImpl ( new_part_name , out_postpone_reason , state_lock ) )
2018-05-21 13:49:54 +00:00
{
if ( ! out_postpone_reason . empty ( ) )
LOG_DEBUG ( log , out_postpone_reason ) ;
return false ;
}
2017-04-01 07:20:54 +00:00
}
}
2018-04-20 16:18:16 +00:00
if ( entry . type = = LogEntry : : MERGE_PARTS | | entry . type = = LogEntry : : MUTATE_PART )
2017-04-01 07:20:54 +00:00
{
2018-04-20 16:18:16 +00:00
/** If any of the required parts are now fetched or in merge process, wait for the end of this operation.
2017-04-01 07:20:54 +00:00
* Otherwise , even if all the necessary parts for the merge are not present , you should try to make a merge .
* If any parts are missing , instead of merge , there will be an attempt to download a part .
* Such a situation is possible if the receive of a part has failed , and it was moved to the end of the queue .
*/
size_t sum_parts_size_in_bytes = 0 ;
2018-04-20 16:18:16 +00:00
for ( const auto & name : entry . source_parts )
2017-04-01 07:20:54 +00:00
{
if ( future_parts . count ( name ) )
{
String reason = " Not merging into part " + entry . new_part_name
+ " because part " + name + " is not ready yet (log entry for that part is being processed). " ;
LOG_TRACE ( log , reason ) ;
out_postpone_reason = reason ;
return false ;
}
2017-12-20 17:52:28 +00:00
auto part = data . getPartIfExists ( name , { MergeTreeDataPartState : : PreCommitted , MergeTreeDataPartState : : Committed , MergeTreeDataPartState : : Outdated } ) ;
2017-04-01 07:20:54 +00:00
if ( part )
2018-03-26 14:18:04 +00:00
sum_parts_size_in_bytes + = part - > bytes_on_disk ;
2017-04-01 07:20:54 +00:00
}
2019-08-01 15:36:12 +00:00
if ( merger_mutator . merges_blocker . isCancelled ( ) )
2017-04-01 07:20:54 +00:00
{
2018-04-20 16:18:16 +00:00
String reason = " Not executing log entry for part " + entry . new_part_name + " because merges and mutations are cancelled now. " ;
2017-04-01 07:20:54 +00:00
LOG_DEBUG ( log , reason ) ;
out_postpone_reason = reason ;
return false ;
}
2019-08-22 19:35:46 +00:00
UInt64 max_source_parts_size = entry . type = = LogEntry : : MERGE_PARTS ? merger_mutator . getMaxSourcePartsSizeForMerge ( )
: merger_mutator . getMaxSourcePartSizeForMutation ( ) ;
/** If there are enough free threads in background pool to do large merges (maximal size of merge is allowed),
* then ignore value returned by getMaxSourcePartsSizeForMerge ( ) and execute merge of any size ,
* because it may be ordered by OPTIMIZE or early with different settings .
* Setting max_bytes_to_merge_at_max_space_in_pool still working for regular merges ,
* because the leader replica does not assign merges of greater size ( except OPTIMIZE PARTITION and OPTIMIZE FINAL ) .
2017-04-01 07:20:54 +00:00
*/
2019-08-26 14:24:29 +00:00
const auto data_settings = data . getSettings ( ) ;
2019-08-26 10:07:32 +00:00
bool ignore_max_size = ( entry . type = = LogEntry : : MERGE_PARTS ) & & ( max_source_parts_size = = data_settings - > max_bytes_to_merge_at_max_space_in_pool ) ;
2019-08-22 19:35:46 +00:00
if ( ! ignore_max_size & & sum_parts_size_in_bytes > max_source_parts_size )
2017-04-01 07:20:54 +00:00
{
2019-08-22 19:35:46 +00:00
String reason = " Not executing log entry " + entry . typeToString ( ) + " for part " + entry . new_part_name
2018-04-20 16:18:16 +00:00
+ " because source parts size ( " + formatReadableSizeWithBinarySuffix ( sum_parts_size_in_bytes )
+ " ) is greater than the current maximum ( " + formatReadableSizeWithBinarySuffix ( max_source_parts_size ) + " ). " ;
2017-04-01 07:20:54 +00:00
LOG_DEBUG ( log , reason ) ;
out_postpone_reason = reason ;
return false ;
}
}
2018-05-21 13:49:54 +00:00
/// TODO: it makes sense to check DROP_RANGE also
if ( entry . type = = LogEntry : : CLEAR_COLUMN | | entry . type = = LogEntry : : REPLACE_RANGE )
2017-06-16 16:47:09 +00:00
{
String conflicts_description ;
2018-05-21 13:49:54 +00:00
String range_name = ( entry . type = = LogEntry : : REPLACE_RANGE ) ? entry . replace_range_entry - > drop_range_part_name : entry . new_part_name ;
auto range = MergeTreePartInfo : : fromPartName ( range_name , format_version ) ;
2020-02-01 20:33:36 +00:00
if ( 0 ! = getConflictsCountForRange ( range , entry , & conflicts_description , state_lock ) )
2017-06-16 16:47:09 +00:00
{
LOG_DEBUG ( log , conflicts_description ) ;
return false ;
}
}
2020-02-17 16:33:05 +00:00
/// Alters must be executed one by one. First metadata change, and after that data alter (MUTATE_PART entries with).
/// corresponding alter_version.
2020-01-28 17:15:22 +00:00
if ( entry . type = = LogEntry : : ALTER_METADATA )
2020-02-13 11:38:04 +00:00
{
2020-02-17 18:07:22 +00:00
if ( ! alter_sequence . canExecuteMetaAlter ( entry . alter_version , state_lock ) )
2020-02-05 11:18:11 +00:00
{
2020-02-17 18:07:22 +00:00
int head_alter = alter_sequence . getHeadAlterVersion ( state_lock ) ;
2020-02-13 16:16:09 +00:00
out_postpone_reason = " Cannot execute alter metadata with version: " + std : : to_string ( entry . alter_version )
2020-02-13 14:48:38 +00:00
+ " because another alter " + std : : to_string ( head_alter )
+ " must be executed before " ;
2020-02-05 11:18:11 +00:00
return false ;
}
2020-01-31 12:25:31 +00:00
}
2020-02-17 16:33:05 +00:00
/// If this MUTATE_PART is part of alter modify/drop query, than we have to execute them one by one
if ( entry . isAlterMutation ( ) )
2020-01-31 12:25:31 +00:00
{
2020-02-17 18:07:22 +00:00
if ( ! alter_sequence . canExecuteDataAlter ( entry . alter_version , state_lock ) )
2020-02-05 11:18:11 +00:00
{
2020-02-17 18:07:22 +00:00
int head_alter = alter_sequence . getHeadAlterVersion ( state_lock ) ;
2020-02-13 14:48:38 +00:00
if ( head_alter = = entry . alter_version )
out_postpone_reason = " Cannot execute alter data with version: "
+ std : : to_string ( entry . alter_version ) + " because metadata still not altered " ;
else
out_postpone_reason = " Cannot execute alter data with version: " + std : : to_string ( entry . alter_version )
+ " because another alter " + std : : to_string ( head_alter ) + " must be executed before " ;
2020-02-05 11:18:11 +00:00
return false ;
}
2020-01-13 16:39:20 +00:00
}
2017-04-01 07:20:54 +00:00
return true ;
2016-01-10 04:44:12 +00:00
}
2018-05-14 14:51:33 +00:00
Int64 ReplicatedMergeTreeQueue : : getCurrentMutationVersionImpl (
2018-06-20 11:12:16 +00:00
const String & partition_id , Int64 data_version , std : : lock_guard < std : : mutex > & /* state_lock */ ) const
2018-05-12 23:05:04 +00:00
{
2018-05-14 14:51:33 +00:00
auto in_partition = mutations_by_partition . find ( partition_id ) ;
2018-05-12 23:05:04 +00:00
if ( in_partition = = mutations_by_partition . end ( ) )
return 0 ;
2018-05-14 14:51:33 +00:00
auto it = in_partition - > second . upper_bound ( data_version ) ;
2018-05-12 23:05:04 +00:00
if ( it = = in_partition - > second . begin ( ) )
return 0 ;
2018-04-20 16:18:16 +00:00
2018-05-12 23:05:04 +00:00
- - it ;
return it - > first ;
}
2018-05-14 14:51:33 +00:00
Int64 ReplicatedMergeTreeQueue : : getCurrentMutationVersion ( const String & partition_id , Int64 data_version ) const
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-05-14 14:51:33 +00:00
return getCurrentMutationVersionImpl ( partition_id , data_version , lock ) ;
}
2019-08-03 11:02:40 +00:00
ReplicatedMergeTreeQueue : : CurrentlyExecuting : : CurrentlyExecuting ( const ReplicatedMergeTreeQueue : : LogEntryPtr & entry_ , ReplicatedMergeTreeQueue & queue_ )
: entry ( entry_ ) , queue ( queue_ )
2016-02-02 21:30:27 +00:00
{
2017-04-01 07:20:54 +00:00
entry - > currently_executing = true ;
+ + entry - > num_tries ;
2017-08-04 14:00:26 +00:00
entry - > last_attempt_time = time ( nullptr ) ;
2016-02-02 21:30:27 +00:00
2018-05-23 12:29:36 +00:00
for ( const String & new_part_name : entry - > getBlockingPartNames ( ) )
2018-05-21 13:49:54 +00:00
{
2018-05-23 14:33:55 +00:00
if ( ! queue . future_parts . emplace ( new_part_name , entry ) . second )
2018-05-21 13:49:54 +00:00
throw Exception ( " Tagging already tagged future part " + new_part_name + " . This is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2016-02-02 21:30:27 +00:00
}
2017-05-12 13:47:42 +00:00
2018-05-23 14:33:55 +00:00
void ReplicatedMergeTreeQueue : : CurrentlyExecuting : : setActualPartName ( ReplicatedMergeTreeQueue : : LogEntry & entry ,
2017-05-12 13:47:42 +00:00
const String & actual_part_name , ReplicatedMergeTreeQueue & queue )
{
if ( ! entry . actual_new_part_name . empty ( ) )
throw Exception ( " Entry actual part isn't empty yet. This is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
entry . actual_new_part_name = actual_part_name ;
/// Check if it is the same (and already added) part.
if ( entry . actual_new_part_name = = entry . new_part_name )
return ;
2018-05-23 14:33:55 +00:00
if ( ! queue . future_parts . emplace ( entry . actual_new_part_name , entry . shared_from_this ( ) ) . second )
2018-05-23 12:29:36 +00:00
throw Exception ( " Attaching already existing future part " + entry . actual_new_part_name + " . This is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-05-12 13:47:42 +00:00
}
2016-02-02 21:30:27 +00:00
ReplicatedMergeTreeQueue : : CurrentlyExecuting : : ~ CurrentlyExecuting ( )
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2016-02-02 21:30:27 +00:00
2017-04-01 07:20:54 +00:00
entry - > currently_executing = false ;
entry - > execution_complete . notify_all ( ) ;
2016-02-02 21:30:27 +00:00
2018-05-23 12:29:36 +00:00
for ( const String & new_part_name : entry - > getBlockingPartNames ( ) )
2018-05-21 13:49:54 +00:00
{
if ( ! queue . future_parts . erase ( new_part_name ) )
LOG_ERROR ( queue . log , " Untagging already untagged future part " + new_part_name + " . This is a bug. " ) ;
}
2017-05-12 13:47:42 +00:00
if ( ! entry - > actual_new_part_name . empty ( ) )
{
if ( entry - > actual_new_part_name ! = entry - > new_part_name & & ! queue . future_parts . erase ( entry - > actual_new_part_name ) )
LOG_ERROR ( queue . log , " Untagging already untagged future part " + entry - > actual_new_part_name + " . This is a bug. " ) ;
entry - > actual_new_part_name . clear ( ) ;
}
2016-02-02 21:30:27 +00:00
}
2018-04-20 16:18:16 +00:00
ReplicatedMergeTreeQueue : : SelectedEntry ReplicatedMergeTreeQueue : : selectEntryToProcess ( MergeTreeDataMergerMutator & merger_mutator , MergeTreeData & data )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
LogEntryPtr entry ;
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-05-21 13:49:54 +00:00
2017-05-12 13:47:42 +00:00
for ( auto it = queue . begin ( ) ; it ! = queue . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
if ( ( * it ) - > currently_executing )
continue ;
2018-04-20 16:18:16 +00:00
if ( shouldExecuteLogEntry ( * * it , ( * it ) - > postpone_reason , merger_mutator , data , lock ) )
2017-04-01 07:20:54 +00:00
{
entry = * it ;
2020-02-17 18:07:22 +00:00
/// We gave a chance for the entry, move it to the tail of the queue, after that
/// we move it to the end of the queue.
2017-04-01 07:20:54 +00:00
queue . splice ( queue . end ( ) , queue , it ) ;
break ;
}
else
{
+ + ( * it ) - > num_postponed ;
2017-08-04 14:00:26 +00:00
( * it ) - > last_postpone_time = time ( nullptr ) ;
2017-04-01 07:20:54 +00:00
}
}
if ( entry )
return { entry , std : : unique_ptr < CurrentlyExecuting > { new CurrentlyExecuting ( entry , * this ) } } ;
else
return { } ;
2016-01-10 04:44:12 +00:00
}
2016-10-24 12:34:08 +00:00
bool ReplicatedMergeTreeQueue : : processEntry (
2017-04-01 07:20:54 +00:00
std : : function < zkutil : : ZooKeeperPtr ( ) > get_zookeeper ,
LogEntryPtr & entry ,
const std : : function < bool ( LogEntryPtr & ) > func )
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
std : : exception_ptr saved_exception ;
try
{
2020-02-17 18:07:22 +00:00
/// We don't have any backoff for failed entries
/// we just count amount of tries for each ot them.
2017-04-01 07:20:54 +00:00
if ( func ( entry ) )
2018-06-06 19:15:10 +00:00
removeProcessedEntry ( get_zookeeper ( ) , entry ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
saved_exception = std : : current_exception ( ) ;
}
if ( saved_exception )
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2019-02-05 13:03:52 +00:00
2017-04-01 07:20:54 +00:00
entry - > exception = saved_exception ;
2019-02-05 13:03:52 +00:00
if ( entry - > type = = ReplicatedMergeTreeLogEntryData : : MUTATE_PART )
{
/// Record the exception in the system.mutations table.
Int64 result_data_version = MergeTreePartInfo : : fromPartName ( entry - > new_part_name , format_version )
. getDataVersion ( ) ;
auto source_part_info = MergeTreePartInfo : : fromPartName (
entry - > source_parts . at ( 0 ) , format_version ) ;
auto in_partition = mutations_by_partition . find ( source_part_info . partition_id ) ;
if ( in_partition ! = mutations_by_partition . end ( ) )
{
auto mutations_begin_it = in_partition - > second . upper_bound ( source_part_info . getDataVersion ( ) ) ;
auto mutations_end_it = in_partition - > second . upper_bound ( result_data_version ) ;
for ( auto it = mutations_begin_it ; it ! = mutations_end_it ; + + it )
{
MutationStatus & status = * it - > second ;
status . latest_failed_part = entry - > source_parts . at ( 0 ) ;
status . latest_failed_part_info = source_part_info ;
status . latest_fail_time = time ( nullptr ) ;
status . latest_fail_reason = getExceptionMessage ( saved_exception , false ) ;
}
}
}
2017-04-01 07:20:54 +00:00
return false ;
}
return true ;
2016-01-10 04:44:12 +00:00
}
2019-08-21 13:10:33 +00:00
std : : pair < size_t , size_t > ReplicatedMergeTreeQueue : : countMergesAndPartMutations ( ) const
2018-07-31 11:36:08 +00:00
{
std : : lock_guard lock ( state_mutex ) ;
2019-08-21 13:10:33 +00:00
size_t count_merges = 0 ;
size_t count_mutations = 0 ;
2018-07-31 11:36:08 +00:00
for ( const auto & entry : queue )
2019-08-21 13:10:33 +00:00
{
if ( entry - > type = = ReplicatedMergeTreeLogEntry : : MERGE_PARTS )
+ + count_merges ;
else if ( entry - > type = = ReplicatedMergeTreeLogEntry : : MUTATE_PART )
+ + count_mutations ;
}
2018-07-31 11:36:08 +00:00
2019-08-21 13:10:33 +00:00
return std : : make_pair ( count_merges , count_mutations ) ;
2018-07-31 11:36:08 +00:00
}
size_t ReplicatedMergeTreeQueue : : countMutations ( ) const
{
std : : lock_guard lock ( state_mutex ) ;
return mutations_by_znode . size ( ) ;
}
size_t ReplicatedMergeTreeQueue : : countFinishedMutations ( ) const
{
std : : lock_guard lock ( state_mutex ) ;
size_t count = 0 ;
for ( const auto & pair : mutations_by_znode )
{
const auto & mutation = pair . second ;
if ( ! mutation . is_done )
break ;
+ + count ;
}
return count ;
}
2018-05-12 23:05:04 +00:00
ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue : : getMergePredicate ( zkutil : : ZooKeeperPtr & zookeeper )
2016-01-10 04:44:12 +00:00
{
2018-05-12 23:05:04 +00:00
return ReplicatedMergeTreeMergePredicate ( * this , zookeeper ) ;
2016-01-10 04:44:12 +00:00
}
2018-05-10 15:01:10 +00:00
2018-04-20 19:11:20 +00:00
MutationCommands ReplicatedMergeTreeQueue : : getMutationCommands (
const MergeTreeData : : DataPartPtr & part , Int64 desired_mutation_version ) const
{
2018-06-05 14:55:35 +00:00
/// NOTE: If the corresponding mutation is not found, the error is logged (and not thrown as an exception)
/// to allow recovering from a mutation that cannot be executed. This way you can delete the mutation entry
/// from /mutations in ZK and the replicas will simply skip the mutation.
2018-05-31 15:21:00 +00:00
if ( part - > info . getDataVersion ( ) > desired_mutation_version )
{
LOG_WARNING ( log , " Data version of part " < < part - > name < < " is already greater than "
" desired mutation version " < < desired_mutation_version ) ;
return MutationCommands { } ;
}
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-04-20 19:11:20 +00:00
auto in_partition = mutations_by_partition . find ( part - > info . partition_id ) ;
if ( in_partition = = mutations_by_partition . end ( ) )
2018-05-31 15:21:00 +00:00
{
2019-02-04 12:53:25 +00:00
LOG_WARNING ( log , " There are no mutations for partition ID " < < part - > info . partition_id
< < " (trying to mutate part " < < part - > name < < " to " < < toString ( desired_mutation_version ) < < " ) " ) ;
2018-05-31 15:21:00 +00:00
return MutationCommands { } ;
}
2018-04-20 19:11:20 +00:00
auto begin = in_partition - > second . upper_bound ( part - > info . getDataVersion ( ) ) ;
2018-05-31 15:21:00 +00:00
auto end = in_partition - > second . lower_bound ( desired_mutation_version ) ;
if ( end = = in_partition - > second . end ( ) | | end - > first ! = desired_mutation_version )
2019-02-04 12:53:25 +00:00
LOG_WARNING ( log , " Mutation with version " < < desired_mutation_version
2018-05-31 15:21:00 +00:00
< < " not found in partition ID " < < part - > info . partition_id
< < " (trying to mutate part " < < part - > name + " ) " ) ;
else
+ + end ;
2018-04-20 19:11:20 +00:00
2018-06-13 13:49:27 +00:00
MutationCommands commands ;
2018-04-20 19:11:20 +00:00
for ( auto it = begin ; it ! = end ; + + it )
2018-06-18 12:17:46 +00:00
commands . insert ( commands . end ( ) , it - > second - > entry - > commands . begin ( ) , it - > second - > entry - > commands . end ( ) ) ;
2018-04-20 19:11:20 +00:00
2018-06-13 13:49:27 +00:00
return commands ;
2018-04-20 19:11:20 +00:00
}
2018-06-21 13:27:36 +00:00
bool ReplicatedMergeTreeQueue : : tryFinalizeMutations ( zkutil : : ZooKeeperPtr zookeeper )
{
std : : vector < ReplicatedMergeTreeMutationEntryPtr > candidates ;
{
std : : lock_guard lock ( state_mutex ) ;
for ( auto & kv : mutations_by_znode )
{
const String & znode = kv . first ;
MutationStatus & mutation = kv . second ;
if ( mutation . is_done )
continue ;
if ( znode < = mutation_pointer )
{
LOG_TRACE ( log , " Marking mutation " < < znode < < " done because it is <= mutation_pointer ( " < < mutation_pointer < < " ) " ) ;
mutation . is_done = true ;
2020-02-17 18:07:22 +00:00
alter_sequence . finishDataAlter ( mutation . entry - > alter_version , lock ) ;
2020-02-05 16:30:02 +00:00
if ( mutation . parts_to_do . size ( ) ! = 0 )
{
LOG_INFO ( log , " Seems like we jumped over mutation " < < znode < < " when downloaded part with bigger mutation number. "
< < " It's OK, tasks for rest parts will be skipped, but probably a lot of mutations were executed concurrently on different replicas. " ) ;
mutation . parts_to_do . clear ( ) ;
}
2018-06-21 13:27:36 +00:00
}
2020-03-08 21:40:00 +00:00
else if ( mutation . parts_to_do . empty ( ) )
2018-06-21 13:27:36 +00:00
{
LOG_TRACE ( log , " Will check if mutation " < < mutation . entry - > znode_name < < " is done " ) ;
candidates . push_back ( mutation . entry ) ;
}
}
}
if ( candidates . empty ( ) )
return false ;
auto merge_pred = getMergePredicate ( zookeeper ) ;
std : : vector < const ReplicatedMergeTreeMutationEntry * > finished ;
for ( const ReplicatedMergeTreeMutationEntryPtr & candidate : candidates )
{
if ( merge_pred . isMutationFinished ( * candidate ) )
finished . push_back ( candidate . get ( ) ) ;
}
if ( ! finished . empty ( ) )
2019-08-19 18:12:22 +00:00
{
2018-06-21 13:27:36 +00:00
zookeeper - > set ( replica_path + " /mutation_pointer " , finished . back ( ) - > znode_name ) ;
std : : lock_guard lock ( state_mutex ) ;
2018-07-31 11:36:08 +00:00
mutation_pointer = finished . back ( ) - > znode_name ;
2018-06-21 13:27:36 +00:00
for ( const ReplicatedMergeTreeMutationEntry * entry : finished )
{
auto it = mutations_by_znode . find ( entry - > znode_name ) ;
if ( it ! = mutations_by_znode . end ( ) )
{
LOG_TRACE ( log , " Mutation " < < entry - > znode_name < < " is done " ) ;
it - > second . is_done = true ;
2020-02-17 16:33:05 +00:00
if ( entry - > isAlterMutation ( ) )
2020-02-05 11:18:11 +00:00
{
LOG_TRACE ( log , " Finishing data alter with version " < < entry - > alter_version < < " for entry " < < entry - > znode_name ) ;
2020-02-17 18:07:22 +00:00
alter_sequence . finishDataAlter ( entry - > alter_version , lock ) ;
2020-02-05 11:18:11 +00:00
}
2018-06-21 13:27:36 +00:00
}
}
}
2020-02-28 08:55:17 +00:00
/// Mutations may finish in non sequential order because we may fetch
/// already mutated parts from other replicas. So, because we updated
/// mutation pointer we have to recheck all previous mutations, they may be
/// also finished.
return ! finished . empty ( ) ;
2018-06-21 13:27:36 +00:00
}
2019-08-16 15:57:19 +00:00
void ReplicatedMergeTreeQueue : : disableMergesInBlockRange ( const String & part_name )
2016-01-10 04:44:12 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
virtual_parts . add ( part_name ) ;
2016-01-10 04:44:12 +00:00
}
2018-03-03 16:26:06 +00:00
ReplicatedMergeTreeQueue : : Status ReplicatedMergeTreeQueue : : getStatus ( ) const
2016-01-10 04:44:12 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
Status res ;
res . future_parts = future_parts . size ( ) ;
res . queue_size = queue . size ( ) ;
res . last_queue_update = last_queue_update ;
res . inserts_in_queue = 0 ;
res . merges_in_queue = 0 ;
2018-06-06 13:22:30 +00:00
res . part_mutations_in_queue = 0 ;
2017-04-01 07:20:54 +00:00
res . queue_oldest_time = 0 ;
res . inserts_oldest_time = 0 ;
res . merges_oldest_time = 0 ;
2018-06-06 13:22:30 +00:00
res . part_mutations_oldest_time = 0 ;
2017-04-01 07:20:54 +00:00
for ( const LogEntryPtr & entry : queue )
{
if ( entry - > create_time & & ( ! res . queue_oldest_time | | entry - > create_time < res . queue_oldest_time ) )
res . queue_oldest_time = entry - > create_time ;
if ( entry - > type = = LogEntry : : GET_PART )
{
+ + res . inserts_in_queue ;
if ( entry - > create_time & & ( ! res . inserts_oldest_time | | entry - > create_time < res . inserts_oldest_time ) )
{
res . inserts_oldest_time = entry - > create_time ;
res . oldest_part_to_get = entry - > new_part_name ;
}
}
if ( entry - > type = = LogEntry : : MERGE_PARTS )
{
+ + res . merges_in_queue ;
if ( entry - > create_time & & ( ! res . merges_oldest_time | | entry - > create_time < res . merges_oldest_time ) )
{
res . merges_oldest_time = entry - > create_time ;
res . oldest_part_to_merge_to = entry - > new_part_name ;
}
}
2018-04-20 16:18:16 +00:00
if ( entry - > type = = LogEntry : : MUTATE_PART )
{
2018-06-06 13:22:30 +00:00
+ + res . part_mutations_in_queue ;
2018-04-20 16:18:16 +00:00
2018-06-06 13:22:30 +00:00
if ( entry - > create_time & & ( ! res . part_mutations_oldest_time | | entry - > create_time < res . part_mutations_oldest_time ) )
2018-04-20 16:18:16 +00:00
{
2018-06-06 13:22:30 +00:00
res . part_mutations_oldest_time = entry - > create_time ;
2018-04-20 16:18:16 +00:00
res . oldest_part_to_mutate_to = entry - > new_part_name ;
}
}
2017-04-01 07:20:54 +00:00
}
return res ;
2016-01-10 04:44:12 +00:00
}
2018-03-03 16:26:06 +00:00
void ReplicatedMergeTreeQueue : : getEntries ( LogEntriesData & res ) const
2016-01-10 04:44:12 +00:00
{
2017-04-01 07:20:54 +00:00
res . clear ( ) ;
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2016-01-10 04:44:12 +00:00
2017-04-01 07:20:54 +00:00
res . reserve ( queue . size ( ) ) ;
for ( const auto & entry : queue )
res . emplace_back ( * entry ) ;
2016-01-10 04:44:12 +00:00
}
2016-01-17 13:00:42 +00:00
void ReplicatedMergeTreeQueue : : getInsertTimes ( time_t & out_min_unprocessed_insert_time , time_t & out_max_processed_insert_time ) const
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2017-04-01 07:20:54 +00:00
out_min_unprocessed_insert_time = min_unprocessed_insert_time ;
out_max_processed_insert_time = max_processed_insert_time ;
2016-01-17 13:00:42 +00:00
}
2018-06-07 13:28:39 +00:00
std : : vector < MergeTreeMutationStatus > ReplicatedMergeTreeQueue : : getMutationsStatus ( ) const
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2018-06-07 13:28:39 +00:00
std : : vector < MergeTreeMutationStatus > result ;
for ( const auto & pair : mutations_by_znode )
{
2018-06-18 12:17:46 +00:00
const MutationStatus & status = pair . second ;
const ReplicatedMergeTreeMutationEntry & entry = * status . entry ;
2020-02-05 16:30:02 +00:00
Names parts_to_mutate = status . parts_to_do . getParts ( ) ;
2018-06-07 13:28:39 +00:00
for ( const MutationCommand & command : entry . commands )
{
std : : stringstream ss ;
formatAST ( * command . ast , ss , false , true ) ;
result . push_back ( MergeTreeMutationStatus
{
entry . znode_name ,
ss . str ( ) ,
entry . create_time ,
entry . block_numbers ,
2019-12-12 16:24:03 +00:00
parts_to_mutate ,
2018-06-21 13:27:36 +00:00
status . is_done ,
2019-02-05 13:03:52 +00:00
status . latest_failed_part ,
status . latest_fail_time ,
status . latest_fail_reason ,
2018-06-07 13:28:39 +00:00
} ) ;
}
}
return result ;
}
2018-05-10 15:01:10 +00:00
ReplicatedMergeTreeMergePredicate : : ReplicatedMergeTreeMergePredicate (
2018-05-12 23:05:04 +00:00
ReplicatedMergeTreeQueue & queue_ , zkutil : : ZooKeeperPtr & zookeeper )
2018-05-10 15:01:10 +00:00
: queue ( queue_ )
2018-05-12 23:05:04 +00:00
, prev_virtual_parts ( queue . format_version )
2018-05-10 15:01:10 +00:00
{
2018-05-12 23:05:04 +00:00
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2018-05-12 23:05:04 +00:00
prev_virtual_parts = queue . virtual_parts ;
}
2018-05-10 15:01:10 +00:00
2018-07-05 10:27:46 +00:00
/// Load current quorum status.
auto quorum_status_future = zookeeper - > asyncTryGet ( queue . zookeeper_path + " /quorum/status " ) ;
2018-05-10 15:01:10 +00:00
/// Load current inserts
2018-07-04 16:31:21 +00:00
std : : unordered_set < String > lock_holder_paths ;
2018-05-10 15:01:10 +00:00
for ( const String & entry : zookeeper - > getChildren ( queue . zookeeper_path + " /temp " ) )
{
if ( startsWith ( entry , " abandonable_lock- " ) )
2018-07-04 16:31:21 +00:00
lock_holder_paths . insert ( queue . zookeeper_path + " /temp/ " + entry ) ;
2018-05-10 15:01:10 +00:00
}
2018-07-04 16:31:21 +00:00
if ( ! lock_holder_paths . empty ( ) )
2018-05-10 15:01:10 +00:00
{
Strings partitions = zookeeper - > getChildren ( queue . zookeeper_path + " /block_numbers " ) ;
2018-08-25 01:58:14 +00:00
std : : vector < std : : future < Coordination : : ListResponse > > lock_futures ;
2018-05-10 15:01:10 +00:00
for ( const String & partition : partitions )
lock_futures . push_back ( zookeeper - > asyncGetChildren ( queue . zookeeper_path + " /block_numbers/ " + partition ) ) ;
2019-08-03 11:02:40 +00:00
struct BlockInfo_
2018-05-10 15:01:10 +00:00
{
String partition ;
Int64 number ;
String zk_path ;
2018-08-25 01:58:14 +00:00
std : : future < Coordination : : GetResponse > contents_future ;
2018-05-10 15:01:10 +00:00
} ;
2019-08-03 11:02:40 +00:00
std : : vector < BlockInfo_ > block_infos ;
2018-05-10 15:01:10 +00:00
for ( size_t i = 0 ; i < partitions . size ( ) ; + + i )
{
Strings partition_block_numbers = lock_futures [ i ] . get ( ) . names ;
for ( const String & entry : partition_block_numbers )
{
/// TODO: cache block numbers that are abandoned.
/// We won't need to check them on the next iteration.
2018-05-12 23:05:04 +00:00
if ( startsWith ( entry , " block- " ) )
{
Int64 block_number = parse < Int64 > ( entry . substr ( strlen ( " block- " ) ) ) ;
String zk_path = queue . zookeeper_path + " /block_numbers/ " + partitions [ i ] + " / " + entry ;
2019-08-03 11:02:40 +00:00
block_infos . emplace_back (
BlockInfo_ { partitions [ i ] , block_number , zk_path , zookeeper - > asyncTryGet ( zk_path ) } ) ;
2018-05-12 23:05:04 +00:00
}
2018-05-10 15:01:10 +00:00
}
}
2019-08-03 11:02:40 +00:00
for ( auto & block : block_infos )
2018-05-10 15:01:10 +00:00
{
2018-08-25 01:58:14 +00:00
Coordination : : GetResponse resp = block . contents_future . get ( ) ;
2018-07-04 16:31:21 +00:00
if ( ! resp . error & & lock_holder_paths . count ( resp . data ) )
2018-05-12 23:05:04 +00:00
committing_blocks [ block . partition ] . insert ( block . number ) ;
2018-05-10 15:01:10 +00:00
}
}
2018-05-31 13:05:05 +00:00
queue_ . pullLogsToQueue ( zookeeper ) ;
2018-05-10 15:01:10 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : GetResponse quorum_status_response = quorum_status_future . get ( ) ;
2018-07-05 10:27:46 +00:00
if ( ! quorum_status_response . error )
2018-05-10 15:01:10 +00:00
{
ReplicatedMergeTreeQuorumEntry quorum_status ;
2018-07-05 10:27:46 +00:00
quorum_status . fromString ( quorum_status_response . data ) ;
2018-05-10 15:01:10 +00:00
inprogress_quorum_part = quorum_status . part_name ;
}
else
inprogress_quorum_part . clear ( ) ;
}
bool ReplicatedMergeTreeMergePredicate : : operator ( ) (
const MergeTreeData : : DataPartPtr & left , const MergeTreeData : : DataPartPtr & right ,
String * out_reason ) const
{
/// A sketch of a proof of why this method actually works:
///
/// The trickiest part is to ensure that no new parts will ever appear in the range of blocks between left and right.
2018-07-04 16:31:21 +00:00
/// Inserted parts get their block numbers by acquiring an ephemeral lock (see EphemeralLockInZooKeeper.h).
2018-05-10 15:01:10 +00:00
/// These block numbers are monotonically increasing in a partition.
///
/// Because there is a window between the moment the inserted part gets its block number and
/// the moment it is committed (appears in the replication log), we can't get the name of all parts up to the given
/// block number just by looking at the replication log - some parts with smaller block numbers may be currently committing
/// and will appear in the log later than the parts with bigger block numbers.
///
/// We also can't take a consistent snapshot of parts that are already committed plus parts that are about to commit
/// due to limitations of ZooKeeper transactions.
///
/// So we do the following (see the constructor):
2018-05-12 23:05:04 +00:00
/// * copy virtual_parts from queue to prev_virtual_parts
/// (a set of parts which corresponds to executing the replication log up to a certain point)
/// * load committing_blocks (inserts and mutations that have already acquired a block number but haven't appeared in the log yet)
/// * do pullLogsToQueue() again to load fresh queue.virtual_parts and mutations.
2018-05-10 15:01:10 +00:00
///
2018-05-12 23:05:04 +00:00
/// Now we have an invariant: if some part is in prev_virtual_parts then:
/// * all parts with smaller block numbers are either in committing_blocks or in queue.virtual_parts
/// (those that managed to commit before we loaded committing_blocks).
/// * all mutations with smaller block numbers are either in committing_blocks or in queue.mutations_by_partition
2018-05-10 15:01:10 +00:00
///
/// So to check that no new parts will ever appear in the range of blocks between left and right we first check that
2018-05-12 23:05:04 +00:00
/// left and right are already present in prev_virtual_parts (we can't give a definite answer for parts that were committed later)
/// and then check that there are no blocks between them in committing_blocks and no parts in queue.virtual_parts.
///
/// Similarly, to check that there will be no mutation with a block number between two parts from prev_virtual_parts
/// (only then we can merge them without mutating the left part), we first check committing_blocks
/// and then check that these two parts have the same mutation version according to queue.mutations_by_partition.
2018-05-10 15:01:10 +00:00
if ( left - > info . partition_id ! = right - > info . partition_id )
{
if ( out_reason )
* out_reason = " Parts " + left - > name + " and " + right - > name + " belong to different partitions " ;
return false ;
}
for ( const MergeTreeData : : DataPartPtr & part : { left , right } )
{
if ( part - > name = = inprogress_quorum_part )
{
if ( out_reason )
* out_reason = " Quorum insert for part " + part - > name + " is currently in progress " ;
return false ;
}
2018-05-12 23:05:04 +00:00
if ( prev_virtual_parts . getContainingPart ( part - > info ) . empty ( ) )
2018-05-10 15:01:10 +00:00
{
if ( out_reason )
* out_reason = " Entry for part " + part - > name + " hasn't been read from the replication log yet " ;
return false ;
}
}
Int64 left_max_block = left - > info . max_block ;
Int64 right_min_block = right - > info . min_block ;
if ( left_max_block > right_min_block )
std : : swap ( left_max_block , right_min_block ) ;
if ( left_max_block + 1 < right_min_block )
{
2018-05-12 23:05:04 +00:00
auto committing_blocks_in_partition = committing_blocks . find ( left - > info . partition_id ) ;
if ( committing_blocks_in_partition ! = committing_blocks . end ( ) )
2018-05-10 15:01:10 +00:00
{
2018-05-12 23:05:04 +00:00
const std : : set < Int64 > & block_numbers = committing_blocks_in_partition - > second ;
2018-05-10 15:01:10 +00:00
auto block_it = block_numbers . upper_bound ( left_max_block ) ;
if ( block_it ! = block_numbers . end ( ) & & * block_it < right_min_block )
{
if ( out_reason )
* out_reason = " Block number " + toString ( * block_it ) + " is still being inserted between parts "
+ left - > name + " and " + right - > name ;
return false ;
}
}
2018-05-12 23:05:04 +00:00
}
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2018-05-12 23:05:04 +00:00
for ( const MergeTreeData : : DataPartPtr & part : { left , right } )
{
/// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer
/// and it is guaranteed that it will contain all merges assigned before this object is constructed.
String containing_part = queue . virtual_parts . getContainingPart ( part - > info ) ;
if ( containing_part ! = part - > name )
{
if ( out_reason )
* out_reason = " Part " + part - > name + " has already been assigned a merge into " + containing_part ;
return false ;
}
}
2018-05-10 15:01:10 +00:00
2018-05-12 23:05:04 +00:00
if ( left_max_block + 1 < right_min_block )
{
2019-08-16 15:57:19 +00:00
/// Fake part which will appear as merge result
2018-05-10 15:01:10 +00:00
MergeTreePartInfo gap_part_info (
2018-06-04 11:23:47 +00:00
left - > info . partition_id , left_max_block + 1 , right_min_block - 1 ,
MergeTreePartInfo : : MAX_LEVEL , MergeTreePartInfo : : MAX_BLOCK_NUMBER ) ;
2018-05-10 15:01:10 +00:00
2019-08-16 15:57:19 +00:00
/// We don't select parts if any smaller part covered by our merge must exist after
/// processing replication log up to log_pointer.
2018-05-12 23:05:04 +00:00
Strings covered = queue . virtual_parts . getPartsCoveredBy ( gap_part_info ) ;
2018-05-10 15:01:10 +00:00
if ( ! covered . empty ( ) )
{
if ( out_reason )
* out_reason = " There are " + toString ( covered . size ( ) ) + " parts (from " + covered . front ( )
2019-08-16 15:57:19 +00:00
+ " to " + covered . back ( ) + " ) that are still not present or beeing processed by "
+ " other background process on this replica between " + left - > name + " and " + right - > name ;
2018-05-10 15:01:10 +00:00
return false ;
}
}
2018-05-14 14:51:33 +00:00
Int64 left_mutation_ver = queue . getCurrentMutationVersionImpl (
2018-06-20 11:12:16 +00:00
left - > info . partition_id , left - > info . getDataVersion ( ) , lock ) ;
2019-08-16 15:57:19 +00:00
2018-05-14 14:51:33 +00:00
Int64 right_mutation_ver = queue . getCurrentMutationVersionImpl (
2018-06-20 11:12:16 +00:00
left - > info . partition_id , right - > info . getDataVersion ( ) , lock ) ;
2019-08-16 15:57:19 +00:00
2018-05-12 23:05:04 +00:00
if ( left_mutation_ver ! = right_mutation_ver )
{
if ( out_reason )
* out_reason = " Current mutation versions of parts " + left - > name + " and " + right - > name + " differ: "
+ toString ( left_mutation_ver ) + " and " + toString ( right_mutation_ver ) + " respectively " ;
return false ;
}
2018-05-10 15:01:10 +00:00
return true ;
}
2020-01-31 12:25:31 +00:00
std : : optional < std : : pair < Int64 , int > > ReplicatedMergeTreeMergePredicate : : getDesiredMutationVersion ( const MergeTreeData : : DataPartPtr & part ) const
2018-04-20 16:18:16 +00:00
{
/// Assigning mutations is easier than assigning merges because mutations appear in the same order as
2019-08-19 19:02:20 +00:00
/// the order of their version numbers (see StorageReplicatedMergeTree::mutate).
2018-04-20 16:18:16 +00:00
/// This means that if we have loaded the mutation with version number X then all mutations with
/// the version numbers less than X are also loaded and if there is no merge or mutation assigned to
/// the part (checked by querying queue.virtual_parts), we can confidently assign a mutation to
/// version X for this part.
2020-03-02 12:56:38 +00:00
/// We cannot mutate part if it's beeing inserted with quorum and it's not
/// already reached.
if ( part - > name = = inprogress_quorum_part )
2018-04-20 16:18:16 +00:00
return { } ;
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2018-04-20 16:18:16 +00:00
if ( queue . virtual_parts . getContainingPart ( part - > info ) ! = part - > name )
return { } ;
auto in_partition = queue . mutations_by_partition . find ( part - > info . partition_id ) ;
if ( in_partition = = queue . mutations_by_partition . end ( ) )
return { } ;
2018-05-14 14:51:33 +00:00
Int64 current_version = queue . getCurrentMutationVersionImpl ( part - > info . partition_id , part - > info . getDataVersion ( ) , lock ) ;
2018-04-20 16:18:16 +00:00
Int64 max_version = in_partition - > second . rbegin ( ) - > first ;
2020-01-31 19:30:33 +00:00
2020-01-31 12:25:31 +00:00
int alter_version = - 1 ;
for ( auto [ mutation_version , mutation_status ] : in_partition - > second )
{
max_version = mutation_version ;
2020-01-31 19:30:33 +00:00
if ( mutation_version > current_version & & mutation_status - > entry - > alter_version ! = - 1 )
2020-01-31 12:25:31 +00:00
{
2020-01-31 19:30:33 +00:00
alter_version = mutation_status - > entry - > alter_version ;
2020-01-31 12:25:31 +00:00
break ;
}
}
2020-02-13 12:03:30 +00:00
2018-04-20 16:18:16 +00:00
if ( current_version > = max_version )
return { } ;
2020-01-31 12:25:31 +00:00
return std : : make_pair ( max_version , alter_version ) ;
2018-04-20 16:18:16 +00:00
}
2018-06-21 13:27:36 +00:00
bool ReplicatedMergeTreeMergePredicate : : isMutationFinished ( const ReplicatedMergeTreeMutationEntry & mutation ) const
{
for ( const auto & kv : mutation . block_numbers )
{
const String & partition_id = kv . first ;
Int64 block_num = kv . second ;
auto partition_it = committing_blocks . find ( partition_id ) ;
if ( partition_it ! = committing_blocks . end ( ) )
{
size_t blocks_count = std : : distance (
partition_it - > second . begin ( ) , partition_it - > second . lower_bound ( block_num ) ) ;
if ( blocks_count )
{
LOG_TRACE ( queue . log , " Mutation " < < mutation . znode_name < < " is not done yet because "
2018-11-26 00:56:50 +00:00
< < " in partition ID " < < partition_id < < " there are still "
2018-06-21 13:27:36 +00:00
< < blocks_count < < " uncommitted blocks. " ) ;
return false ;
}
}
2018-06-21 15:54:01 +00:00
}
2018-06-21 13:27:36 +00:00
2018-06-21 15:54:01 +00:00
{
std : : lock_guard lock ( queue . state_mutex ) ;
2018-06-21 13:27:36 +00:00
2019-12-12 16:24:03 +00:00
size_t suddenly_appeared_parts = getPartNamesToMutate ( mutation , queue . virtual_parts ) . size ( ) ;
2018-06-21 13:27:36 +00:00
if ( suddenly_appeared_parts )
{
LOG_TRACE ( queue . log , " Mutation " < < mutation . znode_name < < " is not done yet because "
2018-06-21 15:54:01 +00:00
< < suddenly_appeared_parts < < " parts to mutate suddenly appeared. " ) ;
2018-06-21 13:27:36 +00:00
return false ;
}
}
return true ;
}
2018-05-21 13:49:54 +00:00
ReplicatedMergeTreeQueue : : SubscriberHandler
ReplicatedMergeTreeQueue : : addSubscriber ( ReplicatedMergeTreeQueue : : SubscriberCallBack & & callback )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( state_mutex ) ;
std : : lock_guard lock_subscribers ( subscribers_mutex ) ;
2018-05-21 13:49:54 +00:00
auto it = subscribers . emplace ( subscribers . end ( ) , std : : move ( callback ) ) ;
/// Atomically notify about current size
( * it ) ( queue . size ( ) ) ;
return SubscriberHandler ( it , * this ) ;
}
ReplicatedMergeTreeQueue : : SubscriberHandler : : ~ SubscriberHandler ( )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( queue . subscribers_mutex ) ;
2018-05-21 13:49:54 +00:00
queue . subscribers . erase ( it ) ;
}
void ReplicatedMergeTreeQueue : : notifySubscribers ( size_t new_queue_size )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock_subscribers ( subscribers_mutex ) ;
2018-05-21 13:49:54 +00:00
for ( auto & subscriber_callback : subscribers )
subscriber_callback ( new_queue_size ) ;
}
ReplicatedMergeTreeQueue : : ~ ReplicatedMergeTreeQueue ( )
{
notifySubscribers ( 0 ) ;
}
2018-04-20 16:18:16 +00:00
2016-01-10 04:44:12 +00:00
String padIndex ( Int64 index )
{
2017-04-01 07:20:54 +00:00
String index_str = toString ( index ) ;
return std : : string ( 10 - index_str . size ( ) , ' 0 ' ) + index_str ;
2016-01-10 04:44:12 +00:00
}
2018-06-07 13:28:39 +00:00
2020-02-18 19:57:48 +00:00
void ReplicatedMergeTreeQueue : : removeCurrentPartsFromMutations ( )
{
std : : lock_guard state_lock ( state_mutex ) ;
for ( const auto & part_name : current_parts . getParts ( ) )
removePartFromMutations ( part_name ) ;
}
2016-01-10 04:44:12 +00:00
}