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>
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>
2020-09-18 10:57:33 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h>
2022-01-10 19:01:41 +00:00
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
2018-01-15 19:07:47 +00:00
# include <Common/StringUtils/StringUtils.h>
2021-07-17 08:39:47 +00:00
# include <Common/CurrentMetrics.h>
2022-01-10 19:01:41 +00:00
# include <Parsers/formatAST.h>
2022-01-30 19:49:48 +00:00
# include <base/sort.h>
2016-01-10 04:44:12 +00:00
2023-02-27 11:27:57 +00:00
# include <ranges>
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 ;
2016-01-12 19:45:51 +00:00
extern const int UNEXPECTED_NODE_IN_ZOOKEEPER ;
2020-04-16 15:30:18 +00:00
extern const int ABORTED ;
2016-01-12 19:45:51 +00:00
}
2016-01-10 04:44:12 +00:00
2020-09-18 10:57:33 +00:00
ReplicatedMergeTreeQueue : : ReplicatedMergeTreeQueue ( StorageReplicatedMergeTree & storage_ , ReplicatedMergeTreeMergeStrategyPicker & merge_strategy_picker_ )
2018-05-20 19:56:03 +00:00
: storage ( storage_ )
2020-09-18 10:57:33 +00:00
, merge_strategy_picker ( merge_strategy_picker_ )
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 )
2023-01-31 12:37:56 +00:00
, drop_parts ( format_version )
2020-07-09 04:43:28 +00:00
{
zookeeper_path = storage . zookeeper_path ;
replica_path = storage . replica_path ;
logger_name = storage . getStorageID ( ) . getFullTableName ( ) + " (ReplicatedMergeTreeQueue) " ;
log = & Poco : : Logger : : get ( logger_name ) ;
}
2021-06-01 13:25:23 +00:00
void ReplicatedMergeTreeQueue : : clear ( )
2020-07-09 04:43:28 +00:00
{
2021-06-01 13:25:23 +00:00
auto locks = lockQueue ( ) ;
2022-09-30 17:11:05 +00:00
chassert ( future_parts . empty ( ) ) ;
2021-06-01 13:25:23 +00:00
current_parts . clear ( ) ;
virtual_parts . clear ( ) ;
queue . clear ( ) ;
inserts_by_time . clear ( ) ;
mutations_by_znode . clear ( ) ;
mutations_by_partition . clear ( ) ;
mutation_pointer . clear ( ) ;
2020-07-09 04:43:28 +00:00
}
2018-05-20 19:56:03 +00:00
2021-10-18 20:16:02 +00:00
void ReplicatedMergeTreeQueue : : setBrokenPartsToEnqueueFetchesOnLoading ( Strings & & parts_to_fetch )
{
std : : lock_guard lock ( state_mutex ) ;
/// Can be called only before queue initialization
assert ( broken_parts_to_enqueue_fetches_on_loading . empty ( ) ) ;
assert ( virtual_parts . size ( ) = = 0 ) ;
broken_parts_to_enqueue_fetches_on_loading = std : : move ( parts_to_fetch ) ;
}
2021-09-09 15:19:12 +00:00
void ReplicatedMergeTreeQueue : : initialize ( zkutil : : ZooKeeperPtr zookeeper )
2016-01-10 04:44:12 +00:00
{
2022-09-30 17:11:05 +00:00
clear ( ) ;
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2021-09-09 15:19:12 +00:00
2021-09-16 14:58:13 +00:00
LOG_TRACE ( log , " Initializing parts in queue " ) ;
2021-09-16 15:39:06 +00:00
2021-09-13 08:00:07 +00:00
/// Get current parts state from zookeeper
2021-09-09 15:19:12 +00:00
Strings parts = zookeeper - > getChildren ( replica_path + " /parts " ) ;
for ( const auto & part_name : parts )
2018-06-18 12:17:46 +00:00
{
2021-09-16 14:58:13 +00:00
LOG_TEST ( log , " Adding part {} to current and virtual parts " , part_name ) ;
2021-09-09 15:19:12 +00:00
current_parts . add ( part_name , nullptr ) ;
virtual_parts . add ( part_name , nullptr ) ;
2018-06-18 12:17:46 +00:00
}
2021-09-16 15:39:06 +00:00
/// Drop parts can negatively affect virtual parts. So when we load parts
/// from zookeeper we can break invariant with virtual parts. To fix this we
/// have it here.
for ( const LogEntryPtr & entry : queue )
2018-06-18 12:17:46 +00:00
{
2021-09-16 15:39:06 +00:00
if ( entry - > isDropPart ( format_version ) )
virtual_parts . removePartAndCoveredParts ( * entry - > getDropRange ( format_version ) ) ;
2018-06-18 12:17:46 +00:00
}
2021-09-16 15:39:06 +00:00
2021-09-16 15:40:25 +00:00
LOG_TRACE ( log , " Queue initialized " ) ;
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 ) ;
2020-11-12 17:36:02 +00:00
auto virtual_part_name = virtual_parts . getContainingPart ( data_part - > info ) ;
return ! virtual_part_name . empty ( ) & & virtual_part_name ! = data_part - > name ;
2019-08-20 08:38:02 +00:00
}
2023-01-31 12:37:56 +00:00
bool ReplicatedMergeTreeQueue : : isGoingToBeDropped ( const MergeTreePartInfo & part_info , MergeTreePartInfo * out_drop_range_info ) const
2022-04-12 12:14:26 +00:00
{
std : : lock_guard lock ( state_mutex ) ;
2023-01-31 12:37:56 +00:00
return isGoingToBeDroppedImpl ( part_info , out_drop_range_info ) ;
}
bool ReplicatedMergeTreeQueue : : isGoingToBeDroppedImpl ( const MergeTreePartInfo & part_info , MergeTreePartInfo * out_drop_range_info ) const
{
String covering_virtual = virtual_parts . getContainingPart ( part_info ) ;
2023-02-01 13:47:57 +00:00
if ( ! covering_virtual . empty ( ) )
2023-01-31 12:37:56 +00:00
{
2023-02-01 13:47:57 +00:00
auto covering_virtual_info = MergeTreePartInfo : : fromPartName ( covering_virtual , format_version ) ;
if ( covering_virtual_info . isFakeDropRangePart ( ) )
{
if ( out_drop_range_info )
* out_drop_range_info = covering_virtual_info ;
return true ;
}
2023-01-31 12:37:56 +00:00
}
return drop_parts . hasDropPart ( part_info , out_drop_range_info ) ;
2022-04-12 12:14:26 +00:00
}
2021-06-30 15:24:51 +00:00
bool ReplicatedMergeTreeQueue : : checkPartInQueueAndGetSourceParts ( const String & part_name , Strings & source_parts ) const
{
std : : lock_guard lock ( state_mutex ) ;
2021-07-02 09:29:45 +00:00
bool found = false ;
2021-06-30 19:11:00 +00:00
for ( const auto & entry : queue )
2021-06-30 15:24:51 +00:00
{
2021-07-02 09:29:45 +00:00
if ( entry - > new_part_name = = part_name & & entry - > source_parts . size ( ) > source_parts . size ( ) )
2021-06-30 15:24:51 +00:00
{
2021-07-02 09:29:45 +00:00
source_parts . clear ( ) ;
2021-06-30 19:11:00 +00:00
source_parts . insert ( source_parts . end ( ) , entry - > source_parts . begin ( ) , entry - > source_parts . end ( ) ) ;
2021-07-02 09:29:45 +00:00
found = true ;
2021-06-30 15:24:51 +00:00
}
}
2021-07-02 09:29:45 +00:00
return found ;
2021-06-30 15:24:51 +00:00
}
2020-10-24 04:21:46 +00:00
2017-11-28 14:07:17 +00:00
bool ReplicatedMergeTreeQueue : : load ( zkutil : : ZooKeeperPtr zookeeper )
2016-01-10 04:44:12 +00:00
{
2021-05-08 10:59:55 +00:00
String queue_path = fs : : path ( replica_path ) / " queue " ;
2020-05-23 22:24:01 +00:00
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
2020-10-24 04:21:46 +00:00
/// Reset batch size on initialization to recover from possible errors of too large batch size.
current_multi_batch_size = 1 ;
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
2023-05-22 13:18:29 +00:00
size_t removed_entries = std : : erase_if ( children ,
[ & ] ( const String & path )
2019-05-03 02:00:57 +00:00
{
return already_loaded_paths . count ( path ) ;
} ) ;
2018-08-09 15:06:39 +00:00
2023-05-22 13:18:29 +00:00
LOG_DEBUG ( log , " Having {} queue entries to load, {} entries already loaded. " , children . size ( ) , removed_entries ) ;
2018-08-09 15:06:39 +00:00
2022-01-30 19:49:48 +00:00
: : sort ( children . begin ( ) , children . end ( ) ) ;
2016-01-10 04:44:12 +00:00
2022-10-11 09:27:46 +00:00
auto children_num = children . size ( ) ;
std : : vector < std : : string > paths ;
paths . reserve ( children_num ) ;
2016-01-10 04:44:12 +00:00
2017-11-28 14:07:17 +00:00
for ( const String & child : children )
2022-10-11 09:27:46 +00:00
paths . emplace_back ( fs : : path ( queue_path ) / child ) ;
2017-11-28 14:07:17 +00:00
2022-10-11 09:27:46 +00:00
auto results = zookeeper - > get ( paths ) ;
for ( size_t i = 0 ; i < children_num ; + + i )
2017-11-28 14:07:17 +00:00
{
2022-10-11 09:27:46 +00:00
auto res = results [ i ] ;
2023-01-31 12:37:56 +00:00
LogEntryPtr entry = LogEntry : : parse ( res . data , res . stat , format_version ) ;
2022-10-11 09:27:46 +00:00
entry - > znode_name = children [ i ] ;
2017-11-28 14:07:17 +00:00
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
2021-09-08 11:37:16 +00:00
{ /// Mutation pointer is a part of "state" and must be updated with state mutex
std : : lock_guard lock ( state_mutex ) ;
zookeeper - > tryGet ( fs : : path ( replica_path ) / " mutation_pointer " , mutation_pointer ) ;
}
2016-01-10 04:44:12 +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
2020-09-18 10:57:33 +00:00
merge_strategy_picker . refreshState ( ) ;
2016-01-12 19:45:51 +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
}
2021-10-18 20:16:02 +00:00
void ReplicatedMergeTreeQueue : : createLogEntriesToFetchBrokenParts ( )
{
2021-10-26 14:06:48 +00:00
Strings broken_parts ;
{
std : : lock_guard lock ( state_mutex ) ;
broken_parts = broken_parts_to_enqueue_fetches_on_loading ;
}
2021-10-18 20:16:02 +00:00
2021-10-26 14:06:48 +00:00
/// It will lock state_mutex
for ( const auto & broken_part_name : broken_parts )
2023-01-06 14:36:53 +00:00
storage . removePartAndEnqueueFetch ( broken_part_name , /* storage_init = */ true ) ;
2021-10-18 20:16:02 +00:00
2021-10-26 14:06:48 +00:00
std : : lock_guard lock ( state_mutex ) ;
/// broken_parts_to_enqueue_fetches_on_loading can be assigned only once on table startup,
/// so actually no race conditions are possible
assert ( broken_parts = = broken_parts_to_enqueue_fetches_on_loading ) ;
2021-10-18 20:16:02 +00:00
broken_parts_to_enqueue_fetches_on_loading . clear ( ) ;
}
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
{
2021-09-14 09:03:48 +00:00
auto entry_virtual_parts = entry - > getVirtualPartNames ( format_version ) ;
2023-01-12 19:32:39 +00:00
LOG_TRACE ( log , " Insert entry {} to queue with type {} " , entry - > znode_name , entry - > getDescriptionForLogs ( format_version ) ) ;
2021-09-14 09:03:48 +00:00
for ( const String & virtual_part_name : entry_virtual_parts )
2018-06-18 12:17:46 +00:00
{
2021-06-30 12:29:09 +00:00
virtual_parts . add ( virtual_part_name , nullptr ) ;
2023-01-31 12:37:56 +00:00
2021-06-23 19:24:43 +00:00
/// Don't add drop range parts to mutations
/// they don't produce any useful parts
2023-01-31 12:37:56 +00:00
/// Note: DROP_PART does not have virtual parts
2021-11-03 18:13:42 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( virtual_part_name , format_version ) ;
2023-01-31 12:37:56 +00:00
if ( part_info . isFakeDropRangePart ( ) )
2021-11-03 18:13:42 +00:00
continue ;
addPartToMutations ( virtual_part_name , part_info ) ;
2018-06-18 12:17:46 +00:00
}
2023-01-31 12:37:56 +00:00
if ( entry - > type = = LogEntry : : DROP_PART )
2021-06-23 19:24:43 +00:00
{
2021-07-06 16:51:23 +00:00
/// DROP PART remove parts, so we remove it from virtual parts to
2021-10-25 16:00:55 +00:00
/// preserve invariant virtual_parts = current_parts + queue.
/// Also remove it from parts_to_do to avoid intersecting parts in parts_to_do
/// if fast replica will execute DROP PART and assign a merge that contains dropped blocks.
2023-01-31 12:37:56 +00:00
drop_parts . addDropPart ( entry ) ;
String drop_part_name = * entry - > getDropRange ( format_version ) ;
virtual_parts . removePartAndCoveredParts ( drop_part_name ) ;
removeCoveredPartsFromMutations ( drop_part_name , /*remove_part = */ true , /*remove_covered_parts = */ true ) ;
}
2021-07-05 19:58:36 +00:00
2023-01-31 12:37:56 +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 - > getDropRange ( format_version ) )
2018-03-28 12:48:40 +00:00
queue . push_front ( entry ) ;
2023-01-31 12:37:56 +00:00
else
queue . push_back ( entry ) ;
2016-01-17 13:00:42 +00:00
2021-02-15 15:06:48 +00:00
if ( entry - > type = = LogEntry : : GET_PART | | entry - > type = = LogEntry : : ATTACH_PART )
2016-01-17 13:00:42 +00:00
{
inserts_by_time . insert ( entry ) ;
if ( entry - > create_time & & ( ! min_unprocessed_insert_time | | entry - > create_time < min_unprocessed_insert_time ) )
2018-03-03 16:46:32 +00:00
{
2023-02-12 06:39:16 +00:00
min_unprocessed_insert_time . store ( entry - > create_time , std : : memory_order_relaxed ) ;
2018-03-03 16:46:32 +00:00
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
}
2016-01-17 13:00:42 +00:00
}
2020-01-28 17:15:22 +00:00
if ( entry - > type = = LogEntry : : ALTER_METADATA )
2020-02-14 20:13:57 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Adding alter metadata version {} to the queue " , entry - > alter_version ) ;
2020-12-25 15:47:17 +00:00
alter_sequence . addMetadataAlter ( entry - > alter_version , 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
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
insertUnlocked ( entry , min_unprocessed_insert_time_changed , lock ) ;
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 (
2016-01-17 13:00:42 +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
{
2021-09-14 09:03:48 +00:00
auto entry_virtual_parts = entry - > getVirtualPartNames ( format_version ) ;
2022-04-26 14:58:40 +00:00
LOG_TEST ( log , " Removing {} entry {} from queue with type {} " ,
2021-09-14 09:03:48 +00:00
is_successful ? " successful " : " unsuccessful " ,
2022-04-26 14:58:40 +00:00
entry - > znode_name , entry - > getDescriptionForLogs ( format_version ) ) ;
2018-06-18 12:17:46 +00:00
/// Update insert times.
2021-02-15 15:06:48 +00:00
if ( entry - > type = = LogEntry : : GET_PART | | entry - > type = = LogEntry : : ATTACH_PART )
2018-06-18 12:17:46 +00:00
{
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 ( ) )
{
2023-02-12 06:39:16 +00:00
min_unprocessed_insert_time . store ( 0 , std : : memory_order_relaxed ) ;
2018-06-18 12:17:46 +00:00
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
}
else if ( ( * inserts_by_time . begin ( ) ) - > create_time > min_unprocessed_insert_time )
{
2023-02-12 06:39:16 +00:00
min_unprocessed_insert_time . store ( ( * inserts_by_time . begin ( ) ) - > create_time , std : : memory_order_relaxed ) ;
2018-06-18 12:17:46 +00:00
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 )
{
2023-02-12 06:39:16 +00:00
max_processed_insert_time . store ( entry - > create_time , std : : memory_order_relaxed ) ;
2018-06-18 12:17:46 +00:00
max_processed_insert_time_changed = max_processed_insert_time ;
}
2016-01-17 13:00:42 +00:00
}
2018-06-18 12:17:46 +00:00
if ( is_successful )
2016-01-17 13:00:42 +00:00
{
2020-02-19 12:50:11 +00:00
if ( ! entry - > actual_new_part_name . empty ( ) )
{
2021-09-14 09:03:48 +00:00
LOG_TEST ( log , " Entry {} has actual new part name {}, removing it from mutations " , entry - > znode_name , entry - > actual_new_part_name ) ;
2020-02-19 12:50:11 +00:00
/// We don't add bigger fetched part to current_parts because we
/// have an invariant `virtual_parts` = `current_parts` + `queue`.
2020-06-15 13:37:40 +00:00
///
/// But we remove covered parts from mutations, because we actually
/// have replacing part.
2020-06-16 10:34:59 +00:00
///
/// NOTE actual_new_part_name is very confusing and error-prone. This approach must be fixed.
2020-06-16 10:50:47 +00:00
removeCoveredPartsFromMutations ( entry - > actual_new_part_name , /*remove_part = */ false , /*remove_covered_parts = */ true ) ;
2020-02-19 12:50:11 +00:00
}
2021-09-14 09:03:48 +00:00
LOG_TEST ( log , " Adding parts [{}] to current parts " , fmt : : join ( entry_virtual_parts , " , " ) ) ;
for ( const String & virtual_part_name : entry_virtual_parts )
2018-06-18 12:17:46 +00:00
{
2021-06-30 12:29:09 +00:00
current_parts . add ( virtual_part_name , nullptr ) ;
2020-06-15 13:37:40 +00:00
/// These parts are already covered by newer part, we don't have to
/// mutate it.
2020-06-16 10:50:47 +00:00
removeCoveredPartsFromMutations ( virtual_part_name , /*remove_part = */ false , /*remove_covered_parts = */ true ) ;
2018-06-18 12:17:46 +00:00
}
2017-04-01 07:20:54 +00:00
2021-06-01 13:25:23 +00:00
if ( auto drop_range_part_name = entry - > getDropRange ( format_version ) )
2018-06-18 12:17:46 +00:00
{
2023-01-31 12:37:56 +00:00
if ( entry - > type = = LogEntry : : DROP_PART )
2021-09-14 09:03:48 +00:00
{
2023-01-31 12:37:56 +00:00
/// DROP PART doesn't have virtual parts so remove from current
/// parts all covered parts.
LOG_TEST ( log , " Removing DROP_PART from current parts {} " , * drop_range_part_name ) ;
2021-07-06 16:51:23 +00:00
current_parts . removePartAndCoveredParts ( * drop_range_part_name ) ;
2023-01-31 12:37:56 +00:00
drop_parts . removeDropPart ( entry ) ;
2021-09-14 09:03:48 +00:00
}
2021-07-06 16:51:23 +00:00
else
2021-09-14 09:03:48 +00:00
{
2023-01-31 12:37:56 +00:00
LOG_TEST ( log , " Removing DROP_RANGE from current and virtual parts {} " , * drop_range_part_name ) ;
2021-07-06 16:51:23 +00:00
current_parts . remove ( * drop_range_part_name ) ;
2023-01-31 12:37:56 +00:00
virtual_parts . remove ( * drop_range_part_name ) ;
2021-09-14 09:03:48 +00:00
}
2021-07-06 16:51:23 +00:00
2022-04-13 08:05:40 +00:00
/// NOTE: we don't need to remove part/covered parts from mutations (removeCoveredPartsFromMutations()) here because:
/// - for DROP PART we have this during inserting to queue (see insertUnlocked())
/// - for DROP PARTITION we have this in the loop above (when we adding parts to current_parts)
2018-06-18 12:17:46 +00:00
}
2020-01-28 17:15:22 +00:00
if ( entry - > type = = LogEntry : : ALTER_METADATA )
2020-02-14 20:13:57 +00:00
{
2020-05-23 22:24:01 +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
2016-01-17 13:00:42 +00:00
{
2023-01-31 12:37:56 +00:00
if ( entry - > type = = LogEntry : : DROP_PART )
2021-07-05 19:58:36 +00:00
{
2023-01-31 12:37:56 +00:00
drop_parts . removeDropPart ( entry ) ;
2021-07-05 19:58:36 +00:00
}
2021-09-14 09:03:48 +00:00
LOG_TEST ( log , " Removing unsuccessful entry {} virtual parts [{}] " , entry - > znode_name , fmt : : join ( entry_virtual_parts , " , " ) ) ;
for ( const String & virtual_part_name : entry_virtual_parts )
2018-06-18 12:17:46 +00:00
{
2021-07-28 10:06:22 +00:00
/// This part will never appear, so remove it from virtual parts
virtual_parts . remove ( virtual_part_name ) ;
2020-06-15 13:37:40 +00:00
/// Because execution of the entry is unsuccessful,
/// `virtual_part_name` will never appear so we won't need to mutate
/// it.
2020-06-16 10:50:47 +00:00
removeCoveredPartsFromMutations ( virtual_part_name , /*remove_part = */ true , /*remove_covered_parts = */ false ) ;
2018-06-21 15:54:01 +00:00
}
}
}
2018-06-18 12:17:46 +00:00
2020-06-16 10:50:47 +00:00
void ReplicatedMergeTreeQueue : : removeCoveredPartsFromMutations ( const String & part_name , bool remove_part , bool remove_covered_parts )
2018-06-21 15:54:01 +00:00
{
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
2021-09-14 09:03:48 +00:00
LOG_TEST ( log , " Removing part {} from mutations (remove_part: {}, remove_covered_parts: {}) " , part_name , remove_part , remove_covered_parts ) ;
2018-06-21 15:54:01 +00:00
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 ;
2021-05-26 12:10:38 +00:00
for ( auto & it : in_partition - > second )
2018-06-21 15:54:01 +00:00
{
2021-05-26 12:10:38 +00:00
MutationStatus & status = * it . second ;
2020-02-11 14:54:46 +00:00
2020-06-16 10:50:47 +00:00
if ( remove_part & & remove_covered_parts )
2020-06-16 10:34:59 +00:00
status . parts_to_do . removePartAndCoveredParts ( part_name ) ;
2020-06-16 10:50:47 +00:00
else if ( remove_covered_parts )
2020-06-16 10:34:59 +00:00
status . parts_to_do . removePartsCoveredBy ( part_name ) ;
2020-06-16 10:50:47 +00:00
else if ( remove_part )
status . parts_to_do . remove ( part_name ) ;
else
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Called remove part from mutations, but nothing removed " ) ;
2020-06-16 10:34:59 +00:00
2020-03-09 02:55:28 +00:00
if ( status . parts_to_do . size ( ) = = 0 )
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 ( ) ;
}
2016-01-17 13:00:42 +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
}
2021-11-03 18:13:42 +00:00
void ReplicatedMergeTreeQueue : : addPartToMutations ( const String & part_name , const MergeTreePartInfo & part_info )
2020-02-05 16:30:02 +00:00
{
2021-09-14 09:03:48 +00:00
LOG_TEST ( log , " Adding part {} to mutations " , part_name ) ;
2021-11-03 18:13:42 +00:00
assert ( ! part_info . isFakeDropRangePart ( ) ) ;
2021-01-25 13:15:47 +00:00
2020-02-05 16:30:02 +00:00
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 (
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-03-12 19:18:07 +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
2016-01-17 13:00:42 +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
2016-01-17 13:00:42 +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
2016-01-17 13:00:42 +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
2020-06-12 15:09:12 +00:00
if ( code ! = Coordination : : Error : : ZOK )
2021-06-01 13:25:23 +00:00
LOG_ERROR ( log , " Couldn't set value of nodes for insert times "
" ({}/min_unprocessed_insert_time, max_processed_insert_time): {}. "
" This shouldn't happen often. " , replica_path , Coordination : : errorMessage ( code ) ) ;
2016-01-17 13:00:42 +00:00
}
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
{
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 ;
2020-08-11 20:58:39 +00:00
bool need_remove_from_zk = true ;
2018-05-21 13:49:54 +00:00
size_t queue_size = 0 ;
2020-05-18 08:08:55 +00:00
/// First remove from memory then from ZooKeeper
2016-01-10 04:44:12 +00:00
{
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( state_mutex ) ;
2020-08-11 20:58:39 +00:00
if ( entry - > removed_by_other_entry )
{
need_remove_from_zk = false ;
queue_size = queue . size ( ) ;
}
else
2016-01-10 04:44:12 +00:00
{
2020-08-11 20:58:39 +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 ( ) ; )
2016-01-17 13:00:42 +00:00
{
2020-08-11 20:58:39 +00:00
- - it ;
2018-06-18 12:17:46 +00:00
2020-08-11 20:58:39 +00:00
if ( * it = = entry )
{
found = true ;
updateStateOnQueueEntryRemoval (
entry , /* is_successful = */ true ,
min_unprocessed_insert_time_changed , max_processed_insert_time_changed , lock ) ;
queue . erase ( it ) ;
queue_size = queue . size ( ) ;
break ;
}
2016-01-17 13:00:42 +00:00
}
2016-01-10 04:44:12 +00:00
}
}
2017-04-01 07:20:54 +00:00
2020-08-11 20:58:39 +00:00
if ( ! found & & need_remove_from_zk )
2021-06-01 13:25:23 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Can't find {} in the memory queue. It is a bug. Entry: {} " ,
entry - > znode_name , entry - > toString ( ) ) ;
2023-03-27 23:39:36 +00:00
notifySubscribers ( queue_size , & ( entry - > znode_name ) ) ;
2018-05-21 13:49:54 +00:00
2020-08-11 20:58:39 +00:00
if ( ! need_remove_from_zk )
return ;
2021-05-08 10:59:55 +00:00
auto code = zookeeper - > tryRemove ( fs : : path ( replica_path ) / " queue " / entry - > znode_name ) ;
2020-06-12 15:09:12 +00:00
if ( code ! = Coordination : : Error : : ZOK )
LOG_ERROR ( log , " Couldn't remove {}/queue/{}: {}. This shouldn't happen often. " , replica_path , entry - > znode_name , Coordination : : errorMessage ( code ) ) ;
2020-05-18 08:08:55 +00:00
2016-01-17 13:00:42 +00:00
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , max_processed_insert_time_changed ) ;
2016-01-10 04:44:12 +00:00
}
2021-06-01 13:25:23 +00:00
bool ReplicatedMergeTreeQueue : : removeFailedQuorumPart ( const MergeTreePartInfo & part_info )
2018-05-10 15:01:10 +00:00
{
2021-06-01 13:25:23 +00:00
assert ( part_info . level = = 0 ) ;
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 ) ;
}
2021-08-18 09:49:22 +00:00
int32_t ReplicatedMergeTreeQueue : : pullLogsToQueue ( zkutil : : ZooKeeperPtr zookeeper , Coordination : : WatchCallback watch_callback , PullLogsReason reason )
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 ) ;
2022-02-03 10:10:05 +00:00
if ( reason ! = LOAD )
2021-08-18 09:49:22 +00:00
{
2022-02-03 10:10:05 +00:00
/// It's totally ok to load queue on readonly replica (that's what RestartingThread does on initialization).
/// It's ok if replica became readonly due to connection loss after we got current zookeeper (in this case zookeeper must be expired).
/// And it's ok if replica became readonly after shutdown.
/// In other cases it's likely that someone called pullLogsToQueue(...) when queue is not initialized yet by RestartingThread.
bool not_completely_initialized = storage . is_readonly & & ! zookeeper - > expired ( ) & & ! storage . shutdown_called ;
if ( not_completely_initialized )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Tried to pull logs to queue (reason: {}) on readonly replica { } , it ' s a bug " ,
reason , storage . getStorageID ( ) . getNameForLogs ( ) ) ;
2021-08-18 09:49:22 +00:00
}
2020-04-16 15:30:18 +00:00
if ( pull_log_blocker . isCancelled ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ABORTED , " Log pulling is cancelled " ) ;
2016-01-13 02:36:11 +00:00
2021-05-08 10:59:55 +00:00
String index_str = zookeeper - > get ( fs : : path ( replica_path ) / " log_pointer " ) ;
2016-01-10 04:44:12 +00:00
UInt64 index ;
2020-06-12 18:24:32 +00:00
/// The version of "/log" is modified when new entries to merge/mutate/drop appear.
Coordination : : Stat stat ;
2021-05-08 10:59:55 +00:00
zookeeper - > get ( fs : : path ( zookeeper_path ) / " log " , & stat ) ;
2020-06-12 18:24:32 +00:00
2021-05-08 10:59:55 +00:00
Strings log_entries = zookeeper - > getChildrenWatch ( fs : : path ( 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
2016-01-10 04:44:12 +00:00
if ( index_str . empty ( ) )
{
2017-03-13 18:01:46 +00:00
/// If we do not already have a pointer to the log, put a pointer to the first entry in it.
2016-01-12 19:45:51 +00:00
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
2021-05-08 10:59:55 +00:00
zookeeper - > set ( fs : : path ( replica_path ) / " log_pointer " , toString ( index ) ) ;
2016-01-10 04:44:12 +00:00
}
else
{
index = parse < UInt64 > ( index_str ) ;
}
2016-01-12 19:45:51 +00:00
String min_log_entry = " log- " + padIndex ( index ) ;
2016-01-10 04:44:12 +00:00
2017-03-12 19:18:07 +00:00
/// Multiple log entries that must be copied to the queue.
2016-01-10 04:44:12 +00:00
2022-04-16 19:43:14 +00:00
std : : erase_if ( log_entries , [ & min_log_entry ] ( const String & entry ) { return entry < min_log_entry ; } ) ;
2016-01-10 04:44:12 +00:00
2016-01-17 05:22:22 +00:00
if ( ! log_entries . empty ( ) )
{
2022-01-30 19:49:48 +00:00
: : sort ( log_entries . begin ( ) , log_entries . end ( ) ) ;
2016-01-10 04:44:12 +00:00
2020-10-28 11:03:26 +00:00
for ( size_t entry_idx = 0 , num_entries = log_entries . size ( ) ; entry_idx < num_entries ; )
2016-01-21 19:43:00 +00:00
{
2019-01-04 12:10:00 +00:00
auto begin = log_entries . begin ( ) + entry_idx ;
2020-10-24 04:13:17 +00:00
auto end = entry_idx + current_multi_batch_size > = log_entries . size ( )
2018-08-22 14:10:00 +00:00
? log_entries . end ( )
2020-10-24 04:13:17 +00:00
: ( begin + current_multi_batch_size ) ;
2016-01-21 19:43:00 +00:00
auto last = end - 1 ;
2016-01-12 19:45:51 +00:00
2020-10-28 11:03:26 +00:00
/// Increment entry_idx before batch size increase (we copied at most current_multi_batch_size entries)
entry_idx + = current_multi_batch_size ;
2020-10-24 04:13:17 +00:00
/// Increase the batch size exponentially, so it will saturate to MAX_MULTI_OPS.
if ( current_multi_batch_size < MAX_MULTI_OPS )
2020-10-24 04:21:46 +00:00
current_multi_batch_size = std : : min < size_t > ( MAX_MULTI_OPS , current_multi_batch_size * 2 ) ;
2020-10-24 04:13:17 +00:00
2016-01-21 19:43:00 +00:00
String last_entry = * last ;
2016-07-14 05:22:09 +00:00
if ( ! startsWith ( last_entry , " log- " ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNEXPECTED_NODE_IN_ZOOKEEPER , " Error in zookeeper data: unexpected node {} in {}/log " ,
last_entry , zookeeper_path ) ;
2016-01-12 19:45:51 +00:00
2016-01-21 19:43:00 +00:00
UInt64 last_entry_index = parse < UInt64 > ( last_entry . substr ( strlen ( " log- " ) ) ) ;
2016-01-12 19:45:51 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Pulling {} entries to queue: {} - {} " , ( end - begin ) , * begin , * last ) ;
2016-01-12 19:45:51 +00:00
2022-10-11 09:27:46 +00:00
Strings get_paths ;
get_paths . reserve ( end - begin ) ;
2016-01-12 19:45:51 +00:00
2016-01-21 19:43:00 +00:00
for ( auto it = begin ; it ! = end ; + + it )
2022-10-11 09:27:46 +00:00
get_paths . emplace_back ( fs : : path ( zookeeper_path ) / " log " / * it ) ;
2016-01-12 19:45:51 +00:00
2017-03-12 19:18:07 +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 ;
2016-01-21 19:43:00 +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
2022-10-11 09:27:46 +00:00
auto get_results = zookeeper - > get ( get_paths ) ;
auto get_num = get_results . size ( ) ;
for ( size_t i = 0 ; i < get_num ; + + i )
2016-01-17 13:00:42 +00:00
{
2022-10-11 09:27:46 +00:00
auto res = get_results [ i ] ;
2018-03-24 20:11:46 +00:00
2023-01-31 12:37:56 +00:00
copied_entries . emplace_back ( LogEntry : : parse ( res . data , res . stat , format_version ) ) ;
2016-01-21 19:43:00 +00:00
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " queue/queue- " , res . data , zkutil : : CreateMode : : PersistentSequential ) ) ;
2016-01-21 19:43:00 +00:00
const auto & entry = * copied_entries . back ( ) ;
2021-02-15 15:06:48 +00:00
if ( entry . type = = LogEntry : : GET_PART | | entry . type = = LogEntry : : ATTACH_PART )
2016-01-17 13:00:42 +00:00
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2016-01-21 19:43:00 +00:00
if ( entry . create_time & & ( ! min_unprocessed_insert_time | | entry . create_time < min_unprocessed_insert_time ) )
{
2023-02-12 06:39:16 +00:00
min_unprocessed_insert_time . store ( entry . create_time , std : : memory_order_relaxed ) ;
2018-03-03 16:46:32 +00:00
min_unprocessed_insert_time_changed = min_unprocessed_insert_time ;
2016-01-21 19:43:00 +00:00
}
2016-01-17 13:00:42 +00:00
}
}
2016-01-12 19:45:51 +00:00
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " log_pointer " , toString ( last_entry_index + 1 ) , - 1 ) ) ;
2016-01-17 13:00:42 +00:00
2016-01-21 19:43:00 +00:00
if ( min_unprocessed_insert_time_changed )
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( 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-03-13 18:01:46 +00:00
/// Now we have successfully updated the queue in ZooKeeper. Update it in RAM.
2016-01-12 19:45:51 +00:00
2016-01-21 19:43:00 +00:00
try
2016-01-17 05:22:22 +00:00
{
2019-01-04 12:10:00 +00:00
std : : lock_guard state_lock ( state_mutex ) ;
2018-05-10 15:01:10 +00:00
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 )
2016-01-21 19:43:00 +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 ) ;
2016-01-21 19:43:00 +00:00
}
2017-04-01 07:20:54 +00:00
2017-08-04 14:00:26 +00:00
last_queue_update = time ( nullptr ) ;
2016-01-21 19:43:00 +00:00
}
catch ( . . . )
{
2020-01-15 13:00:08 +00:00
tryLogCurrentException ( log ) ;
2017-03-13 18:01:46 +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.
2016-01-21 19:43:00 +00:00
std : : terminate ( ) ;
2016-01-17 05:22:22 +00:00
}
2016-01-12 19:45:51 +00:00
2016-01-21 19:43:00 +00:00
if ( ! copied_entries . empty ( ) )
2020-09-18 10:57:33 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Pulled {} entries to queue. " , copied_entries . size ( ) ) ;
2020-11-03 15:58:17 +00:00
merge_strategy_picker . refreshState ( ) ;
2020-09-18 10:57:33 +00:00
}
2016-01-12 19:45:51 +00:00
}
2017-04-01 07:20:54 +00:00
2021-09-08 00:21:21 +00:00
storage . background_operations_assignee . trigger ( ) ;
2018-05-20 19:56:03 +00:00
}
2020-06-12 18:24:32 +00:00
return stat . version ;
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
{
2022-11-14 23:53:20 +00:00
2022-11-17 12:10:14 +00:00
/// Simplified representation of queue entry. Contain two sets
/// 1) Which parts we will receive after entry execution
/// 2) Which parts we will drop/remove after entry execution
///
/// We use this representation to understand which parts mutation actually have to mutate.
2022-11-14 23:53:20 +00:00
struct QueueEntryRepresentation
{
std : : vector < std : : string > produced_parts ;
std : : vector < std : : string > dropped_parts ;
} ;
using QueueRepresentation = std : : map < std : : string , QueueEntryRepresentation > ;
2022-11-17 12:10:14 +00:00
/// Produce a map from queue znode name to simplified entry representation.
2022-11-14 23:53:20 +00:00
QueueRepresentation getQueueRepresentation ( const std : : list < ReplicatedMergeTreeLogEntryPtr > & entries , MergeTreeDataFormatVersion format_version )
{
using LogEntryType = ReplicatedMergeTreeLogEntryData : : Type ;
QueueRepresentation result ;
for ( const auto & entry : entries )
{
const auto & key = entry - > znode_name ;
switch ( entry - > type )
{
2022-11-17 12:10:14 +00:00
/// explicetely specify all types of entries without default, so if
/// someone decide to add new type it will produce a compiler warning (error in our case)
2022-11-14 23:53:20 +00:00
case LogEntryType : : GET_PART :
case LogEntryType : : ATTACH_PART :
case LogEntryType : : MERGE_PARTS :
case LogEntryType : : MUTATE_PART :
{
result [ key ] . produced_parts . push_back ( entry - > new_part_name ) ;
break ;
}
case LogEntryType : : REPLACE_RANGE :
{
2022-11-17 12:10:14 +00:00
/// Quite tricky entry, it both produce and drop parts (in some cases)
2022-11-14 23:53:20 +00:00
const auto & new_parts = entry - > replace_range_entry - > new_part_names ;
auto & produced_parts = result [ key ] . produced_parts ;
produced_parts . insert (
produced_parts . end ( ) , new_parts . begin ( ) , new_parts . end ( ) ) ;
if ( auto drop_range = entry - > getDropRange ( format_version ) )
{
auto & dropped_parts = result [ key ] . dropped_parts ;
dropped_parts . push_back ( * drop_range ) ;
}
break ;
}
case LogEntryType : : DROP_RANGE :
2023-01-31 12:37:56 +00:00
case LogEntryType : : DROP_PART :
2022-11-14 23:53:20 +00:00
{
result [ key ] . dropped_parts . push_back ( entry - > new_part_name ) ;
break ;
}
2022-11-17 12:10:14 +00:00
/// These entries don't produce/drop any parts
2022-11-14 23:53:20 +00:00
case LogEntryType : : EMPTY :
case LogEntryType : : ALTER_METADATA :
case LogEntryType : : CLEAR_INDEX :
case LogEntryType : : CLEAR_COLUMN :
case LogEntryType : : SYNC_PINNED_PART_UUIDS :
case LogEntryType : : CLONE_PART_FROM_SHARD :
{
break ;
}
}
}
return result ;
}
2022-11-24 13:02:19 +00:00
/// Try to understand which part we need to mutate to finish mutation. In ReplicatedQueue we have two sets of parts:
2022-11-17 12:10:14 +00:00
/// current parts -- set of parts which we actually have (on disk)
/// virtual parts -- set of parts which we will have after we will execute our queue
///
/// From the first glance it can sound that these two sets should be enough to understand which parts we have to mutate
/// to finish mutation but it's not true:
2023-05-08 22:21:07 +00:00
/// 1) Obviously we cannot rely on current_parts because we can have stale state (some parts are absent, some merges not finished).
/// We also have to account parts which we will get after queue execution.
/// 2) But we cannot rely on virtual_parts for this, because they contain parts which we will get after we have executed our queue.
/// So if we need to execute mutation 0000000001 for part all_0_0_0 and we have already pulled entry
/// to mutate this part into own queue our virtual parts will contain part all_0_0_0_1, not part all_0_0_0.
2022-11-17 12:10:14 +00:00
///
2023-05-08 22:21:07 +00:00
/// To avoid such issues we simply traverse all entries in queue in order and applying diff (add parts/remove parts) to current parts
/// if they could be affected by mutation. Such approach is expensive but we do it only once since we get the mutation.
/// After that we just update parts_to_do for each mutation when pulling entries into our queue (addPartToMutations, removePartFromMutations).
2022-11-14 23:53:20 +00:00
ActiveDataPartSet getPartNamesToMutate (
const ReplicatedMergeTreeMutationEntry & mutation , const ActiveDataPartSet & current_parts ,
const QueueRepresentation & queue_representation , MergeTreeDataFormatVersion format_version )
2018-06-21 15:54:01 +00:00
{
2022-11-14 23:53:20 +00:00
ActiveDataPartSet result ( format_version ) ;
2022-11-17 12:10:14 +00:00
/// Traverse mutation by partition
for ( const auto & [ partition_id , block_num ] : mutation . block_numbers )
2018-06-21 15:54:01 +00:00
{
/// 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 ) ;
2022-11-14 23:53:20 +00:00
2022-11-17 12:10:14 +00:00
/// First of all add all affected current_parts
2022-11-14 23:53:20 +00:00
for ( const String & covered_part_name : current_parts . getPartsCoveredBy ( covering_part_info ) )
2018-06-21 15:54:01 +00:00
{
2022-11-14 23:53:20 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( covered_part_name , current_parts . getFormatVersion ( ) ) ;
2022-11-14 21:59:09 +00:00
if ( part_info . getDataVersion ( ) < block_num )
2022-11-14 23:53:20 +00:00
result . add ( covered_part_name ) ;
}
2022-11-17 12:10:14 +00:00
/// Traverse queue and update affected current_parts
2022-11-14 23:53:20 +00:00
for ( const auto & [ _ , entry_representation ] : queue_representation )
{
2022-11-17 12:10:14 +00:00
/// First we have to drop something if entry drop parts
2022-11-14 23:53:20 +00:00
for ( const auto & part_to_drop : entry_representation . dropped_parts )
{
auto part_to_drop_info = MergeTreePartInfo : : fromPartName ( part_to_drop , format_version ) ;
2022-11-24 13:02:26 +00:00
if ( part_to_drop_info . partition_id = = partition_id )
2022-11-14 23:53:20 +00:00
result . removePartAndCoveredParts ( part_to_drop ) ;
}
2022-11-24 13:04:03 +00:00
/// After we have to add parts if entry adds them
2022-11-14 23:53:20 +00:00
for ( const auto & part_to_add : entry_representation . produced_parts )
2021-11-03 18:13:42 +00:00
{
2022-11-14 23:53:20 +00:00
auto part_to_add_info = MergeTreePartInfo : : fromPartName ( part_to_add , format_version ) ;
if ( part_to_add_info . partition_id = = partition_id & & part_to_add_info . getDataVersion ( ) < block_num )
result . add ( part_to_add ) ;
2021-11-03 18:13:42 +00:00
}
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 ) ;
2021-05-08 10:59:55 +00:00
Strings entries_in_zk = zookeeper - > getChildrenWatch ( fs : : path ( 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 ;
2022-04-18 10:18:43 +00:00
if ( ! entries_in_zk_set . contains ( entry . znode_name ) )
2018-04-19 14:20:18 +00:00
{
2019-02-05 17:22:23 +00:00
if ( ! it - > second . is_done )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removing killed mutation {} from local state. " , entry . znode_name ) ;
2019-02-05 17:22:23 +00:00
some_active_mutations_were_killed = true ;
2020-11-27 15:46:52 +00:00
if ( entry . isAlterMutation ( ) )
{
LOG_DEBUG ( log , " Removed alter {} because mutation {} were killed. " , entry . alter_version , entry . znode_name ) ;
alter_sequence . finishDataAlter ( entry . alter_version , state_lock ) ;
}
2019-02-05 17:22:23 +00:00
}
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removing obsolete mutation {} from local state. " , entry . znode_name ) ;
2019-02-05 17:22:23 +00:00
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 )
{
2022-04-18 10:18:43 +00:00
if ( ! mutations_by_znode . contains ( znode ) )
2018-04-19 14:20:18 +00:00
entries_to_load . push_back ( znode ) ;
}
}
2020-10-14 07:22:48 +00:00
if ( some_active_mutations_were_killed )
2021-09-08 00:21:21 +00:00
storage . background_operations_assignee . trigger ( ) ;
2019-02-05 17:22:23 +00:00
2018-04-19 14:20:18 +00:00
if ( ! entries_to_load . empty ( ) )
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Loading {} mutation entries: {} - {} " , toString ( entries_to_load . size ( ) ) , entries_to_load . front ( ) , entries_to_load . back ( ) ) ;
2018-04-19 14:20:18 +00:00
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 )
2021-08-18 09:49:22 +00:00
futures . emplace_back ( zookeeper - > asyncTryGet ( fs : : path ( zookeeper_path ) / " mutations " / entry ) ) ;
2018-04-19 14:20:18 +00:00
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 )
{
2021-08-18 09:49:22 +00:00
auto maybe_response = futures [ i ] . get ( ) ;
if ( maybe_response . error ! = Coordination : : Error : : ZOK )
{
assert ( maybe_response . error = = Coordination : : Error : : ZNONODE ) ;
/// It's ok if it happened on server startup or table creation and replica loads all mutation entries.
/// It's also ok if mutation was killed.
LOG_WARNING ( log , " Cannot get mutation node {} ({}), probably it was concurrently removed " , entries_to_load [ i ] , maybe_response . error ) ;
continue ;
}
2018-06-07 11:00:43 +00:00
new_mutations . push_back ( std : : make_shared < ReplicatedMergeTreeMutationEntry > (
2021-08-18 09:49:22 +00:00
ReplicatedMergeTreeMutationEntry : : parse ( maybe_response . 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 ) ;
}
2023-05-15 16:28:12 +00:00
LOG_TRACE ( log , " Adding mutation {} for {} partitions (data versions: {}) " ,
entry - > znode_name , entry - > block_numbers . size ( ) , entry - > getBlockNumbersForLogs ( ) ) ;
2018-04-19 14:20:18 +00:00
2022-11-17 12:10:14 +00:00
/// Initialize `mutation.parts_to_do`. We cannot use only current_parts + virtual_parts here so we
/// traverse all the queue and build correct state of parts_to_do.
2022-11-14 23:53:20 +00:00
auto queue_representation = getQueueRepresentation ( queue , format_version ) ;
2023-05-15 16:28:12 +00:00
mutation . parts_to_do = getPartNamesToMutate ( * entry , current_parts , queue_representation , format_version ) ;
2018-06-21 15:54:01 +00:00
2020-03-09 02:55:28 +00:00
if ( mutation . parts_to_do . size ( ) = = 0 )
2018-06-21 13:27:36 +00:00
some_mutations_are_probably_done = true ;
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
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Adding mutation {} with alter version {} to the queue " , entry - > znode_name , entry - > alter_version ) ;
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 ) ;
2021-05-08 10:59:55 +00:00
auto rc = zookeeper - > tryRemove ( fs : : path ( zookeeper_path ) / " mutations " / mutation_id ) ;
2020-06-12 15:09:12 +00:00
if ( rc = = Coordination : : Error : : ZOK )
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed mutation {} from ZooKeeper. " , mutation_id ) ;
2019-02-04 12:53:25 +00:00
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
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed alter {} because mutation {} were killed. " , entry - > alter_version , entry - > znode_name ) ;
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 ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed mutation {} from local state. " , entry - > znode_name ) ;
2019-02-04 12:53:25 +00:00
}
2020-10-14 07:22:48 +00:00
if ( mutation_was_active )
2021-09-08 00:21:21 +00:00
storage . background_operations_assignee . trigger ( ) ;
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
2017-03-13 18:01:46 +00:00
/// Let's find the action to merge this part with others. Let's remember others.
2016-01-10 04:44:12 +00:00
StringSet parts_for_merge ;
2019-01-09 15:44:20 +00:00
Queue : : iterator merge_entry = queue . end ( ) ;
2016-01-10 04:44:12 +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 )
2016-01-10 04:44:12 +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 ( ) )
2016-01-10 04:44:12 +00:00
{
2018-04-20 16:18:16 +00:00
parts_for_merge = StringSet ( ( * it ) - > source_parts . begin ( ) , ( * it ) - > source_parts . end ( ) ) ;
2016-01-10 04:44:12 +00:00
merge_entry = it ;
break ;
}
}
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
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`.
2016-01-10 04:44:12 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; )
{
auto it0 = it ;
+ + it ;
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
if ( it0 = = merge_entry )
break ;
2017-04-01 07:20:54 +00:00
2021-02-15 15:06:48 +00:00
const auto t = ( * it0 ) - > type ;
if ( ( t = = LogEntry : : MERGE_PARTS | |
t = = LogEntry : : GET_PART | |
t = = LogEntry : : ATTACH_PART | |
t = = LogEntry : : MUTATE_PART )
2022-04-18 10:18:43 +00:00
& & parts_for_merge . contains ( ( * it0 ) - > new_part_name ) )
2016-01-10 04:44:12 +00:00
{
queue . splice ( queue . end ( ) , queue , it0 , it ) ;
}
}
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
return parts_for_merge ;
}
2022-03-13 12:23:51 +00:00
bool ReplicatedMergeTreeQueue : : checkReplaceRangeCanBeRemoved ( const MergeTreePartInfo & part_info , LogEntryPtr entry_ptr , const ReplicatedMergeTreeLogEntryData & current ) const
2018-08-06 12:59:51 +00:00
{
if ( entry_ptr - > type ! = LogEntry : : REPLACE_RANGE )
return false ;
2021-05-17 19:23:38 +00:00
assert ( entry_ptr - > replace_range_entry ) ;
2018-08-06 12:59:51 +00:00
2023-01-31 12:37:56 +00:00
if ( current . type ! = LogEntry : : REPLACE_RANGE & & current . type ! = LogEntry : : DROP_RANGE & & current . type ! = LogEntry : : DROP_PART )
2018-08-06 12:59:51 +00:00
return false ;
2021-05-17 19:23:38 +00:00
if ( entry_ptr - > replace_range_entry = = current . replace_range_entry ) /// same partition, don't want to drop ourselves
2018-08-06 12:59:51 +00:00
return false ;
2021-06-24 08:16:54 +00:00
if ( ! part_info . contains ( MergeTreePartInfo : : fromPartName ( entry_ptr - > replace_range_entry - > drop_range_part_name , format_version ) ) )
return false ;
2021-05-17 19:23:38 +00:00
size_t number_of_covered_parts = 0 ;
2018-08-06 12:59:51 +00:00
for ( const String & new_part_name : entry_ptr - > replace_range_entry - > new_part_names )
2021-05-17 19:23:38 +00:00
{
if ( part_info . contains ( MergeTreePartInfo : : fromPartName ( new_part_name , format_version ) ) )
+ + number_of_covered_parts ;
}
2016-01-10 04:44:12 +00:00
2021-05-17 19:23:38 +00:00
/// It must either cover all new parts from REPLACE_RANGE or no one. Otherwise it's a bug in replication,
/// which may lead to intersecting entries.
assert ( number_of_covered_parts = = 0 | | number_of_covered_parts = = entry_ptr - > replace_range_entry - > new_part_names . size ( ) ) ;
return number_of_covered_parts = = entry_ptr - > replace_range_entry - > new_part_names . size ( ) ;
2018-08-06 12:59:51 +00:00
}
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 ,
2023-02-13 13:46:46 +00:00
const std : : optional < ReplicatedMergeTreeLogEntryData > & covering_entry )
2016-01-10 04:44:12 +00:00
{
2020-08-11 20:58:39 +00:00
/// TODO is it possible to simplify it?
2016-01-10 04:44:12 +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
2017-03-13 18:01:46 +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 ) ;
2020-08-11 20:58:39 +00:00
2021-10-25 14:01:23 +00:00
[[maybe_unused]] bool called_from_alter_query_directly = covering_entry & & covering_entry - > replace_range_entry
& & covering_entry - > replace_range_entry - > columns_version < 0 ;
[[maybe_unused]] bool called_for_broken_part = ! covering_entry ;
2023-02-13 13:46:46 +00:00
assert ( currently_executing_drop_replace_ranges . contains ( part_info ) | | called_from_alter_query_directly | | called_for_broken_part ) ;
2022-11-10 19:53:05 +00:00
2016-01-10 04:44:12 +00:00
for ( Queue : : iterator it = queue . begin ( ) ; it ! = queue . end ( ) ; )
{
2023-02-13 13:46:46 +00:00
auto type = ( * it ) - > type ;
bool is_simple_producing_op = type = = LogEntry : : GET_PART | |
type = = LogEntry : : ATTACH_PART | |
type = = LogEntry : : MERGE_PARTS | |
type = = LogEntry : : MUTATE_PART ;
2021-05-10 18:03:37 +00:00
bool simple_op_covered = is_simple_producing_op & & part_info . contains ( MergeTreePartInfo : : fromPartName ( ( * it ) - > new_part_name , format_version ) ) ;
2021-10-25 14:01:23 +00:00
bool replace_range_covered = covering_entry & & checkReplaceRangeCanBeRemoved ( part_info , * it , * covering_entry ) ;
2021-10-18 20:16:02 +00:00
if ( simple_op_covered | | replace_range_covered )
2016-01-10 04:44:12 +00:00
{
2023-04-04 15:14:50 +00:00
const String & znode_name = ( * it ) - > znode_name ;
2016-01-10 04:44:12 +00:00
if ( ( * it ) - > currently_executing )
to_wait . push_back ( * it ) ;
2023-02-13 13:46:46 +00:00
2023-04-04 15:14:50 +00:00
auto code = zookeeper - > tryRemove ( fs : : path ( replica_path ) / " queue " / znode_name ) ;
2020-06-12 15:09:12 +00:00
if ( code ! = Coordination : : Error : : ZOK )
2023-04-04 15:14:50 +00:00
LOG_INFO ( log , " Couldn't remove {}: {} " , ( fs : : path ( replica_path ) / " queue " / znode_name ) . string ( ) , Coordination : : errorMessage ( code ) ) ;
2017-04-01 07:20:54 +00:00
2018-06-18 12:17:46 +00:00
updateStateOnQueueEntryRemoval (
* it , /* is_successful = */ false ,
min_unprocessed_insert_time_changed , max_processed_insert_time_changed , lock ) ;
2020-08-11 20:58:39 +00:00
2023-04-07 18:16:25 +00:00
LogEntryPtr removing_entry = std : : move ( * it ) ; /// Make it live a bit longer
removing_entry - > removed_by_other_entry = true ;
2021-12-20 10:42:31 +00:00
it = queue . erase ( it ) ;
2023-04-04 15:14:50 +00:00
notifySubscribers ( queue . size ( ) , & znode_name ) ;
2016-01-10 04:44:12 +00:00
+ + removed_entries ;
}
else
+ + it ;
}
2017-04-01 07:20:54 +00:00
2016-01-17 13:00:42 +00:00
updateTimesInZooKeeper ( zookeeper , min_unprocessed_insert_time_changed , max_processed_insert_time_changed ) ;
2017-04-01 07:20:54 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed {} entries from queue. Waiting for {} entries that are currently executing. " , removed_entries , to_wait . size ( ) ) ;
2017-04-01 07:20:54 +00:00
2017-03-13 18:01:46 +00:00
/// Let's wait for the operations with the parts contained in the range to be deleted.
2016-01-10 04:44:12 +00:00
for ( LogEntryPtr & entry : to_wait )
entry - > execution_complete . wait ( lock , [ & entry ] { return ! entry - > currently_executing ; } ) ;
}
2022-06-15 20:08:45 +00:00
bool ReplicatedMergeTreeQueue : : isCoveredByFuturePartsImpl ( const LogEntry & entry , const String & new_part_name ,
String & out_reason , std : : unique_lock < std : : mutex > & /* queue_lock */ ,
std : : vector < LogEntryPtr > * covered_entries_to_wait ) const
2017-05-12 13:47:42 +00:00
{
/// Let's check if the same part is now being created by another action.
2021-10-18 20:16:02 +00:00
auto entry_for_same_part_it = future_parts . find ( new_part_name ) ;
if ( entry_for_same_part_it ! = future_parts . end ( ) )
2017-05-12 13:47:42 +00:00
{
2021-10-18 20:16:02 +00:00
const LogEntry & another_entry = * entry_for_same_part_it - > second ;
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} "
" because another log entry {} of type {} for the same part ({}) is being processed. " ;
2023-01-13 19:34:31 +00:00
LOG_INFO ( LogToStr ( out_reason , log ) , fmt_string , entry . znode_name , entry . type , entry . new_part_name ,
another_entry . znode_name , another_entry . type , another_entry . new_part_name ) ;
2022-06-15 20:08:45 +00:00
return true ;
2017-05-12 13:47:42 +00:00
/** 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
2021-01-27 10:07:18 +00:00
/// It can slow down when the size of `future_parts` is large. But it can not be large, since background pool 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
2022-06-15 20:08:45 +00:00
if ( future_part . isDisjoint ( result_part ) )
continue ;
2022-07-14 19:41:01 +00:00
/// Parts are not disjoint. They can be even intersecting and it's not a problem,
/// because we may have two queue entries producing intersecting parts if there's DROP_RANGE between them (so virtual_parts are ok).
2022-06-30 14:23:53 +00:00
2022-11-10 19:53:05 +00:00
/// Give priority to DROP_RANGEs and allow processing them even if covered entries are currently executing.
/// DROP_RANGE will cancel covered operations and will wait for them in removePartProducingOpsInRange.
if ( result_part . isFakeDropRangePart ( ) & & result_part . contains ( future_part ) )
continue ;
/// In other cases we cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`)
2022-06-17 11:47:14 +00:00
/// while any covered or covering parts are processed.
/// But we also cannot simply return true and postpone entry processing, because it may lead to kind of livelock.
/// Since queue is processed in multiple threads, it's likely that there will be at least one thread
/// executing faulty entry for some small part, so bigger covering part will never be processed.
/// That's why it's better to wait for covered entry to be executed (does not matter successfully or not)
/// instead of exiting and postponing covering entry.
2022-06-15 20:08:45 +00:00
if ( covered_entries_to_wait )
2017-05-12 13:47:42 +00:00
{
2022-06-16 17:41:32 +00:00
if ( entry . znode_name < future_part_elem . second - > znode_name )
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} for part {} "
" because it is not disjoint with part {} that is currently executing and another entry {} is newer. " ;
2023-01-13 19:34:31 +00:00
LOG_TRACE ( LogToStr ( out_reason , log ) , fmt_string , entry . znode_name , new_part_name , future_part_elem . first , future_part_elem . second - > znode_name ) ;
2022-06-16 17:41:32 +00:00
return true ;
}
2022-06-15 20:08:45 +00:00
covered_entries_to_wait - > push_back ( future_part_elem . second ) ;
continue ;
2017-05-12 13:47:42 +00:00
}
2022-06-15 20:08:45 +00:00
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} for part {} "
" because it is not disjoint with part {} that is currently executing. " ;
2023-02-14 14:33:27 +00:00
/// This message can be too noisy, do not print it more than once per second
2023-03-13 23:23:36 +00:00
LOG_TEST ( LogToStr ( out_reason , LogFrequencyLimiter ( log , 5 ) ) , fmt_string , entry . znode_name , new_part_name , future_part_elem . first ) ;
2022-06-15 20:08:45 +00:00
return true ;
2017-05-12 13:47:42 +00:00
}
2022-06-15 20:08:45 +00:00
return false ;
2017-05-12 13:47:42 +00:00
}
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
{
2021-08-13 17:33:10 +00:00
/// We have found `part_name` on some replica and are going to fetch it instead of covered `entry->new_part_name`.
2022-06-15 20:08:45 +00:00
std : : unique_lock lock ( state_mutex ) ;
2017-05-12 13:47:42 +00:00
2023-01-31 12:37:56 +00:00
String covering_part = virtual_parts . getContainingPart ( part_name ) ;
if ( covering_part . empty ( ) )
2021-08-13 17:33:10 +00:00
{
/// We should not fetch any parts that absent in our `virtual_parts` set,
/// because we do not know about such parts according to our replication queue (we know about them from some side-channel).
/// Otherwise, it may break invariants in replication queue reordering, for example:
/// 1. Our queue contains GET_PART all_2_2_0, log contains DROP_RANGE all_2_2_0 and MERGE_PARTS all_1_3_1
/// 2. We execute GET_PART all_2_2_0, but fetch all_1_3_1 instead
2023-01-31 12:37:56 +00:00
/// (drop_parts.isAffectedByDropPart(...) is false-negative, because DROP_RANGE all_2_2_0 is not pulled yet).
2021-08-13 17:33:10 +00:00
/// It actually means, that MERGE_PARTS all_1_3_1 is executed too, but it's not even pulled yet.
/// 3. Then we pull log, trying to execute DROP_RANGE all_2_2_0
/// and reveal that it was incorrectly reordered with MERGE_PARTS all_1_3_1 (drop range intersects merged part).
reject_reason = fmt : : format ( " Log entry for part {} or covering part is not pulled from log to queue yet. " , part_name ) ;
return false ;
}
2021-07-06 10:58:53 +00:00
/// FIXME get rid of actual_part_name.
2023-01-31 12:37:56 +00:00
/// If new covering part jumps over non-disjoint DROP_PART we should execute DROP_PART first to avoid intersection
if ( drop_parts . isAffectedByDropPart ( part_name , reject_reason ) )
2021-07-06 08:36:39 +00:00
return false ;
2022-06-15 20:08:45 +00:00
std : : vector < LogEntryPtr > covered_entries_to_wait ;
if ( isCoveredByFuturePartsImpl ( entry , part_name , reject_reason , lock , & covered_entries_to_wait ) )
return false ;
CurrentlyExecuting : : setActualPartName ( entry , part_name , * this , lock , covered_entries_to_wait ) ;
return true ;
2017-05-12 13:47:42 +00:00
}
2016-10-30 11:05:45 +00:00
bool ReplicatedMergeTreeQueue : : shouldExecuteLogEntry (
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 ,
2022-06-15 20:08:45 +00:00
std : : unique_lock < std : : mutex > & state_lock ) const
2016-01-10 04:44:12 +00:00
{
2020-08-08 00:47:03 +00:00
/// If our entry produce part which is already covered by
2020-02-17 16:33:05 +00:00
/// some other entry which is currently executing, then we can postpone this entry.
2021-06-04 11:49:00 +00:00
for ( const String & new_part_name : entry . getVirtualPartNames ( format_version ) )
2016-01-10 04:44:12 +00:00
{
2022-06-17 11:47:14 +00:00
/// Do not wait for any entries here, because we have only one thread that scheduling queue entries.
/// We can wait in worker threads, but not in scheduler.
if ( isCoveredByFuturePartsImpl ( entry , new_part_name , out_postpone_reason , state_lock , /* covered_entries_to_wait */ nullptr ) )
2021-06-04 11:49:00 +00:00
return false ;
2016-01-10 04:44:12 +00:00
}
2017-04-01 07:20:54 +00:00
2023-01-31 12:37:56 +00:00
if ( entry . type ! = LogEntry : : DROP_RANGE & & entry . type ! = LogEntry : : DROP_PART )
{
/// Do not touch any entries that are not disjoint with some DROP_PART to avoid intersecting parts
if ( drop_parts . isAffectedByDropPart ( entry , out_postpone_reason ) )
return false ;
}
/// Optimization: it does not really make sense to generate parts that are going to be dropped anyway
if ( ! entry . new_part_name . empty ( ) )
{
auto new_part_info = MergeTreePartInfo : : fromPartName ( entry . new_part_name , format_version ) ;
MergeTreePartInfo drop_info ;
if ( entry . type ! = LogEntry : : DROP_PART & & ! new_part_info . isFakeDropRangePart ( ) & & isGoingToBeDroppedImpl ( new_part_info , & drop_info ) )
{
out_postpone_reason = fmt : : format (
" Not executing {} because it produces part {} that is going to be dropped by {} " ,
entry . znode_name , entry . new_part_name , drop_info . getPartNameForLogs ( ) ) ;
return false ;
}
}
2021-07-05 19:58:36 +00:00
2020-11-09 09:14:20 +00:00
/// Check that fetches pool is not overloaded
2021-02-15 15:06:48 +00:00
if ( ( entry . type = = LogEntry : : GET_PART | | entry . type = = LogEntry : : ATTACH_PART )
& & ! storage . canExecuteFetch ( entry , out_postpone_reason ) )
2020-10-26 11:02:47 +00:00
{
2020-11-19 17:03:20 +00:00
/// Don't print log message about this, because we can have a lot of fetches,
/// for example during replica recovery.
return false ;
2020-10-26 11:02:47 +00:00
}
2018-04-20 16:18:16 +00:00
if ( entry . type = = LogEntry : : MERGE_PARTS | | entry . type = = LogEntry : : MUTATE_PART )
2016-01-10 04:44:12 +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-03-13 18:01:46 +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 .
2016-01-10 04:44:12 +00:00
*/
2016-10-30 11:05:45 +00:00
size_t sum_parts_size_in_bytes = 0 ;
2018-04-20 16:18:16 +00:00
for ( const auto & name : entry . source_parts )
2016-01-10 04:44:12 +00:00
{
2022-04-18 10:18:43 +00:00
if ( future_parts . contains ( name ) )
2016-01-10 04:44:12 +00:00
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} "
" because part {} is not ready yet (log entry for that part is being processed). " ;
2023-01-13 19:34:31 +00:00
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . typeToString ( ) , entry . new_part_name , name ) ;
2016-01-10 04:44:12 +00:00
return false ;
}
2017-04-01 07:20:54 +00:00
2021-12-30 14:27:22 +00:00
auto part = data . getPartIfExists ( name , { MergeTreeDataPartState : : PreActive , MergeTreeDataPartState : : Active , MergeTreeDataPartState : : Outdated } ) ;
2016-10-30 11:05:45 +00:00
if ( part )
2020-05-27 20:05:55 +00:00
{
2020-06-05 20:47:46 +00:00
if ( auto part_in_memory = asInMemoryPart ( part ) )
2020-05-27 20:05:55 +00:00
sum_parts_size_in_bytes + = part_in_memory - > block . bytes ( ) ;
else
sum_parts_size_in_bytes + = part - > getBytesOnDisk ( ) ;
}
2016-01-10 04:44:12 +00:00
}
2017-04-01 07:20:54 +00:00
2019-08-01 15:36:12 +00:00
if ( merger_mutator . merges_blocker . isCancelled ( ) )
2016-01-10 04:44:12 +00:00
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} because merges and mutations are cancelled now. " ;
2023-01-13 19:34:31 +00:00
LOG_DEBUG ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . typeToString ( ) , entry . new_part_name ) ;
2016-01-10 04:44:12 +00:00
return false ;
}
2017-04-01 07:20:54 +00:00
2022-02-10 19:45:52 +00:00
const auto data_settings = data . getSettings ( ) ;
if ( data_settings - > allow_remote_fs_zero_copy_replication )
2020-09-18 10:57:33 +00:00
{
2022-02-01 11:17:46 +00:00
auto disks = storage . getDisks ( ) ;
bool only_s3_storage = true ;
for ( const auto & disk : disks )
2022-05-11 22:04:54 +00:00
if ( ! disk - > supportZeroCopyReplication ( ) )
2022-02-01 11:17:46 +00:00
only_s3_storage = false ;
2023-01-20 03:04:36 +00:00
String replica_to_execute_merge ;
if ( ! disks . empty ( ) & & only_s3_storage & & storage . checkZeroCopyLockExists ( entry . new_part_name , disks [ 0 ] , replica_to_execute_merge ) )
2022-02-10 19:45:52 +00:00
{
2023-01-20 03:04:36 +00:00
constexpr auto fmt_string = " Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after. " ;
2023-03-17 16:45:02 +00:00
out_postpone_reason = fmt : : format ( fmt_string , entry . new_part_name , replica_to_execute_merge ) ;
LOG_TEST ( log , fmt_string , entry . new_part_name , replica_to_execute_merge ) ;
2022-02-10 19:45:52 +00:00
return false ;
}
2022-02-01 11:17:46 +00:00
}
2022-02-10 19:45:52 +00:00
if ( merge_strategy_picker . shouldMergeOnSingleReplica ( entry ) )
2022-02-01 11:17:46 +00:00
{
2020-11-03 15:58:17 +00:00
auto replica_to_execute_merge = merge_strategy_picker . pickReplicaToExecuteMerge ( entry ) ;
if ( replica_to_execute_merge & & ! merge_strategy_picker . isMergeFinishedByReplica ( replica_to_execute_merge . value ( ) , entry ) )
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing merge for the part {}, waiting for {} to execute merge. " ;
out_postpone_reason = fmt : : format ( fmt_string , entry . new_part_name , replica_to_execute_merge . value ( ) ) ;
2020-11-03 15:58:17 +00:00
return false ;
}
2020-09-18 10:57:33 +00:00
}
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 ) .
2016-10-31 19:45:22 +00:00
*/
2020-09-04 10:08:09 +00:00
bool ignore_max_size = false ;
if ( entry . type = = LogEntry : : MERGE_PARTS )
{
ignore_max_size = max_source_parts_size = = data_settings - > max_bytes_to_merge_at_max_space_in_pool ;
if ( isTTLMergeType ( entry . merge_type ) )
{
if ( merger_mutator . ttl_merges_blocker . isCancelled ( ) )
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} for part {} because merges with TTL are cancelled now. " ;
2023-01-13 19:34:31 +00:00
LOG_DEBUG ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . new_part_name ) ;
2020-09-04 10:08:09 +00:00
return false ;
}
size_t total_merges_with_ttl = data . getTotalMergesWithTTLInMergeList ( ) ;
if ( total_merges_with_ttl > = data_settings - > max_number_of_merges_with_ttl_in_pool )
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} for part {} because {} merges with TTL already executing, maximum {}. " ;
2023-01-13 19:34:31 +00:00
LOG_DEBUG ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . new_part_name , total_merges_with_ttl ,
data_settings - > max_number_of_merges_with_ttl_in_pool ) ;
2020-09-04 10:08:09 +00:00
return false ;
}
}
}
2019-08-22 19:35:46 +00:00
if ( ! ignore_max_size & & sum_parts_size_in_bytes > max_source_parts_size )
2016-10-30 11:05:45 +00:00
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} "
" because source parts size ({}) is greater than the current maximum ({}). " ;
2023-03-13 23:23:36 +00:00
LOG_DEBUG ( LogToStr ( out_postpone_reason , LogFrequencyLimiter ( log , 5 ) ) , fmt_string , entry . znode_name , entry . typeToString ( ) , entry . new_part_name ,
2023-01-13 19:34:31 +00:00
ReadableSize ( sum_parts_size_in_bytes ) , ReadableSize ( max_source_parts_size ) ) ;
2020-05-30 21:35:52 +00:00
2016-10-30 11:05:45 +00:00
return false ;
}
2016-01-10 04:44:12 +00:00
}
2017-04-01 07:20:54 +00:00
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 ) ;
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Cannot execute alter metadata {} with version {} because another alter {} must be executed before " ;
2023-01-13 19:34:31 +00:00
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . alter_version , head_alter ) ;
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 )
2020-10-30 12:41:39 +00:00
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Cannot execute alter data {} with version {} because metadata still not altered " ;
2023-01-13 19:34:31 +00:00
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . alter_version ) ;
2020-10-30 12:41:39 +00:00
}
2020-02-13 14:48:38 +00:00
else
2020-10-30 12:41:39 +00:00
{
2022-12-23 18:40:29 +00:00
constexpr auto fmt_string = " Cannot execute alter data {} with version {} because another alter {} must be executed before " ;
2023-01-13 19:34:31 +00:00
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . alter_version , head_alter ) ;
2020-10-30 12:41:39 +00:00
}
2020-02-13 14:48:38 +00:00
2020-02-05 11:18:11 +00:00
return false ;
}
2020-01-13 16:39:20 +00:00
}
2023-01-31 12:37:56 +00:00
/// DROP_RANGE, DROP_PART and REPLACE_RANGE entries remove other entries, which produce parts in the range.
/// If such part producing operations are currently executing, then DROP/REPLACE RANGE wait them to finish.
/// Deadlock is possible if multiple DROP/REPLACE RANGE entries are executing in parallel and wait each other.
/// But it should not happen if ranges are disjoint.
/// See also removePartProducingOpsInRange(...) and ReplicatedMergeTreeQueue::CurrentlyExecuting.
2022-08-15 18:32:03 +00:00
2023-01-31 12:37:56 +00:00
if ( auto drop_range = entry . getDropRange ( format_version ) )
{
auto drop_range_info = MergeTreePartInfo : : fromPartName ( * drop_range , format_version ) ;
for ( const auto & info : currently_executing_drop_replace_ranges )
2020-10-30 12:41:39 +00:00
{
2023-01-31 12:37:56 +00:00
if ( drop_range_info . isDisjoint ( info ) )
continue ;
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} "
" because another DROP_RANGE or REPLACE_RANGE entry with not disjoint range {} is currently executing. " ;
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name ,
entry . typeToString ( ) ,
entry . new_part_name ,
info . getPartNameForLogs ( ) ) ;
return false ;
2020-10-30 12:41:39 +00:00
}
2023-01-31 12:37:56 +00:00
}
2021-09-10 14:32:45 +00:00
2023-01-31 12:37:56 +00:00
if ( entry . type = = LogEntry : : DROP_PART )
{
/// We should avoid reordering of REPLACE_RANGE and DROP_PART,
/// because if replace_range_entry->new_part_names contains drop_range_entry->new_part_name
/// and we execute DROP PART before REPLACE_RANGE, then DROP PART will be no-op
/// (because part is not created yet, so there is nothing to drop;
/// DROP_RANGE does not cover all parts of REPLACE_RANGE, so removePartProducingOpsInRange(...) will not remove anything too)
/// and part will never be removed. Replicas may diverge due to such reordering.
/// We don't need to do anything for other entry types, because removePartProducingOpsInRange(...) will remove them as expected.
auto drop_part_info = MergeTreePartInfo : : fromPartName ( entry . new_part_name , format_version ) ;
for ( const auto & replace_entry : queue )
2021-09-10 14:32:45 +00:00
{
2023-01-31 12:37:56 +00:00
if ( replace_entry - > type ! = LogEntry : : REPLACE_RANGE )
continue ;
2021-09-10 14:32:45 +00:00
2023-01-31 12:37:56 +00:00
for ( const auto & new_part_name : replace_entry - > replace_range_entry - > new_part_names )
2021-09-10 14:32:45 +00:00
{
2023-01-31 12:37:56 +00:00
auto new_part_info = MergeTreePartInfo : : fromPartName ( new_part_name , format_version ) ;
if ( ! new_part_info . isDisjoint ( drop_part_info ) )
2021-09-10 14:32:45 +00:00
{
2023-01-31 12:37:56 +00:00
constexpr auto fmt_string = " Not executing log entry {} of type {} for part {} "
" because it probably depends on {} (REPLACE_RANGE). " ;
LOG_TRACE ( LogToStr ( out_postpone_reason , log ) , fmt_string , entry . znode_name , entry . typeToString ( ) ,
entry . new_part_name , replace_entry - > znode_name ) ;
return false ;
2021-09-10 14:32:45 +00:00
}
}
}
2020-08-11 20:58:39 +00:00
}
2016-01-10 04:44:12 +00:00
return true ;
}
2023-04-12 22:20:43 +00:00
Int64 ReplicatedMergeTreeQueue : : getCurrentMutationVersion (
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 ;
}
2022-01-26 17:44:35 +00:00
ReplicatedMergeTreeQueue : : CurrentlyExecuting : : CurrentlyExecuting (
2022-06-15 20:08:45 +00:00
const ReplicatedMergeTreeQueue : : LogEntryPtr & entry_ , ReplicatedMergeTreeQueue & queue_ , std : : unique_lock < std : : mutex > & /* state_lock */ )
2019-08-03 11:02:40 +00:00
: entry ( entry_ ) , queue ( queue_ )
2016-02-02 21:30:27 +00:00
{
2022-08-15 18:32:03 +00:00
if ( auto drop_range = entry - > getDropRange ( queue . format_version ) )
2020-08-11 20:58:39 +00:00
{
2022-08-15 18:32:03 +00:00
auto drop_range_info = MergeTreePartInfo : : fromPartName ( * drop_range , queue . format_version ) ;
[[maybe_unused]] bool inserted = queue . currently_executing_drop_replace_ranges . emplace ( drop_range_info ) . second ;
assert ( inserted ) ;
2020-08-11 20:58:39 +00:00
}
2016-02-02 21:30:27 +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
2021-06-04 11:49:00 +00:00
for ( const String & new_part_name : entry - > getVirtualPartNames ( queue . format_version ) )
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 )
2021-06-01 13:25:23 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Tagging already tagged future part {}. This is a bug. "
" It happened on attempt to execute {}: {} " ,
new_part_name , entry - > znode_name , entry - > toString ( ) ) ;
2018-05-21 13:49:54 +00:00
}
2016-02-02 21:30:27 +00:00
}
2017-05-12 13:47:42 +00:00
2022-01-26 17:44:35 +00:00
void ReplicatedMergeTreeQueue : : CurrentlyExecuting : : setActualPartName (
ReplicatedMergeTreeQueue : : LogEntry & entry ,
const String & actual_part_name ,
ReplicatedMergeTreeQueue & queue ,
2022-06-15 20:08:45 +00:00
std : : unique_lock < std : : mutex > & state_lock ,
std : : vector < LogEntryPtr > & covered_entries_to_wait )
2017-05-12 13:47:42 +00:00
{
if ( ! entry . actual_new_part_name . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Entry actual part isn't empty yet. This is a bug. " ) ;
2017-05-12 13:47:42 +00:00
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 )
2021-06-01 13:25:23 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Attaching already existing future part {}. This is a bug. "
" It happened on attempt to execute {}: {} " ,
entry . actual_new_part_name , entry . znode_name , entry . toString ( ) ) ;
2022-06-15 20:08:45 +00:00
for ( LogEntryPtr & covered_entry : covered_entries_to_wait )
2022-06-16 17:41:32 +00:00
{
if ( & entry = = covered_entry . get ( ) )
continue ;
LOG_TRACE ( queue . log , " Waiting for {} producing {} to finish before executing {} producing not disjoint part {} " ,
covered_entry - > znode_name , covered_entry - > new_part_name , entry . znode_name , entry . new_part_name ) ;
covered_entry - > execution_complete . wait ( state_lock , [ & covered_entry ] { return ! covered_entry - > currently_executing ; } ) ;
}
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
2022-08-15 18:32:03 +00:00
if ( auto drop_range = entry - > getDropRange ( queue . format_version ) )
2020-08-11 20:58:39 +00:00
{
2022-08-15 18:32:03 +00:00
auto drop_range_info = MergeTreePartInfo : : fromPartName ( * drop_range , queue . format_version ) ;
[[maybe_unused]] bool removed = queue . currently_executing_drop_replace_ranges . erase ( drop_range_info ) ;
assert ( removed ) ;
2020-08-11 20:58:39 +00:00
}
2016-02-02 21:30:27 +00:00
entry - > currently_executing = false ;
entry - > execution_complete . notify_all ( ) ;
2021-06-04 11:49:00 +00:00
for ( const String & new_part_name : entry - > getVirtualPartNames ( queue . format_version ) )
2018-05-21 13:49:54 +00:00
{
if ( ! queue . future_parts . erase ( new_part_name ) )
2021-06-01 13:25:23 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( queue . log , " Untagging already untagged future part {}. This is a bug. " , new_part_name ) ;
2021-06-01 13:25:23 +00:00
assert ( false ) ;
}
2018-05-21 13:49:54 +00:00
}
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 ) )
2021-06-01 13:25:23 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( queue . log , " Untagging already untagged future part {}. This is a bug. " , entry - > actual_new_part_name ) ;
2021-06-01 13:25:23 +00:00
assert ( false ) ;
}
2017-05-12 13:47:42 +00:00
entry - > actual_new_part_name . clear ( ) ;
}
2016-02-02 21:30:27 +00:00
}
2020-10-23 08:54:00 +00:00
ReplicatedMergeTreeQueue : : SelectedEntryPtr ReplicatedMergeTreeQueue : : selectEntryToProcess ( MergeTreeDataMergerMutator & merger_mutator , MergeTreeData & data )
2016-01-10 04:44:12 +00:00
{
LogEntryPtr entry ;
2017-04-01 07:20:54 +00:00
2022-06-15 20:08:45 +00:00
std : : unique_lock 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 )
2016-01-10 04:44:12 +00:00
{
if ( ( * it ) - > currently_executing )
continue ;
2017-04-01 07:20:54 +00:00
2018-04-20 16:18:16 +00:00
if ( shouldExecuteLogEntry ( * * it , ( * it ) - > postpone_reason , merger_mutator , data , lock ) )
2016-01-10 04:44:12 +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.
2016-01-10 04:44:12 +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 ) ;
2016-01-10 04:44:12 +00:00
}
}
2017-04-01 07:20:54 +00:00
2016-02-02 21:30:27 +00:00
if ( entry )
2022-01-26 17:44:35 +00:00
return std : : make_shared < SelectedEntry > ( entry , std : : unique_ptr < CurrentlyExecuting > { new CurrentlyExecuting ( entry , * this , lock ) } ) ;
2016-02-02 21:30:27 +00:00
else
return { } ;
2016-01-10 04:44:12 +00:00
}
2016-10-24 12:34:08 +00:00
bool ReplicatedMergeTreeQueue : : processEntry (
std : : function < zkutil : : ZooKeeperPtr ( ) > get_zookeeper ,
LogEntryPtr & entry ,
2022-03-13 12:23:51 +00:00
std : : function < bool ( LogEntryPtr & ) > func )
2016-01-10 04:44:12 +00:00
{
std : : exception_ptr saved_exception ;
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
try
{
2020-02-17 18:07:22 +00:00
/// We don't have any backoff for failed entries
2020-08-08 00:47:03 +00:00
/// we just count amount of tries for each of them.
2016-01-10 04:44:12 +00:00
if ( func ( entry ) )
2018-06-06 19:15:10 +00:00
removeProcessedEntry ( get_zookeeper ( ) , entry ) ;
2016-01-10 04:44:12 +00:00
}
catch ( . . . )
{
saved_exception = std : : current_exception ( ) ;
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
if ( saved_exception )
{
2018-06-20 11:12:16 +00:00
std : : lock_guard lock ( state_mutex ) ;
2016-01-10 04:44:12 +00:00
entry - > exception = saved_exception ;
2023-01-13 03:26:41 +00:00
entry - > last_exception_time = time ( nullptr ) ;
2016-01-10 04:44:12 +00:00
return false ;
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
return true ;
}
2020-09-03 13:00:13 +00:00
ReplicatedMergeTreeQueue : : OperationsInQueue 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 ;
2020-09-03 13:00:13 +00:00
size_t count_merges_with_ttl = 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 )
2020-09-03 13:00:13 +00:00
{
2019-08-21 13:10:33 +00:00
+ + count_merges ;
2020-09-03 13:00:13 +00:00
if ( isTTLMergeType ( entry - > merge_type ) )
+ + count_merges_with_ttl ;
}
2019-08-21 13:10:33 +00:00
else if ( entry - > type = = ReplicatedMergeTreeLogEntry : : MUTATE_PART )
+ + count_mutations ;
}
2018-07-31 11:36:08 +00:00
2020-09-03 13:00:13 +00:00
return OperationsInQueue { count_merges , count_mutations , count_merges_with_ttl } ;
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 ;
2023-04-24 18:21:49 +00:00
for ( const auto & [ _ , status ] : mutations_by_znode )
2018-07-31 11:36:08 +00:00
{
2023-04-24 18:21:49 +00:00
if ( ! status . is_done )
2018-07-31 11:36:08 +00:00
break ;
2023-04-24 18:21:49 +00:00
+ + count ;
}
return count ;
}
2018-07-31 11:36:08 +00:00
2023-04-24 18:21:49 +00:00
size_t ReplicatedMergeTreeQueue : : countUnfinishedMutations ( ) const
{
std : : lock_guard lock ( state_mutex ) ;
size_t count = 0 ;
for ( const auto & [ _ , status ] : mutations_by_znode | std : : views : : reverse )
{
if ( status . is_done )
break ;
2018-07-31 11:36:08 +00:00
+ + count ;
}
return count ;
}
2023-05-07 22:22:07 +00:00
ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue : : getMergePredicate ( zkutil : : ZooKeeperPtr & zookeeper ,
std : : optional < PartitionIdsHint > & & partition_ids_hint )
2016-01-10 04:44:12 +00:00
{
2022-11-25 15:41:20 +00:00
return ReplicatedMergeTreeMergePredicate ( * this , zookeeper , std : : move ( partition_ids_hint ) ) ;
2016-01-10 04:44:12 +00:00
}
2018-05-10 15:01:10 +00:00
2023-02-27 11:27:57 +00:00
std : : map < int64_t , MutationCommands > ReplicatedMergeTreeQueue : : getAlterMutationCommandsForPart ( const MergeTreeData : : DataPartPtr & part ) const
2020-03-24 17:05:38 +00:00
{
2023-02-27 11:27:57 +00:00
std : : unique_lock lock ( state_mutex ) ;
2023-05-25 22:54:54 +00:00
2020-03-24 17:05:38 +00:00
auto in_partition = mutations_by_partition . find ( part - > info . partition_id ) ;
if ( in_partition = = mutations_by_partition . end ( ) )
2023-02-27 11:27:57 +00:00
return { } ;
2020-03-24 17:05:38 +00:00
2023-05-25 22:54:54 +00:00
Int64 part_data_version = part - > info . getDataVersion ( ) ;
2023-02-27 11:27:57 +00:00
Int64 part_metadata_version = part - > getMetadataVersion ( ) ;
2023-05-25 22:54:54 +00:00
LOG_DEBUG ( log , " Looking for mutations for part {} (part data version {}, part metadata version {}) " , part - > name , part_data_version , part_metadata_version ) ;
2023-02-27 11:27:57 +00:00
std : : map < int64_t , MutationCommands > result ;
/// Here we return mutation commands for part which has bigger alter version than part metadata version.
/// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions
/// of part's metadata.
for ( const auto & [ mutation_version , mutation_status ] : in_partition - > second | std : : views : : reverse )
{
2023-05-25 22:54:54 +00:00
auto alter_version = mutation_status - > entry - > alter_version ;
2023-02-27 11:27:57 +00:00
if ( alter_version ! = - 1 )
{
2023-03-02 18:17:18 +00:00
if ( alter_version > storage . getInMemoryMetadataPtr ( ) - > getMetadataVersion ( ) )
2023-02-27 11:27:57 +00:00
continue ;
2020-03-24 17:05:38 +00:00
2023-05-25 22:54:54 +00:00
/// We take commands with bigger metadata version
2023-02-27 11:27:57 +00:00
if ( alter_version > part_metadata_version )
result [ mutation_version ] = mutation_status - > entry - > commands ;
2023-05-25 22:54:54 +00:00
}
else if ( mutation_version > part_data_version )
{
result [ mutation_version ] = mutation_status - > entry - > commands ;
2023-02-27 11:27:57 +00:00
}
}
2020-03-24 17:05:38 +00:00
2023-05-25 22:54:54 +00:00
LOG_TRACE ( log , " Got {} commands for part {} (part data version {}, part metadata version {}) " ,
result . size ( ) , part - > name , part_data_version , part_metadata_version ) ;
2023-02-27 11:27:57 +00:00
return result ;
2020-03-24 17:05:38 +00:00
}
2018-04-20 19:11:20 +00:00
MutationCommands ReplicatedMergeTreeQueue : : getMutationCommands (
2023-05-15 16:28:12 +00:00
const MergeTreeData : : DataPartPtr & part , Int64 desired_mutation_version , Strings & mutation_ids ) const
2018-04-20 19:11:20 +00:00
{
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.
2021-08-18 09:49:22 +00:00
/// NOTE: However, it's quite dangerous to skip MUTATE_PART. Replicas may diverge if one of them have executed part mutation,
/// and then mutation was killed before execution of MUTATE_PART on remaining replicas.
2018-05-31 15:21:00 +00:00
if ( part - > info . getDataVersion ( ) > desired_mutation_version )
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Data version of part {} is already greater than desired mutation version {} " , part - > name , desired_mutation_version ) ;
2018-05-31 15:21:00 +00:00
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
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " There are no mutations for partition ID {} (trying to mutate part {} to {}) " , part - > info . partition_id , part - > name , 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 )
2022-02-01 09:30:58 +00:00
LOG_WARNING ( log ,
" Mutation with version {} not found in partition ID {} (trying to mutate part {}) " ,
desired_mutation_version ,
part - > info . partition_id ,
part - > name ) ;
2018-05-31 15:21:00 +00:00
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 )
2023-02-27 11:27:57 +00:00
{
2023-05-16 16:18:48 +00:00
/// FIXME uncomment this assertion after relesing 23.5 (currently it fails in Upgrade check)
/// chassert(mutation_pointer < it->second->entry->znode_name);
2023-05-15 16:28:12 +00:00
mutation_ids . push_back ( it - > second - > entry - > znode_name ) ;
2023-02-27 11:27:57 +00:00
const auto & commands_from_entry = it - > second - > entry - > commands ;
2023-02-27 13:48:47 +00:00
commands . insert ( commands . end ( ) , commands_from_entry . begin ( ) , commands_from_entry . end ( ) ) ;
2023-02-27 11:27:57 +00:00
}
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 )
{
2022-11-16 10:50:51 +00:00
std : : vector < ReplicatedMergeTreeMutationEntryPtr > candidates ;
2018-06-21 13:27:36 +00:00
{
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 )
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Marking mutation {} done because it is <= mutation_pointer ({}) " , znode , mutation_pointer ) ;
2018-06-21 13:27:36 +00:00
mutation . is_done = true ;
2022-02-04 13:27:59 +00:00
mutation . latest_fail_reason . clear ( ) ;
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 )
{
2023-05-08 22:21:07 +00:00
LOG_INFO ( log , " Seems like we jumped over mutation {} 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. " , znode ) ;
2020-02-05 16:30:02 +00:00
mutation . parts_to_do . clear ( ) ;
}
2018-06-21 13:27:36 +00:00
}
2020-03-09 02:55:28 +00:00
else if ( mutation . parts_to_do . size ( ) = = 0 )
2018-06-21 13:27:36 +00:00
{
2022-11-17 12:10:14 +00:00
/// Why it doesn't mean that mutation 100% finished? Because when we were creating part_to_do set
/// some INSERT queries could be in progress. So we have to double-check that no affected committing block
/// numbers exist and no new parts were surprisingly committed.
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Will check if mutation {} is done " , mutation . entry - > znode_name ) ;
2022-11-16 10:50:51 +00:00
candidates . emplace_back ( mutation . entry ) ;
2018-06-21 13:27:36 +00:00
}
}
}
if ( candidates . empty ( ) )
return false ;
2020-05-21 15:52:11 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Trying to finalize {} mutations " , candidates . size ( ) ) ;
2018-06-21 13:27:36 +00:00
2022-11-17 12:10:14 +00:00
/// We need to check committing block numbers and new parts which could be committed.
/// Actually we don't need most of predicate logic here but it all the code related to committing blocks
2022-11-25 15:41:20 +00:00
/// and updatating queue state is implemented there.
PartitionIdsHint partition_ids_hint ;
for ( const auto & candidate : candidates )
for ( const auto & partitions : candidate - > block_numbers )
2023-05-08 22:21:07 +00:00
if ( ! candidate - > checked_partitions_cache . contains ( partitions . first ) )
partition_ids_hint . insert ( partitions . first ) ;
2022-11-25 15:41:20 +00:00
2022-11-28 15:02:37 +00:00
auto merge_pred = getMergePredicate ( zookeeper , std : : move ( partition_ids_hint ) ) ;
2018-06-21 13:27:36 +00:00
std : : vector < const ReplicatedMergeTreeMutationEntry * > finished ;
2022-11-15 13:46:38 +00:00
for ( const auto & candidate : candidates )
2018-06-21 13:27:36 +00:00
{
2023-05-08 22:21:07 +00:00
if ( merge_pred . isMutationFinished ( candidate - > znode_name , candidate - > block_numbers , candidate - > checked_partitions_cache ) )
2022-11-16 10:50:51 +00:00
finished . push_back ( candidate . get ( ) ) ;
2018-06-21 13:27:36 +00:00
}
if ( ! finished . empty ( ) )
2019-08-19 18:12:22 +00:00
{
2021-05-08 10:59:55 +00:00
zookeeper - > set ( fs : : path ( replica_path ) / " mutation_pointer " , finished . back ( ) - > znode_name ) ;
2018-06-21 13:27:36 +00:00
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 ( ) )
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Mutation {} is done " , entry - > znode_name ) ;
2018-06-21 13:27:36 +00:00
it - > second . is_done = true ;
2022-02-04 13:27:59 +00:00
it - > second . latest_fail_reason . clear ( ) ;
2020-02-17 16:33:05 +00:00
if ( entry - > isAlterMutation ( ) )
2020-02-05 11:18:11 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Finishing data alter with version {} for entry {} " , entry - > alter_version , 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
}
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
2016-01-10 04:44:12 +00:00
Status res ;
2017-04-01 07:20:54 +00:00
2022-10-07 10:46:45 +00:00
res . future_parts = static_cast < UInt32 > ( future_parts . size ( ) ) ;
res . queue_size = static_cast < UInt32 > ( queue . size ( ) ) ;
res . last_queue_update = static_cast < UInt32 > ( last_queue_update ) ;
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
res . inserts_in_queue = 0 ;
res . merges_in_queue = 0 ;
2018-06-06 13:22:30 +00:00
res . part_mutations_in_queue = 0 ;
2016-01-10 04:44:12 +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
2016-01-10 04:44:12 +00:00
for ( const LogEntryPtr & entry : queue )
{
if ( entry - > create_time & & ( ! res . queue_oldest_time | | entry - > create_time < res . queue_oldest_time ) )
2022-10-07 10:46:45 +00:00
res . queue_oldest_time = static_cast < UInt32 > ( entry - > create_time ) ;
2017-04-01 07:20:54 +00:00
2021-02-15 15:06:48 +00:00
if ( entry - > type = = LogEntry : : GET_PART | | entry - > type = = LogEntry : : ATTACH_PART )
2016-01-10 04:44:12 +00:00
{
+ + res . inserts_in_queue ;
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
if ( entry - > create_time & & ( ! res . inserts_oldest_time | | entry - > create_time < res . inserts_oldest_time ) )
{
2022-10-07 10:46:45 +00:00
res . inserts_oldest_time = static_cast < UInt32 > ( entry - > create_time ) ;
2016-01-10 04:44:12 +00:00
res . oldest_part_to_get = entry - > new_part_name ;
}
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
if ( entry - > type = = LogEntry : : MERGE_PARTS )
{
+ + res . merges_in_queue ;
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
if ( entry - > create_time & & ( ! res . merges_oldest_time | | entry - > create_time < res . merges_oldest_time ) )
{
2022-10-07 10:46:45 +00:00
res . merges_oldest_time = static_cast < UInt32 > ( entry - > create_time ) ;
2016-01-10 04:44:12 +00:00
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
{
2022-10-07 10:46:45 +00:00
res . part_mutations_oldest_time = static_cast < UInt32 > ( entry - > create_time ) ;
2018-04-20 16:18:16 +00:00
res . oldest_part_to_mutate_to = entry - > new_part_name ;
}
}
2016-01-10 04:44:12 +00:00
}
2017-04-01 07:20:54 +00:00
2016-01-10 04:44:12 +00:00
return res ;
}
2018-03-03 16:26:06 +00:00
void ReplicatedMergeTreeQueue : : getEntries ( LogEntriesData & res ) const
2016-01-10 04:44:12 +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
res . reserve ( queue . size ( ) ) ;
for ( const auto & entry : queue )
res . emplace_back ( * entry ) ;
}
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
{
2023-02-12 06:39:16 +00:00
out_min_unprocessed_insert_time = min_unprocessed_insert_time . load ( std : : memory_order_relaxed ) ;
out_max_processed_insert_time = max_processed_insert_time . load ( std : : memory_order_relaxed ) ;
2016-01-17 13:00:42 +00:00
}
2020-07-31 11:37:16 +00:00
std : : optional < MergeTreeMutationStatus > ReplicatedMergeTreeQueue : : getIncompleteMutationsStatus ( const String & znode_name , std : : set < String > * mutation_ids ) const
2020-07-22 12:36:19 +00:00
{
std : : lock_guard lock ( state_mutex ) ;
2020-07-22 15:19:54 +00:00
auto current_mutation_it = mutations_by_znode . find ( znode_name ) ;
2020-07-22 12:36:19 +00:00
/// killed
2020-07-22 15:19:54 +00:00
if ( current_mutation_it = = mutations_by_znode . end ( ) )
2020-07-22 12:36:19 +00:00
return { } ;
2020-07-22 15:19:54 +00:00
const MutationStatus & status = current_mutation_it - > second ;
2020-07-22 12:36:19 +00:00
MergeTreeMutationStatus result
{
. is_done = status . is_done ,
. latest_failed_part = status . latest_failed_part ,
. latest_fail_time = status . latest_fail_time ,
. latest_fail_reason = status . latest_fail_reason ,
} ;
2020-07-22 15:19:54 +00:00
if ( mutation_ids & & ! status . latest_fail_reason . empty ( ) )
{
const auto & latest_failed_part_info = status . latest_failed_part_info ;
auto in_partition = mutations_by_partition . find ( latest_failed_part_info . partition_id ) ;
if ( in_partition ! = mutations_by_partition . end ( ) )
{
const auto & version_to_status = in_partition - > second ;
auto begin_it = version_to_status . upper_bound ( latest_failed_part_info . getDataVersion ( ) ) ;
for ( auto it = begin_it ; it ! = version_to_status . end ( ) ; + + it )
{
/// All mutations with the same failure
if ( ! it - > second - > is_done & & it - > second - > latest_fail_reason = = status . latest_fail_reason )
2020-07-31 11:37:16 +00:00
mutation_ids - > insert ( it - > second - > entry - > znode_name ) ;
2020-07-22 15:19:54 +00:00
}
}
}
2020-07-22 12:36:19 +00:00
return result ;
}
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 )
{
2020-11-09 16:05:40 +00:00
WriteBufferFromOwnString buf ;
formatAST ( * command . ast , buf , false , true ) ;
2018-06-07 13:28:39 +00:00
result . push_back ( MergeTreeMutationStatus
{
entry . znode_name ,
2020-11-09 16:05:40 +00:00
buf . str ( ) ,
2018-06-07 13:28:39 +00:00
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 ;
}
2020-03-27 10:53:04 +00:00
ReplicatedMergeTreeQueue : : QueueLocks ReplicatedMergeTreeQueue : : lockQueue ( )
{
return QueueLocks ( state_mutex , pull_logs_to_queue_mutex , update_mutations_mutex ) ;
}
2018-06-07 13:28:39 +00:00
2023-05-22 13:18:29 +00:00
LocalMergePredicate : : LocalMergePredicate ( ReplicatedMergeTreeQueue & queue_ )
2022-09-29 09:19:47 +00:00
: queue ( queue_ )
2023-05-07 22:22:07 +00:00
{
}
ReplicatedMergeTreeMergePredicate : : ReplicatedMergeTreeMergePredicate (
ReplicatedMergeTreeQueue & queue_ , zkutil : : ZooKeeperPtr & zookeeper , std : : optional < PartitionIdsHint > & & partition_ids_hint_ )
: nested_pred ( queue_ )
, queue ( queue_ )
2022-11-25 15:41:20 +00:00
, partition_ids_hint ( std : : move ( partition_ids_hint_ ) )
2022-09-29 09:19:47 +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.
2021-05-08 10:59:55 +00:00
auto quorum_status_future = zookeeper - > asyncTryGet ( fs : : path ( queue . zookeeper_path ) / " quorum " / " status " ) ;
2018-07-05 10:27:46 +00:00
2018-05-10 15:01:10 +00:00
/// Load current inserts
2022-11-25 15:41:20 +00:00
/// Hint avoids listing partitions that we don't really need.
/// Dropped (or cleaned up by TTL) partitions are never removed from ZK,
/// so without hint it can do a few thousands requests (if not using MultiRead).
Strings partitions ;
2023-05-07 22:22:07 +00:00
if ( ! partition_ids_hint )
2022-11-25 15:41:20 +00:00
partitions = zookeeper - > getChildren ( fs : : path ( queue . zookeeper_path ) / " block_numbers " ) ;
else
2023-05-07 22:22:07 +00:00
std : : copy ( partition_ids_hint - > begin ( ) , partition_ids_hint - > end ( ) , std : : back_inserter ( partitions ) ) ;
2022-11-25 15:41:20 +00:00
2022-11-02 13:28:49 +00:00
std : : vector < std : : string > paths ;
paths . reserve ( partitions . size ( ) ) ;
for ( const String & partition : partitions )
paths . push_back ( fs : : path ( queue . zookeeper_path ) / " block_numbers " / partition ) ;
2022-09-29 09:19:47 +00:00
2023-05-09 21:26:56 +00:00
auto locks_children = zookeeper - > tryGetChildren ( paths ) ;
2018-05-10 15:01:10 +00:00
2022-11-02 13:28:49 +00:00
for ( size_t i = 0 ; i < partitions . size ( ) ; + + i )
{
2023-05-09 21:26:56 +00:00
auto & response = locks_children [ i ] ;
if ( response . error ! = Coordination : : Error : : ZOK & & ! partition_ids_hint )
throw Coordination : : Exception ( response . error , paths [ i ] ) ;
if ( response . error ! = Coordination : : Error : : ZOK )
{
/// Probably a wrong hint was provided (it's ok if a user passed non-existing partition to OPTIMIZE)
LOG_WARNING ( queue . log , " Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper " , partitions [ i ] ) ;
partition_ids_hint - > erase ( partitions [ i ] ) ;
continue ;
}
Strings partition_block_numbers = response . names ;
2022-11-02 13:28:49 +00:00
for ( const String & entry : partition_block_numbers )
2018-05-10 15:01:10 +00:00
{
2022-11-02 13:28:49 +00:00
if ( ! startsWith ( entry , " block- " ) )
continue ;
Int64 block_number = parse < Int64 > ( entry . substr ( strlen ( " block- " ) ) ) ;
String zk_path = fs : : path ( queue . zookeeper_path ) / " block_numbers " / partitions [ i ] / entry ;
committing_blocks [ partitions [ i ] ] . insert ( block_number ) ;
2018-05-10 15:01:10 +00:00
}
}
2021-08-18 09:49:22 +00:00
merges_version = queue_ . pullLogsToQueue ( zookeeper , { } , ReplicatedMergeTreeQueue : : MERGE_PREDICATE ) ;
2018-05-10 15:01:10 +00:00
2020-11-24 14:24:48 +00:00
{
/// We avoid returning here a version to be used in a lightweight transaction.
///
/// When pinned parts set is changed a log entry is added to the queue in the same transaction.
/// The log entry serves as a synchronization point, and it also increments `merges_version`.
///
/// If pinned parts are fetched after logs are pulled then we can safely say that it contains all locks up to `merges_version`.
String s = zookeeper - > get ( queue . zookeeper_path + " /pinned_part_uuids " ) ;
pinned_part_uuids . fromString ( s ) ;
}
2018-08-25 01:58:14 +00:00
Coordination : : GetResponse quorum_status_response = quorum_status_future . get ( ) ;
2020-06-12 15:09:12 +00:00
if ( quorum_status_response . error = = Coordination : : Error : : ZOK )
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 ( ) ;
}
2023-05-22 13:18:29 +00:00
bool LocalMergePredicate : : operator ( ) (
2023-05-07 22:22:07 +00:00
const MergeTreeData : : DataPartPtr & left ,
const MergeTreeData : : DataPartPtr & right ,
const MergeTreeTransaction * ,
String * out_reason ) const
{
if ( left )
return canMergeTwoParts ( left , right , out_reason ) ;
else
return canMergeSinglePart ( right , out_reason ) ;
}
2018-05-10 15:01:10 +00:00
bool ReplicatedMergeTreeMergePredicate : : operator ( ) (
2020-04-16 18:47:20 +00:00
const MergeTreeData : : DataPartPtr & left ,
const MergeTreeData : : DataPartPtr & right ,
2021-05-18 17:07:29 +00:00
const MergeTreeTransaction * ,
2020-04-16 18:47:20 +00:00
String * out_reason ) const
{
if ( left )
return canMergeTwoParts ( left , right , out_reason ) ;
else
return canMergeSinglePart ( right , out_reason ) ;
}
bool ReplicatedMergeTreeMergePredicate : : canMergeTwoParts (
const MergeTreeData : : DataPartPtr & left ,
const MergeTreeData : : DataPartPtr & right ,
String * out_reason ) const
2018-05-10 15:01:10 +00:00
{
/// 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 )
{
2021-05-10 18:03:37 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Parts {} and {} belong to different partitions " , left - > name , right - > name ) ;
2018-05-10 15:01:10 +00:00
}
for ( const MergeTreeData : : DataPartPtr & part : { left , right } )
{
2020-11-24 14:24:48 +00:00
if ( pinned_part_uuids . part_uuids . contains ( part - > uuid ) )
{
if ( out_reason )
* out_reason = " Part " + part - > name + " has uuid " + toString ( part - > uuid ) + " which is currently pinned " ;
return false ;
}
2018-05-10 15:01:10 +00:00
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 )
{
2023-05-07 22:22:07 +00:00
if ( partition_ids_hint & & ! partition_ids_hint - > contains ( left - > info . partition_id ) )
2022-11-25 15:41:20 +00:00
{
if ( out_reason )
* out_reason = fmt : : format ( " Uncommitted block were not loaded for unexpected partition {} " , left - > info . partition_id ) ;
return false ;
}
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
}
2023-05-07 22:22:07 +00:00
return nested_pred . canMergeTwoParts ( left , right , out_reason ) ;
}
2023-05-22 13:18:29 +00:00
bool LocalMergePredicate : : canMergeTwoParts (
2023-05-07 22:22:07 +00:00
const MergeTreeData : : DataPartPtr & left ,
const MergeTreeData : : DataPartPtr & right ,
String * out_reason ) const
{
Int64 left_max_block = left - > info . max_block ;
Int64 right_min_block = right - > info . min_block ;
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 ( )
2020-04-10 17:23:45 +00:00
+ " to " + covered . back ( ) + " ) that are still not present or being processed by "
2019-08-16 15:57:19 +00:00
+ " other background process on this replica between " + left - > name + " and " + right - > name ;
2018-05-10 15:01:10 +00:00
return false ;
}
}
2023-04-12 22:20:43 +00:00
Int64 left_mutation_ver = queue . getCurrentMutationVersion (
2018-06-20 11:12:16 +00:00
left - > info . partition_id , left - > info . getDataVersion ( ) , lock ) ;
2019-08-16 15:57:19 +00:00
2023-04-12 22:20:43 +00:00
Int64 right_mutation_ver = queue . getCurrentMutationVersion (
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 ;
}
2021-02-10 14:12:49 +00:00
return MergeTreeData : : partsContainSameProjections ( left , right ) ;
2018-05-10 15:01:10 +00:00
}
2020-04-16 18:47:20 +00:00
bool ReplicatedMergeTreeMergePredicate : : canMergeSinglePart (
const MergeTreeData : : DataPartPtr & part ,
String * out_reason ) const
2020-04-10 21:29:54 +00:00
{
2020-11-24 14:24:48 +00:00
if ( pinned_part_uuids . part_uuids . contains ( part - > uuid ) )
{
if ( out_reason )
2022-04-12 12:14:26 +00:00
* out_reason = fmt : : format ( " Part {} has uuid {} which is currently pinned " , part - > name , part - > uuid ) ;
2020-11-24 14:24:48 +00:00
return false ;
}
2020-04-10 21:29:54 +00:00
if ( part - > name = = inprogress_quorum_part )
{
if ( out_reason )
2022-04-12 12:14:26 +00:00
* out_reason = fmt : : format ( " Quorum insert for part {} is currently in progress " , part - > name ) ;
2020-04-10 21:29:54 +00:00
return false ;
}
if ( prev_virtual_parts . getContainingPart ( part - > info ) . empty ( ) )
{
if ( out_reason )
2022-04-12 12:14:26 +00:00
* out_reason = fmt : : format ( " Entry for part {} hasn't been read from the replication log yet " , part - > name ) ;
2020-04-10 21:29:54 +00:00
return false ;
}
2023-05-07 22:22:07 +00:00
return nested_pred . canMergeSinglePart ( part , out_reason ) ;
}
2023-05-22 13:18:29 +00:00
bool LocalMergePredicate : : canMergeSinglePart ( const MergeTreeData : : DataPartPtr & part , String * out_reason ) const
2023-05-07 22:22:07 +00:00
{
2023-03-02 13:36:47 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2020-04-10 21:29:54 +00:00
/// 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 )
2022-04-12 12:14:26 +00:00
* out_reason = fmt : : format ( " Part {} has already been assigned a merge into {} " , part - > name , containing_part ) ;
2020-04-10 21:29:54 +00:00
return false ;
}
return true ;
}
2018-05-10 15:01:10 +00:00
2022-06-01 18:11:53 +00:00
bool ReplicatedMergeTreeMergePredicate : : partParticipatesInReplaceRange ( const MergeTreeData : : DataPartPtr & part , String * out_reason ) const
{
2023-03-02 13:36:47 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2022-06-01 18:11:53 +00:00
for ( const auto & entry : queue . queue )
{
if ( entry - > type ! = ReplicatedMergeTreeLogEntry : : REPLACE_RANGE )
continue ;
for ( const auto & part_name : entry - > replace_range_entry - > new_part_names )
{
if ( part - > info . isDisjoint ( MergeTreePartInfo : : fromPartName ( part_name , queue . format_version ) ) )
continue ;
if ( out_reason )
* out_reason = fmt : : format ( " Part {} participates in REPLACE_RANGE {} ({}) " , part_name , entry - > new_part_name , entry - > znode_name ) ;
return true ;
}
}
return false ;
}
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-04-10 17:23:45 +00:00
/// We cannot mutate part if it's being inserted with quorum and it's not
2020-03-02 12:56:38 +00:00
/// 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 { } ;
2023-04-12 22:20:43 +00:00
UInt64 mutations_limit = queue . storage . getSettings ( ) - > replicated_max_mutations_in_one_entry ;
UInt64 mutations_count = 0 ;
Int64 current_version = queue . getCurrentMutationVersion ( part - > info . partition_id , part - > info . getDataVersion ( ) , lock ) ;
2023-02-27 13:35:26 +00:00
Int64 max_version = in_partition - > second . begin ( ) - > first ;
2020-01-31 19:30:33 +00:00
2020-01-31 12:25:31 +00:00
int alter_version = - 1 ;
2023-02-27 13:35:26 +00:00
bool barrier_found = false ;
2020-01-31 12:25:31 +00:00
for ( auto [ mutation_version , mutation_status ] : in_partition - > second )
{
2023-02-27 13:35:26 +00:00
/// Some commands cannot stick together with other commands
if ( mutation_status - > entry - > commands . containBarrierCommand ( ) )
{
/// We already collected some mutation, we don't want to stick it with barrier
if ( max_version ! = mutation_version & & max_version > current_version )
break ;
/// This mutations is fresh, but it's barrier, let's execute only it
if ( mutation_version > current_version )
barrier_found = true ;
}
2020-01-31 12:25:31 +00:00
max_version = mutation_version ;
2023-04-13 16:13:44 +00:00
if ( current_version < max_version )
+ + mutations_count ;
2023-02-27 13:35:26 +00:00
2020-05-04 16:30:39 +00:00
if ( mutation_status - > entry - > isAlterMutation ( ) )
2020-01-31 12:25:31 +00:00
{
2020-05-04 16:30:39 +00:00
/// We want to assign mutations for part which version is bigger
2020-08-08 00:47:03 +00:00
/// than part current version. But it doesn't make sense to assign
2020-05-04 16:30:39 +00:00
/// more fresh versions of alter-mutations if previous alter still
/// not done because alters execute one by one in strict order.
if ( mutation_version > current_version | | ! mutation_status - > is_done )
{
alter_version = mutation_status - > entry - > alter_version ;
break ;
}
2020-01-31 12:25:31 +00:00
}
2023-04-12 22:20:43 +00:00
if ( mutations_limit & & mutations_count = = mutations_limit )
{
LOG_WARNING ( queue . log , " Will apply only {} of {} mutations and mutate part {} to version {} (the last version is {}) " ,
mutations_count , in_partition - > second . size ( ) , part - > name , max_version , in_partition - > second . rbegin ( ) - > first ) ;
break ;
}
2023-04-13 16:13:44 +00:00
2023-02-27 13:35:26 +00:00
if ( barrier_found = = true )
break ;
2020-01-31 12:25:31 +00:00
}
2020-02-13 12:03:30 +00:00
2018-04-20 16:18:16 +00:00
if ( current_version > = max_version )
return { } ;
2023-04-12 22:20:43 +00:00
LOG_TRACE ( queue . log , " Will apply {} mutations and mutate part {} to version {} (the last version is {}) " ,
mutations_count , part - > name , max_version , in_partition - > second . rbegin ( ) - > first ) ;
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
2023-05-08 22:21:07 +00:00
bool ReplicatedMergeTreeMergePredicate : : isMutationFinished ( const std : : string & znode_name , const std : : map < String , int64_t > & block_numbers ,
std : : unordered_set < String > & checked_partitions_cache ) const
2018-06-21 13:27:36 +00:00
{
2022-11-17 12:10:14 +00:00
/// Check committing block numbers, maybe some affected inserts
/// still not written to disk and committed to ZK.
2022-11-16 10:50:51 +00:00
for ( const auto & kv : block_numbers )
2018-06-21 13:27:36 +00:00
{
const String & partition_id = kv . first ;
Int64 block_num = kv . second ;
2023-05-08 22:21:07 +00:00
/// Maybe we already know that there are no relevant uncommitted blocks
if ( checked_partitions_cache . contains ( partition_id ) )
continue ;
2023-05-07 22:22:07 +00:00
if ( partition_ids_hint & & ! partition_ids_hint - > contains ( partition_id ) )
2022-11-25 15:41:20 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Partition id {} was not provided as hint, it's a bug " , partition_id ) ;
2018-06-21 13:27:36 +00:00
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 )
{
2022-11-16 10:50:51 +00:00
LOG_TRACE ( queue . log , " Mutation {} is not done yet because in partition ID {} there are still {} uncommitted blocks. " , znode_name , partition_id , blocks_count ) ;
2018-06-21 13:27:36 +00:00
return false ;
}
}
2023-05-08 22:21:07 +00:00
/// There are no committing blocks less than block_num in that partition and there's no way they can appear
/// TODO Why not to get committing blocks when pulling a mutation? We could get rid of finalization task or simplify it
checked_partitions_cache . insert ( partition_id ) ;
2018-06-21 15:54:01 +00:00
}
2018-06-21 13:27:36 +00:00
2022-11-15 13:46:38 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2022-11-17 12:10:14 +00:00
/// When we creating predicate we have updated the queue. Some committing inserts can now be committed so
/// we check parts_to_do one more time. Also this code is async so mutation actually could be deleted from memory.
2022-11-16 10:50:51 +00:00
if ( auto it = queue . mutations_by_znode . find ( znode_name ) ; it ! = queue . mutations_by_znode . end ( ) )
2022-11-15 13:46:38 +00:00
{
2022-11-16 10:50:51 +00:00
if ( it - > second . parts_to_do . size ( ) = = 0 )
return true ;
LOG_TRACE ( queue . log , " Mutation {} is not done because some parts [{}] were just committed " , znode_name , fmt : : join ( it - > second . parts_to_do . getParts ( ) , " , " ) ) ;
return false ;
2022-11-15 13:46:38 +00:00
}
else
{
2022-11-16 10:50:51 +00:00
LOG_TRACE ( queue . log , " Mutation {} is done because it doesn't exist anymore " , znode_name ) ;
return true ;
2022-11-15 13:46:38 +00:00
}
2018-06-21 13:27:36 +00:00
}
2023-01-31 12:37:56 +00:00
bool ReplicatedMergeTreeMergePredicate : : isGoingToBeDropped ( const MergeTreePartInfo & new_drop_range_info ,
MergeTreePartInfo * out_drop_range_info ) const
2021-07-05 19:58:36 +00:00
{
2023-01-31 12:37:56 +00:00
return queue . isGoingToBeDropped ( new_drop_range_info , out_drop_range_info ) ;
2021-07-05 19:58:36 +00:00
}
2022-07-18 21:37:07 +00:00
String ReplicatedMergeTreeMergePredicate : : getCoveringVirtualPart ( const String & part_name ) const
{
2023-03-02 13:36:47 +00:00
std : : lock_guard lock ( queue . state_mutex ) ;
2022-07-18 21:37:07 +00:00
return queue . virtual_parts . getContainingPart ( MergeTreePartInfo : : fromPartName ( part_name , queue . format_version ) ) ;
}
2018-06-21 13:27:36 +00:00
2018-05-21 13:49:54 +00:00
ReplicatedMergeTreeQueue : : SubscriberHandler
2023-03-31 14:09:00 +00:00
ReplicatedMergeTreeQueue : : addSubscriber ( ReplicatedMergeTreeQueue : : SubscriberCallBack & & callback ,
std : : unordered_set < String > & out_entry_names , SyncReplicaMode sync_mode )
2018-05-21 13:49:54 +00:00
{
2023-03-27 23:39:36 +00:00
std : : lock_guard < std : : mutex > lock ( state_mutex ) ;
2019-01-02 06:44:36 +00:00
std : : lock_guard lock_subscribers ( subscribers_mutex ) ;
2018-05-21 13:49:54 +00:00
2023-03-31 14:09:00 +00:00
if ( sync_mode ! = SyncReplicaMode : : PULL )
{
/// We must get the list of entries to wait atomically with adding the callback
bool lightweight_entries_only = sync_mode = = SyncReplicaMode : : LIGHTWEIGHT ;
static constexpr std : : array lightweight_entries =
{
LogEntry : : GET_PART ,
LogEntry : : ATTACH_PART ,
LogEntry : : DROP_RANGE ,
LogEntry : : REPLACE_RANGE ,
LogEntry : : DROP_PART
} ;
out_entry_names . reserve ( queue . size ( ) ) ;
for ( const auto & entry : queue )
{
if ( ! lightweight_entries_only
| | std : : find ( lightweight_entries . begin ( ) , lightweight_entries . end ( ) , entry - > type ) ! = lightweight_entries . end ( ) )
out_entry_names . insert ( entry - > znode_name ) ;
}
2023-04-07 18:16:25 +00:00
LOG_TEST ( log , " Waiting for {} entries to be processed: {} " , out_entry_names . size ( ) , fmt : : join ( out_entry_names , " , " ) ) ;
2023-03-31 14:09:00 +00:00
}
2018-05-21 13:49:54 +00:00
auto it = subscribers . emplace ( subscribers . end ( ) , std : : move ( callback ) ) ;
2023-03-27 23:39:36 +00:00
/// Atomically notify about current size
( * it ) ( queue . size ( ) , nullptr ) ;
2023-01-30 07:18:20 +00:00
2018-05-21 13:49:54 +00:00
return SubscriberHandler ( it , * this ) ;
}
2023-03-27 23:39:36 +00:00
void ReplicatedMergeTreeQueue : : notifySubscribersOnPartialShutdown ( )
2018-05-21 13:49:54 +00:00
{
2023-03-27 23:39:36 +00:00
size_t queue_size ;
{
std : : lock_guard < std : : mutex > lock ( state_mutex ) ;
queue_size = 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 )
2023-03-27 23:39:36 +00:00
subscriber_callback ( queue_size , nullptr ) ;
}
ReplicatedMergeTreeQueue : : SubscriberHandler : : ~ SubscriberHandler ( )
{
std : : lock_guard lock ( queue . subscribers_mutex ) ;
queue . subscribers . erase ( it ) ;
2018-05-21 13:49:54 +00:00
}
2023-03-27 23:39:36 +00:00
void ReplicatedMergeTreeQueue : : notifySubscribers ( size_t new_queue_size , const String * removed_log_entry_id )
2018-05-21 13:49:54 +00:00
{
2023-03-27 23:39:36 +00:00
std : : lock_guard lock_subscribers ( subscribers_mutex ) ;
for ( auto & subscriber_callback : subscribers )
subscriber_callback ( new_queue_size , removed_log_entry_id ) ;
2018-05-21 13:49:54 +00:00
}
2018-04-20 16:18:16 +00:00
2016-01-10 04:44:12 +00:00
String padIndex ( Int64 index )
{
String index_str = toString ( index ) ;
return std : : string ( 10 - index_str . size ( ) , ' 0 ' ) + index_str ;
}
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 ( ) )
2023-05-15 16:28:12 +00:00
removeCoveredPartsFromMutations ( part_name , /*remove_part = */ false , /*remove_covered_parts = */ true ) ;
2020-02-18 19:57:48 +00:00
}
2020-06-16 10:50:47 +00:00
2016-01-10 04:44:12 +00:00
}