2017-04-01 09:19:00 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
# include <Storages/StorageReplicatedMergeTree.h>
2017-07-24 20:12:59 +00:00
# include <Poco/Timestamp.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2021-12-07 16:55:55 +00:00
# include <Common/ZooKeeper/KeeperException.h>
2014-10-15 01:22:06 +00:00
2018-04-06 19:43:37 +00:00
# include <random>
2018-08-28 00:10:05 +00:00
# include <unordered_set>
2018-04-06 19:43:37 +00:00
2022-01-30 19:49:48 +00:00
# include <base/sort.h>
2014-10-15 01:22:06 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
extern const int NOT_FOUND_NODE ;
2018-08-22 14:01:54 +00:00
extern const int ALL_REPLICAS_LOST ;
2018-08-23 13:55:59 +00:00
extern const int REPLICA_STATUS_CHANGED ;
2016-01-11 21:46:36 +00:00
}
2014-10-15 01:22:06 +00:00
ReplicatedMergeTreeCleanupThread : : ReplicatedMergeTreeCleanupThread ( StorageReplicatedMergeTree & storage_ )
2018-05-31 13:05:05 +00:00
: storage ( storage_ )
2019-12-03 16:25:32 +00:00
, log_name ( storage . getStorageID ( ) . getFullTableName ( ) + " (ReplicatedMergeTreeCleanupThread) " )
2020-05-30 21:57:37 +00:00
, log ( & Poco : : Logger : : get ( log_name ) )
2017-11-15 16:32:47 +00:00
{
2021-04-10 23:33:54 +00:00
task = storage . getContext ( ) - > getSchedulePool ( ) . createTask ( log_name , [ this ] { run ( ) ; } ) ;
2017-12-29 22:32:04 +00:00
}
2014-10-15 01:22:06 +00:00
void ReplicatedMergeTreeCleanupThread : : run ( )
{
2019-08-26 14:24:29 +00:00
auto storage_settings = storage . getSettings ( ) ;
2020-03-23 02:12:31 +00:00
const auto sleep_ms = storage_settings - > cleanup_delay_period * 1000
2019-08-13 10:29:31 +00:00
+ std : : uniform_int_distribution < UInt64 > ( 0 , storage_settings - > cleanup_delay_period_random_add * 1000 ) ( rng ) ;
2014-10-15 01:22:06 +00:00
2017-12-29 22:32:04 +00:00
try
2014-10-15 01:22:06 +00:00
{
2017-12-29 22:32:04 +00:00
iterate ( ) ;
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2018-04-24 17:11:59 +00:00
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-12-21 18:17:06 +00:00
2020-06-12 15:09:12 +00:00
if ( e . code = = Coordination : : Error : : ZSESSIONEXPIRED )
2018-04-24 17:11:59 +00:00
return ;
2014-10-15 01:22:06 +00:00
}
2017-12-29 22:32:04 +00:00
catch ( . . . )
{
2018-04-10 13:20:14 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2014-10-15 01:22:06 +00:00
}
2020-03-23 02:12:31 +00:00
task - > scheduleAfter ( sleep_ms ) ;
2014-10-15 01:22:06 +00:00
}
2017-12-21 18:17:06 +00:00
2014-10-15 01:22:06 +00:00
void ReplicatedMergeTreeCleanupThread : : iterate ( )
{
2017-10-06 11:30:57 +00:00
storage . clearOldPartsAndRemoveFromZK ( ) ;
2019-03-29 09:28:16 +00:00
{
2020-06-18 16:10:47 +00:00
auto lock = storage . lockForShare ( RWLockImpl : : NO_QUERY , storage . getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2020-09-25 12:07:38 +00:00
/// Both use relative_data_path which changes during rename, so we
/// do it under share lock
storage . clearOldWriteAheadLogs ( ) ;
2022-02-24 12:23:26 +00:00
storage . clearOldTemporaryDirectories ( storage . getSettings ( ) - > temporary_directories_lifetime . totalSeconds ( ) ) ;
2022-06-10 11:19:37 +00:00
if ( storage . getSettings ( ) - > merge_tree_enable_clear_old_broken_detached )
2022-09-05 01:50:24 +00:00
storage . clearOldBrokenPartsFromDetachedDirectory ( ) ;
2019-03-29 09:28:16 +00:00
}
2017-04-01 07:20:54 +00:00
2017-11-19 21:17:58 +00:00
/// This is loose condition: no problem if we actually had lost leadership at this moment
/// and two replicas will try to do cleanup simultaneously.
2018-04-06 16:06:07 +00:00
if ( storage . is_leader )
2014-10-15 01:22:06 +00:00
{
clearOldLogs ( ) ;
2022-11-22 14:12:00 +00:00
auto storage_settings = storage . getSettings ( ) ;
2022-12-30 15:30:33 +00:00
clearOldBlocks ( " blocks " , storage_settings - > replicated_deduplication_window_seconds , storage_settings - > replicated_deduplication_window , cached_block_stats_for_sync_inserts ) ;
clearOldBlocks ( " async_blocks " , storage_settings - > replicated_deduplication_window_seconds_for_async_inserts , storage_settings - > replicated_deduplication_window_for_async_inserts , cached_block_stats_for_async_inserts ) ;
2018-07-31 11:36:08 +00:00
clearOldMutations ( ) ;
2020-11-11 16:18:21 +00:00
storage . clearEmptyParts ( ) ;
2014-10-15 01:22:06 +00:00
}
}
void ReplicatedMergeTreeCleanupThread : : clearOldLogs ( )
{
2014-12-12 20:50:32 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
2019-08-26 14:24:29 +00:00
auto storage_settings = storage . getSettings ( ) ;
2017-04-01 07:20:54 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2014-12-12 20:50:32 +00:00
if ( ! zookeeper - > exists ( storage . zookeeper_path + " /log " , & stat ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_FOUND_NODE , " {}/log doesn't exist " , storage . zookeeper_path ) ;
2017-04-01 07:20:54 +00:00
2014-10-15 01:22:06 +00:00
int children_count = stat . numChildren ;
2017-04-01 07:20:54 +00:00
2020-06-15 02:12:06 +00:00
/// We will wait for 1.05 to 1.15 times more records to accumulate than necessary.
/// Randomization is needed to spread the time when multiple replicas come here.
/// Numbers are arbitrary.
std : : uniform_real_distribution < double > distr ( 1.05 , 1.15 ) ;
double ratio = distr ( rng ) ;
2022-09-11 01:21:34 +00:00
size_t min_replicated_logs_to_keep = static_cast < size_t > ( storage_settings - > min_replicated_logs_to_keep * ratio ) ;
2020-06-15 02:12:06 +00:00
if ( static_cast < double > ( children_count ) < min_replicated_logs_to_keep )
2014-10-15 01:22:06 +00:00
return ;
2017-04-01 07:20:54 +00:00
2014-12-12 20:50:32 +00:00
Strings replicas = zookeeper - > getChildren ( storage . zookeeper_path + " /replicas " , & stat ) ;
2018-08-28 00:10:05 +00:00
/// We will keep logs after and including this threshold.
2018-08-23 13:55:59 +00:00
UInt64 min_saved_log_pointer = std : : numeric_limits < UInt64 > : : max ( ) ;
2018-08-28 00:10:05 +00:00
2018-08-27 16:22:39 +00:00
UInt64 min_log_pointer_lost_candidate = std : : numeric_limits < UInt64 > : : max ( ) ;
2018-08-07 15:21:42 +00:00
Strings entries = zookeeper - > getChildren ( storage . zookeeper_path + " /log " ) ;
if ( entries . empty ( ) )
return ;
2022-01-30 19:49:48 +00:00
: : sort ( entries . begin ( ) , entries . end ( ) ) ;
2018-08-07 15:21:42 +00:00
2018-08-28 00:17:27 +00:00
String min_saved_record_log_str = entries [
2019-08-13 10:29:31 +00:00
entries . size ( ) > storage_settings - > max_replicated_logs_to_keep
? entries . size ( ) - storage_settings - > max_replicated_logs_to_keep
2018-08-28 00:17:27 +00:00
: 0 ] ;
2018-07-30 16:31:14 +00:00
2018-08-28 00:10:05 +00:00
/// Replicas that were marked is_lost but are active.
2018-08-27 12:09:22 +00:00
std : : unordered_set < String > recovering_replicas ;
2018-08-28 00:10:05 +00:00
/// Lost replica -> a version of 'host' node.
2018-08-27 15:44:51 +00:00
std : : unordered_map < String , UInt32 > host_versions_lost_replicas ;
2018-08-28 00:10:05 +00:00
2018-08-28 00:17:27 +00:00
/// Replica -> log pointer.
2018-08-27 15:44:51 +00:00
std : : unordered_map < String , String > log_pointers_candidate_lost_replicas ;
2018-08-23 15:58:29 +00:00
2018-08-27 15:44:51 +00:00
size_t num_replicas_were_marked_is_lost = 0 ;
2018-08-09 15:06:39 +00:00
2014-10-15 01:22:06 +00:00
for ( const String & replica : replicas )
{
2018-08-27 13:51:22 +00:00
Coordination : : Stat host_stat ;
2018-08-20 13:31:24 +00:00
zookeeper - > get ( storage . zookeeper_path + " /replicas/ " + replica + " /host " , & host_stat ) ;
2018-08-23 14:04:53 +00:00
String pointer = zookeeper - > get ( storage . zookeeper_path + " /replicas/ " + replica + " /log_pointer " ) ;
2018-08-07 15:21:42 +00:00
2019-01-09 15:44:20 +00:00
UInt64 log_pointer = 0 ;
2018-08-27 15:44:51 +00:00
if ( ! pointer . empty ( ) )
log_pointer = parse < UInt64 > ( pointer ) ;
2018-08-27 19:06:32 +00:00
2018-07-30 16:31:14 +00:00
/// Check status of replica (active or not).
2018-08-28 00:10:05 +00:00
/// If replica was not active, we could check when its log_pointer locates.
2018-08-27 12:09:22 +00:00
2018-08-28 00:22:32 +00:00
/// There can be three possibilities for "is_lost" node:
/// It doesn't exist: in old version of ClickHouse.
/// It exists and value is 0.
/// It exists and value is 1.
2018-08-27 15:44:51 +00:00
String is_lost_str ;
2018-08-27 12:09:22 +00:00
2018-08-28 00:10:05 +00:00
bool has_is_lost_node = zookeeper - > tryGet ( storage . zookeeper_path + " /replicas/ " + replica + " /is_lost " , is_lost_str ) ;
2018-08-27 12:09:22 +00:00
2018-08-23 13:55:59 +00:00
if ( zookeeper - > exists ( storage . zookeeper_path + " /replicas/ " + replica + " /is_active " ) )
2018-08-27 15:44:51 +00:00
{
2018-08-28 00:10:05 +00:00
if ( has_is_lost_node & & is_lost_str = = " 1 " )
2018-08-27 15:44:51 +00:00
{
2018-08-28 00:17:27 +00:00
/// Lost and active: recovering.
2018-08-27 12:09:22 +00:00
recovering_replicas . insert ( replica ) ;
2018-08-27 15:44:51 +00:00
+ + num_replicas_were_marked_is_lost ;
}
2018-08-27 12:09:22 +00:00
else
2018-08-28 00:17:27 +00:00
{
/// Not lost and active: usual case.
2018-08-27 12:09:22 +00:00
min_saved_log_pointer = std : : min ( min_saved_log_pointer , log_pointer ) ;
2018-08-28 00:17:27 +00:00
}
2018-08-27 15:44:51 +00:00
}
2018-07-30 16:31:14 +00:00
else
2018-08-07 15:21:42 +00:00
{
2018-08-28 00:10:05 +00:00
if ( ! has_is_lost_node )
2018-08-27 12:09:22 +00:00
{
/// Only to support old versions CH.
/// If replica did not have "/is_lost" we must save it's log_pointer.
/// Because old version CH can not work with recovering.
2018-08-23 13:55:59 +00:00
min_saved_log_pointer = std : : min ( min_saved_log_pointer , log_pointer ) ;
2018-08-27 12:09:22 +00:00
}
2018-08-23 13:55:59 +00:00
else
2018-08-27 15:44:51 +00:00
{
if ( is_lost_str = = " 0 " )
2018-08-22 14:01:54 +00:00
{
2018-08-28 00:17:27 +00:00
/// Not active and not lost: a candidate to be marked as lost.
2018-08-23 13:55:59 +00:00
String log_pointer_str = " log- " + padIndex ( log_pointer ) ;
if ( log_pointer_str > = min_saved_record_log_str )
2018-08-28 00:17:27 +00:00
{
/// Its log pointer is fresh enough.
2018-08-23 13:55:59 +00:00
min_saved_log_pointer = std : : min ( min_saved_log_pointer , log_pointer ) ;
2018-08-28 00:17:27 +00:00
}
2018-08-23 13:55:59 +00:00
else
{
2018-08-28 00:17:27 +00:00
/// Its log pointer is stale: will mark replica as lost.
2018-08-27 15:44:51 +00:00
host_versions_lost_replicas [ replica ] = host_stat . version ;
log_pointers_candidate_lost_replicas [ replica ] = log_pointer_str ;
2018-08-27 16:22:39 +00:00
min_log_pointer_lost_candidate = std : : min ( min_log_pointer_lost_candidate , log_pointer ) ;
2018-08-23 13:55:59 +00:00
}
2018-08-22 14:01:54 +00:00
}
2018-08-23 13:55:59 +00:00
else
2018-08-23 17:19:05 +00:00
{
2018-08-27 15:44:51 +00:00
+ + num_replicas_were_marked_is_lost ;
host_versions_lost_replicas [ replica ] = host_stat . version ;
2018-08-23 17:19:05 +00:00
}
2018-08-27 15:44:51 +00:00
}
2018-08-07 15:21:42 +00:00
}
2014-10-15 01:22:06 +00:00
}
2017-04-01 07:20:54 +00:00
2018-08-27 15:44:51 +00:00
/// We must check log_pointer of recovering replicas at the end.
/// Because log pointer of recovering replicas can move backward.
2018-08-27 12:09:22 +00:00
for ( const String & replica : recovering_replicas )
{
String pointer = zookeeper - > get ( storage . zookeeper_path + " /replicas/ " + replica + " /log_pointer " ) ;
2019-01-09 15:44:20 +00:00
UInt64 log_pointer = 0 ;
2018-08-27 15:44:51 +00:00
if ( ! pointer . empty ( ) )
log_pointer = parse < UInt64 > ( pointer ) ;
2018-08-27 12:09:22 +00:00
min_saved_log_pointer = std : : min ( min_saved_log_pointer , log_pointer ) ;
}
2018-08-27 13:51:22 +00:00
if ( ! recovering_replicas . empty ( ) )
2018-08-27 16:22:39 +00:00
min_saved_log_pointer = std : : min ( min_saved_log_pointer , min_log_pointer_lost_candidate ) ;
2018-08-27 12:09:22 +00:00
2018-08-07 15:21:42 +00:00
/// We will not touch the last `min_replicated_logs_to_keep` records.
2019-08-13 10:29:31 +00:00
entries . erase ( entries . end ( ) - std : : min < UInt64 > ( entries . size ( ) , storage_settings - > min_replicated_logs_to_keep ) , entries . end ( ) ) ;
2018-08-28 00:01:03 +00:00
/// We will not touch records that are no less than `min_saved_log_pointer`.
2018-08-23 13:55:59 +00:00
entries . erase ( std : : lower_bound ( entries . begin ( ) , entries . end ( ) , " log- " + padIndex ( min_saved_log_pointer ) ) , entries . end ( ) ) ;
2018-08-07 15:21:42 +00:00
2014-10-15 01:22:06 +00:00
if ( entries . empty ( ) )
return ;
2017-04-01 07:20:54 +00:00
2020-06-15 02:12:06 +00:00
markLostReplicas (
host_versions_lost_replicas ,
log_pointers_candidate_lost_replicas ,
replicas . size ( ) - num_replicas_were_marked_is_lost ,
zookeeper ) ;
2018-08-27 19:06:32 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2020-06-15 02:12:06 +00:00
size_t i = 0 ;
for ( ; i < entries . size ( ) ; + + i )
2014-10-15 01:22:06 +00:00
{
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( storage . zookeeper_path + " /log/ " + entries [ i ] , - 1 ) ) ;
2018-08-23 14:20:13 +00:00
2017-08-10 15:19:36 +00:00
if ( ops . size ( ) > 4 * zkutil : : MULTI_BATCH_SIZE | | i + 1 = = entries . size ( ) )
2014-10-15 01:22:06 +00:00
{
2018-08-27 15:44:51 +00:00
/// We need to check this because the replica that was restored from one of the marked replicas does not copy a non-valid log_pointer.
2018-08-27 23:59:49 +00:00
for ( const auto & host_version : host_versions_lost_replicas )
2018-08-27 15:44:51 +00:00
ops . emplace_back ( zkutil : : makeCheckRequest ( storage . zookeeper_path + " /replicas/ " + host_version . first + " /host " , host_version . second ) ) ;
2018-08-23 15:58:29 +00:00
2017-03-13 18:01:46 +00:00
/// Simultaneously with clearing the log, we check to see if replica was added since we received replicas list.
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeCheckRequest ( storage . zookeeper_path + " /replicas " , stat . version ) ) ;
2020-06-15 02:12:06 +00:00
2022-07-08 11:57:53 +00:00
Coordination : : Responses responses ;
Coordination : : Error e = zookeeper - > tryMulti ( ops , responses ) ;
if ( e = = Coordination : : Error : : ZNONODE )
2020-06-15 02:12:06 +00:00
{
2022-07-08 11:57:53 +00:00
/// Another replica already deleted the same node concurrently.
break ;
2020-06-15 02:12:06 +00:00
}
2022-07-08 11:57:53 +00:00
else
2020-06-15 02:12:06 +00:00
{
2022-07-08 11:57:53 +00:00
zkutil : : KeeperMultiException : : check ( e , ops , responses ) ;
2020-06-15 02:12:06 +00:00
}
2014-10-15 01:22:06 +00:00
ops . clear ( ) ;
}
}
2017-04-01 07:20:54 +00:00
2020-06-15 02:12:06 +00:00
if ( i ! = 0 )
LOG_DEBUG ( log , " Removed {} old log entries: {} - {} " , i , entries [ 0 ] , entries [ i - 1 ] ) ;
2014-10-15 01:22:06 +00:00
}
2017-08-10 15:19:36 +00:00
2018-08-27 15:44:51 +00:00
void ReplicatedMergeTreeCleanupThread : : markLostReplicas ( const std : : unordered_map < String , UInt32 > & host_versions_lost_replicas ,
const std : : unordered_map < String , String > & log_pointers_candidate_lost_replicas ,
2018-08-23 15:58:29 +00:00
size_t replicas_count , const zkutil : : ZooKeeperPtr & zookeeper )
2018-08-09 15:06:39 +00:00
{
2018-08-27 15:54:07 +00:00
Strings candidate_lost_replicas ;
2018-08-27 13:51:22 +00:00
std : : vector < Coordination : : Requests > requests ;
2018-08-09 15:06:39 +00:00
2018-08-27 16:22:39 +00:00
for ( const auto & pair : log_pointers_candidate_lost_replicas )
2018-08-09 15:06:39 +00:00
{
String replica = pair . first ;
2020-10-06 20:05:28 +00:00
LOG_WARNING ( log , " Will mark replica {} as lost, because it has stale log pointer: {} " , replica , pair . second ) ;
2018-08-27 13:51:22 +00:00
Coordination : : Requests ops ;
2018-08-23 15:58:29 +00:00
/// If host changed version we can not mark replicas, because replica started to be active.
2020-06-15 02:12:06 +00:00
ops . emplace_back ( zkutil : : makeCheckRequest (
storage . zookeeper_path + " /replicas/ " + replica + " /host " , host_versions_lost_replicas . at ( replica ) ) ) ;
ops . emplace_back ( zkutil : : makeSetRequest (
storage . zookeeper_path + " /replicas/ " + replica + " /is_lost " , " 1 " , - 1 ) ) ;
2018-08-27 15:44:51 +00:00
candidate_lost_replicas . push_back ( replica ) ;
requests . push_back ( ops ) ;
2018-08-09 15:06:39 +00:00
}
2018-08-27 19:06:32 +00:00
2018-08-27 15:44:51 +00:00
if ( candidate_lost_replicas . size ( ) = = replicas_count )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ALL_REPLICAS_LOST , " All replicas are stale: we won't mark any replica as lost " ) ;
2018-08-27 19:06:32 +00:00
2018-08-28 00:17:27 +00:00
std : : vector < zkutil : : ZooKeeper : : FutureMulti > futures ;
2018-08-27 15:54:07 +00:00
for ( size_t i = 0 ; i < candidate_lost_replicas . size ( ) ; + + i )
2021-06-01 07:30:06 +00:00
futures . emplace_back ( zookeeper - > asyncTryMultiNoThrow ( requests [ i ] ) ) ;
2018-08-09 15:06:39 +00:00
2018-08-27 15:54:07 +00:00
for ( size_t i = 0 ; i < candidate_lost_replicas . size ( ) ; + + i )
2018-08-20 13:31:24 +00:00
{
2018-08-28 00:23:38 +00:00
auto multi_responses = futures [ i ] . get ( ) ;
2018-08-27 13:51:22 +00:00
if ( multi_responses . responses [ 0 ] - > error = = Coordination : : Error : : ZBADVERSION )
2023-01-23 13:16:14 +00:00
throw Exception ( DB : : ErrorCodes : : REPLICA_STATUS_CHANGED , " {} became active when we marked lost replicas. " , candidate_lost_replicas [ i ] ) ;
2018-08-27 15:44:51 +00:00
zkutil : : KeeperMultiException : : check ( multi_responses . error , requests [ i ] , multi_responses . responses ) ;
2018-08-20 13:31:24 +00:00
}
2018-08-09 15:06:39 +00:00
}
2017-08-09 21:09:44 +00:00
struct ReplicatedMergeTreeCleanupThread : : NodeWithStat
{
String node ;
2017-11-15 16:32:47 +00:00
Int64 ctime = 0 ;
2021-04-19 09:52:37 +00:00
Int32 version = 0 ;
2017-08-09 21:09:44 +00:00
2021-04-19 09:52:37 +00:00
NodeWithStat ( String node_ , Int64 ctime_ , Int32 version_ ) : node ( std : : move ( node_ ) ) , ctime ( ctime_ ) , version ( version_ ) { }
2017-08-09 21:09:44 +00:00
2017-11-15 16:32:47 +00:00
static bool greaterByTime ( const NodeWithStat & lhs , const NodeWithStat & rhs )
2017-08-09 21:09:44 +00:00
{
2017-11-15 16:32:47 +00:00
return std : : forward_as_tuple ( lhs . ctime , lhs . node ) > std : : forward_as_tuple ( rhs . ctime , rhs . node ) ;
2017-08-09 21:09:44 +00:00
}
} ;
2022-12-30 15:30:33 +00:00
void ReplicatedMergeTreeCleanupThread : : clearOldBlocks ( const String & blocks_dir_name , UInt64 window_seconds , UInt64 window_size , NodeCTimeAndVersionCache & cached_block_stats )
2014-10-15 01:22:06 +00:00
{
2014-12-12 20:50:32 +00:00
auto zookeeper = storage . getZooKeeper ( ) ;
2017-08-09 21:09:44 +00:00
std : : vector < NodeWithStat > timed_blocks ;
2022-12-30 15:30:33 +00:00
getBlocksSortedByTime ( blocks_dir_name , * zookeeper , timed_blocks , cached_block_stats ) ;
2017-08-09 21:09:44 +00:00
if ( timed_blocks . empty ( ) )
return ;
/// Use ZooKeeper's first node (last according to time) timestamp as "current" time.
2017-11-15 16:32:47 +00:00
Int64 current_time = timed_blocks . front ( ) . ctime ;
2020-06-15 02:12:06 +00:00
Int64 time_threshold = std : : max (
static_cast < Int64 > ( 0 ) ,
2022-11-22 14:12:00 +00:00
current_time - static_cast < Int64 > ( 1000 * window_seconds ) ) ;
2017-08-10 15:19:36 +00:00
/// Virtual node, all nodes that are "greater" than this one will be deleted
2021-04-19 09:52:37 +00:00
NodeWithStat block_threshold { { } , time_threshold , 0 } ;
2017-08-09 21:09:44 +00:00
2022-11-22 14:12:00 +00:00
size_t current_deduplication_window = std : : min < size_t > ( timed_blocks . size ( ) , window_size ) ;
2017-08-09 21:09:44 +00:00
auto first_outdated_block_fixed_threshold = timed_blocks . begin ( ) + current_deduplication_window ;
2020-06-15 02:12:06 +00:00
auto first_outdated_block_time_threshold = std : : upper_bound (
timed_blocks . begin ( ) , timed_blocks . end ( ) , block_threshold , NodeWithStat : : greaterByTime ) ;
2017-08-09 21:09:44 +00:00
auto first_outdated_block = std : : min ( first_outdated_block_fixed_threshold , first_outdated_block_time_threshold ) ;
2021-04-15 18:34:53 +00:00
auto num_nodes_to_delete = timed_blocks . end ( ) - first_outdated_block ;
if ( ! num_nodes_to_delete )
return ;
auto last_outdated_block = timed_blocks . end ( ) - 1 ;
LOG_TRACE ( log , " Will clear {} old blocks from {} (ctime {}) to {} (ctime {}) " , num_nodes_to_delete ,
first_outdated_block - > node , first_outdated_block - > ctime ,
last_outdated_block - > node , last_outdated_block - > ctime ) ;
2018-08-25 01:58:14 +00:00
zkutil : : AsyncResponses < Coordination : : RemoveResponse > try_remove_futures ;
2017-08-09 21:09:44 +00:00
for ( auto it = first_outdated_block ; it ! = timed_blocks . end ( ) ; + + it )
{
2022-11-22 14:12:00 +00:00
String path = storage . zookeeper_path + " / " + blocks_dir_name + " / " + it - > node ;
2021-04-19 09:52:37 +00:00
try_remove_futures . emplace_back ( path , zookeeper - > asyncTryRemove ( path , it - > version ) ) ;
2017-08-09 21:09:44 +00:00
}
2017-11-15 16:32:47 +00:00
for ( auto & pair : try_remove_futures )
2017-08-09 21:09:44 +00:00
{
2017-11-15 16:32:47 +00:00
const String & path = pair . first ;
2020-06-12 15:09:12 +00:00
Coordination : : Error rc = pair . second . get ( ) . error ;
if ( rc = = Coordination : : Error : : ZNOTEMPTY )
2017-11-15 16:32:47 +00:00
{
2017-12-21 17:43:32 +00:00
/// Can happen if there are leftover block nodes with children created by previous server versions.
2017-11-15 16:32:47 +00:00
zookeeper - > removeRecursive ( path ) ;
2019-06-06 15:28:02 +00:00
cached_block_stats . erase ( first_outdated_block - > node ) ;
2017-11-15 16:32:47 +00:00
}
2021-04-19 10:27:32 +00:00
else if ( rc = = Coordination : : Error : : ZOK | | rc = = Coordination : : Error : : ZNONODE | | rc = = Coordination : : Error : : ZBADVERSION )
2019-06-06 15:28:02 +00:00
{
2020-06-15 02:12:06 +00:00
/// No node is Ok. Another replica is removing nodes concurrently.
2019-06-06 15:28:02 +00:00
/// Successfully removed blocks have to be removed from cache
cached_block_stats . erase ( first_outdated_block - > node ) ;
}
2020-06-15 02:12:06 +00:00
else
{
LOG_WARNING ( log , " Error while deleting ZooKeeper path `{}`: {}, ignoring. " , path , Coordination : : errorMessage ( rc ) ) ;
}
2019-06-06 15:28:02 +00:00
first_outdated_block + + ;
2017-08-09 21:09:44 +00:00
}
2021-04-15 18:34:53 +00:00
LOG_TRACE ( log , " Cleared {} old blocks from ZooKeeper " , num_nodes_to_delete ) ;
2017-08-09 21:09:44 +00:00
}
2022-12-30 15:30:33 +00:00
void ReplicatedMergeTreeCleanupThread : : getBlocksSortedByTime ( const String & blocks_dir_name , zkutil : : ZooKeeper & zookeeper , std : : vector < NodeWithStat > & timed_blocks , NodeCTimeAndVersionCache & cached_block_stats )
2017-08-09 21:09:44 +00:00
{
timed_blocks . clear ( ) ;
2017-08-06 21:40:38 +00:00
Strings blocks ;
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2022-12-27 19:12:09 +00:00
if ( Coordination : : Error : : ZOK ! = zookeeper . tryGetChildren ( storage . zookeeper_path + " / " + blocks_dir_name , blocks , & stat ) )
throw Exception ( ErrorCodes : : NOT_FOUND_NODE , " {}/{} doesn't exist " , storage . zookeeper_path , blocks_dir_name ) ;
2014-10-15 01:22:06 +00:00
2019-06-06 15:28:02 +00:00
/// Seems like this code is obsolete, because we delete blocks from cache
/// when they are deleted from zookeeper. But we don't know about all (maybe future) places in code
/// where they can be removed, so just to be sure that cache would not leak we check it here.
2017-08-06 21:40:38 +00:00
{
NameSet blocks_set ( blocks . begin ( ) , blocks . end ( ) ) ;
2017-11-15 16:32:47 +00:00
for ( auto it = cached_block_stats . begin ( ) ; it ! = cached_block_stats . end ( ) ; )
2017-08-06 21:40:38 +00:00
{
2022-04-18 10:18:43 +00:00
if ( ! blocks_set . contains ( it - > first ) )
2017-11-15 16:32:47 +00:00
it = cached_block_stats . erase ( it ) ;
2017-08-06 21:40:38 +00:00
else
+ + it ;
}
}
2014-10-15 01:22:06 +00:00
2017-11-15 16:32:47 +00:00
auto not_cached_blocks = stat . numChildren - cached_block_stats . size ( ) ;
2017-09-26 15:17:31 +00:00
if ( not_cached_blocks )
{
2023-01-02 10:59:44 +00:00
LOG_TRACE ( log , " Checking {} {} ({} are not cached){}, path is {} " , stat . numChildren , blocks_dir_name , not_cached_blocks , " to clear old ones from ZooKeeper. " , storage . zookeeper_path + " / " + blocks_dir_name ) ;
2017-09-26 15:17:31 +00:00
}
2014-10-15 01:22:06 +00:00
2022-10-11 09:27:46 +00:00
std : : vector < std : : string > exists_paths ;
2014-10-15 01:22:06 +00:00
for ( const String & block : blocks )
{
2017-11-15 16:32:47 +00:00
auto it = cached_block_stats . find ( block ) ;
if ( it = = cached_block_stats . end ( ) )
2017-08-06 21:40:38 +00:00
{
2017-11-15 16:32:47 +00:00
/// New block. Fetch its stat asynchronously.
2022-12-27 19:12:09 +00:00
exists_paths . emplace_back ( storage . zookeeper_path + " / " + blocks_dir_name + " / " + block ) ;
2017-08-06 21:40:38 +00:00
}
else
{
/// Cached block
2021-04-19 09:52:37 +00:00
const auto & ctime_and_version = it - > second ;
timed_blocks . emplace_back ( block , ctime_and_version . first , ctime_and_version . second ) ;
2017-08-06 21:40:38 +00:00
}
2014-10-15 01:22:06 +00:00
}
2022-10-11 09:27:46 +00:00
auto exists_size = exists_paths . size ( ) ;
auto exists_results = zookeeper . exists ( exists_paths ) ;
2017-08-09 21:09:44 +00:00
/// Put fetched stats into the cache
2022-10-11 09:27:46 +00:00
for ( size_t i = 0 ; i < exists_size ; + + i )
2017-08-09 21:09:44 +00:00
{
2022-10-11 09:27:46 +00:00
auto status = exists_results [ i ] ;
2020-06-12 15:09:12 +00:00
if ( status . error ! = Coordination : : Error : : ZNONODE )
2017-11-15 16:32:47 +00:00
{
2022-10-11 09:27:46 +00:00
auto node_name = fs : : path ( exists_paths [ i ] ) . filename ( ) ;
cached_block_stats . emplace ( node_name , std : : make_pair ( status . stat . ctime , status . stat . version ) ) ;
timed_blocks . emplace_back ( node_name , status . stat . ctime , status . stat . version ) ;
2017-11-15 16:32:47 +00:00
}
2017-08-09 21:09:44 +00:00
}
2017-07-24 20:12:59 +00:00
2022-01-30 19:49:48 +00:00
: : sort ( timed_blocks . begin ( ) , timed_blocks . end ( ) , NodeWithStat : : greaterByTime ) ;
2017-08-09 21:09:44 +00:00
}
2017-07-24 20:12:59 +00:00
2018-07-31 11:36:08 +00:00
void ReplicatedMergeTreeCleanupThread : : clearOldMutations ( )
{
2019-08-26 14:24:29 +00:00
auto storage_settings = storage . getSettings ( ) ;
2019-08-13 10:29:31 +00:00
if ( ! storage_settings - > finished_mutations_to_keep )
2018-07-31 11:36:08 +00:00
return ;
2019-08-13 10:29:31 +00:00
if ( storage . queue . countFinishedMutations ( ) < = storage_settings - > finished_mutations_to_keep )
2018-07-31 11:36:08 +00:00
{
/// Not strictly necessary, but helps to avoid unnecessary ZooKeeper requests.
/// If even this replica hasn't finished enough mutations yet, then we don't need to clean anything.
return ;
}
auto zookeeper = storage . getZooKeeper ( ) ;
2018-08-25 01:58:14 +00:00
Coordination : : Stat replicas_stat ;
2018-07-31 11:36:08 +00:00
Strings replicas = zookeeper - > getChildren ( storage . zookeeper_path + " /replicas " , & replicas_stat ) ;
UInt64 min_pointer = std : : numeric_limits < UInt64 > : : max ( ) ;
for ( const String & replica : replicas )
{
String pointer ;
2022-04-25 16:00:39 +00:00
// No Need to check return value to delete mutations.
2018-07-31 11:36:08 +00:00
zookeeper - > tryGet ( storage . zookeeper_path + " /replicas/ " + replica + " /mutation_pointer " , pointer ) ;
2022-04-25 16:00:39 +00:00
if ( pointer . empty ( ) )
return ; /// One replica hasn't done anything yet so we can't delete any mutations.
min_pointer = std : : min ( parse < UInt64 > ( pointer ) , min_pointer ) ;
2018-07-31 11:36:08 +00:00
}
Strings entries = zookeeper - > getChildren ( storage . zookeeper_path + " /mutations " ) ;
2022-01-30 19:49:48 +00:00
: : sort ( entries . begin ( ) , entries . end ( ) ) ;
2018-07-31 11:36:08 +00:00
/// Do not remove entries that are greater than `min_pointer` (they are not done yet).
entries . erase ( std : : upper_bound ( entries . begin ( ) , entries . end ( ) , padIndex ( min_pointer ) ) , entries . end ( ) ) ;
2019-08-13 10:29:31 +00:00
/// Do not remove last `storage_settings->finished_mutations_to_keep` entries.
if ( entries . size ( ) < = storage_settings - > finished_mutations_to_keep )
2018-07-31 11:36:08 +00:00
return ;
2019-08-13 10:29:31 +00:00
entries . erase ( entries . end ( ) - storage_settings - > finished_mutations_to_keep , entries . end ( ) ) ;
2018-07-31 11:36:08 +00:00
if ( entries . empty ( ) )
return ;
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2018-07-31 11:36:08 +00:00
size_t batch_start_i = 0 ;
for ( size_t i = 0 ; i < entries . size ( ) ; + + i )
{
ops . emplace_back ( zkutil : : makeRemoveRequest ( storage . zookeeper_path + " /mutations/ " + entries [ i ] , - 1 ) ) ;
if ( ops . size ( ) > 4 * zkutil : : MULTI_BATCH_SIZE | | i + 1 = = entries . size ( ) )
{
/// Simultaneously with clearing the log, we check to see if replica was added since we received replicas list.
ops . emplace_back ( zkutil : : makeCheckRequest ( storage . zookeeper_path + " /replicas " , replicas_stat . version ) ) ;
2020-06-15 02:12:06 +00:00
try
{
zookeeper - > multi ( ops ) ;
}
catch ( const zkutil : : KeeperMultiException & e )
{
/// Another replica already deleted the same node concurrently.
if ( e . code = = Coordination : : Error : : ZNONODE )
break ;
throw ;
}
LOG_DEBUG ( log , " Removed {} old mutation entries: {} - {} " ,
i + 1 - batch_start_i , entries [ batch_start_i ] , entries [ i ] ) ;
2018-07-31 11:36:08 +00:00
batch_start_i = i + 1 ;
ops . clear ( ) ;
}
}
}
2014-10-15 01:22:06 +00:00
}