2020-04-15 20:28:05 +00:00
# include <Core/Defines.h>
2021-06-20 08:24:43 +00:00
# include "Common/hex.h"
2019-05-03 02:00:57 +00:00
# include <Common/Macros.h>
# include <Common/StringUtils/StringUtils.h>
# include <Common/ThreadPool.h>
2019-11-27 09:39:44 +00:00
# include <Common/ZooKeeper/KeeperException.h>
# include <Common/ZooKeeper/Types.h>
# include <Common/escapeForFileName.h>
# include <Common/formatReadable.h>
# include <Common/thread_local_rng.h>
# include <Common/typeid_cast.h>
2017-04-01 09:19:00 +00:00
2018-12-25 23:15:28 +00:00
# include <Storages/AlterCommands.h>
2018-12-25 23:18:07 +00:00
# include <Storages/PartitionCommands.h>
2017-04-01 09:19:00 +00:00
# include <Storages/ColumnsDescription.h>
# include <Storages/StorageReplicatedMergeTree.h>
2019-10-10 16:30:30 +00:00
# include <Storages/MergeTree/IMergeTreeDataPart.h>
2018-12-11 13:30:20 +00:00
# include <Storages/MergeTree/MergeList.h>
2021-09-06 12:01:16 +00:00
# include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
2021-06-29 15:14:44 +00:00
# include <Storages/MergeTree/MergedBlockOutputStream.h>
2020-11-24 14:24:48 +00:00
# include <Storages/MergeTree/PinnedPartUUIDs.h>
2020-10-22 06:18:10 +00:00
# include <Storages/MergeTree/PartitionPruner.h>
2018-11-01 13:30:38 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
2021-07-26 16:48:25 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeSink.h>
2017-04-01 09:19:00 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
2018-04-19 10:33:16 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h>
2017-04-01 09:19:00 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
2018-09-19 11:08:04 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h>
2018-12-11 13:30:20 +00:00
# include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
2021-09-16 21:19:58 +00:00
# include <Storages/MergeTree/MergeFromLogEntryTask.h>
# include <Storages/MergeTree/MutateFromLogEntryTask.h>
2019-05-03 02:00:57 +00:00
# include <Storages/VirtualColumnUtils.h>
2021-04-04 09:23:40 +00:00
# include <Storages/MergeTree/MergeTreeReaderCompact.h>
2017-04-01 09:19:00 +00:00
2020-04-15 20:28:05 +00:00
2017-04-01 09:19:00 +00:00
# include <Databases/IDatabase.h>
2020-11-24 10:24:39 +00:00
# include <Databases/DatabaseOnDisk.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/formatAST.h>
2018-10-13 20:58:04 +00:00
# include <Parsers/ASTDropQuery.h>
2017-06-22 15:01:08 +00:00
# include <Parsers/ASTOptimizeQuery.h>
2017-09-06 20:34:26 +00:00
# include <Parsers/ASTLiteral.h>
2019-05-03 02:00:57 +00:00
# include <Parsers/queryToString.h>
2019-07-03 13:17:19 +00:00
# include <Parsers/ASTCheckQuery.h>
2019-12-26 18:17:05 +00:00
# include <Parsers/ASTSetQuery.h>
2017-04-01 09:19:00 +00:00
2021-09-08 18:29:38 +00:00
# include <Processors/QueryPlan/QueryPlan.h>
2021-03-04 17:38:12 +00:00
# include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
# include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2017-04-01 09:19:00 +00:00
# include <IO/ReadBufferFromString.h>
# include <IO/Operators.h>
2017-12-27 17:58:52 +00:00
# include <IO/ConnectionTimeouts.h>
2020-12-10 22:05:02 +00:00
# include <IO/ConnectionTimeoutsContext.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/InterpreterAlterQuery.h>
# include <Interpreters/PartLog.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2020-11-20 16:06:27 +00:00
# include <Interpreters/DDLTask.h>
2021-04-07 13:52:11 +00:00
# include <Interpreters/InterserverCredentials.h>
2017-04-01 09:19:00 +00:00
2015-04-16 06:12:35 +00:00
# include <Poco/DirectoryIterator.h>
2014-03-21 13:42:14 +00:00
2021-10-02 07:13:14 +00:00
# include <base/range.h>
# include <base/scope_guard.h>
# include <base/scope_guard_safe.h>
2016-08-10 07:20:21 +00:00
2021-11-18 18:07:35 +00:00
# include <boost/algorithm/string/join.hpp>
# include <boost/algorithm/string/replace.hpp>
2021-06-20 08:24:43 +00:00
# include <algorithm>
2016-01-28 01:00:27 +00:00
# include <ctime>
2021-06-20 08:24:43 +00:00
# include <filesystem>
# include <iterator>
# include <numeric>
2016-01-28 01:00:27 +00:00
# include <thread>
# include <future>
2021-05-08 10:59:55 +00:00
namespace fs = std : : filesystem ;
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2017-04-01 07:20:54 +00:00
extern const Event ReplicatedPartFailedFetches ;
extern const Event ReplicatedPartFetchesOfMerged ;
extern const Event ObsoleteReplicatedParts ;
extern const Event ReplicatedPartFetches ;
2020-06-12 20:38:43 +00:00
extern const Event CreatedLogEntryForMerge ;
extern const Event NotCreatedLogEntryForMerge ;
extern const Event CreatedLogEntryForMutation ;
extern const Event NotCreatedLogEntryForMutation ;
2018-04-06 16:06:07 +00:00
}
2020-10-26 11:02:47 +00:00
namespace CurrentMetrics
{
extern const Metric BackgroundFetchesPoolTask ;
}
2018-04-06 16:06:07 +00:00
2014-03-21 13:42:14 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int CANNOT_READ_ALL_DATA ;
extern const int NOT_IMPLEMENTED ;
2017-04-01 07:20:54 +00:00
extern const int NO_ZOOKEEPER ;
extern const int INCORRECT_DATA ;
extern const int INCOMPATIBLE_COLUMNS ;
extern const int REPLICA_IS_ALREADY_EXIST ;
extern const int NO_REPLICA_HAS_PART ;
extern const int LOGICAL_ERROR ;
extern const int TOO_MANY_UNEXPECTED_DATA_PARTS ;
extern const int ABORTED ;
extern const int REPLICA_IS_NOT_IN_QUORUM ;
extern const int TABLE_IS_READ_ONLY ;
extern const int NOT_FOUND_NODE ;
extern const int NO_ACTIVE_REPLICAS ;
2020-06-19 14:18:58 +00:00
extern const int NOT_A_LEADER ;
2017-04-01 07:20:54 +00:00
extern const int TABLE_WAS_NOT_DROPPED ;
extern const int PARTITION_ALREADY_EXISTS ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS ;
2017-04-01 07:20:54 +00:00
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER ;
extern const int PARTITION_DOESNT_EXIST ;
extern const int UNFINISHED ;
2017-04-06 13:03:23 +00:00
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS ;
2017-11-20 19:33:12 +00:00
extern const int PART_IS_TEMPORARILY_LOCKED ;
2018-01-12 17:30:21 +00:00
extern const int CANNOT_ASSIGN_OPTIMIZE ;
2018-05-21 13:49:54 +00:00
extern const int KEEPER_EXCEPTION ;
2018-08-20 13:31:24 +00:00
extern const int ALL_REPLICAS_LOST ;
2018-08-23 13:55:59 +00:00
extern const int REPLICA_STATUS_CHANGED ;
2020-02-13 14:48:38 +00:00
extern const int CANNOT_ASSIGN_ALTER ;
2020-08-28 00:53:22 +00:00
extern const int DIRECTORY_ALREADY_EXISTS ;
2020-08-27 14:19:18 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT ;
2020-10-16 11:58:47 +00:00
extern const int UNKNOWN_POLICY ;
2020-11-03 09:24:10 +00:00
extern const int NO_SUCH_DATA_PART ;
2020-12-25 13:38:04 +00:00
extern const int INTERSERVER_SCHEME_DOESNT_MATCH ;
2021-04-14 09:11:59 +00:00
extern const int DUPLICATE_DATA_PART ;
2020-11-24 14:24:48 +00:00
extern const int BAD_ARGUMENTS ;
2021-06-20 08:24:43 +00:00
extern const int CONCURRENT_ACCESS_NOT_SUPPORTED ;
2021-08-18 09:49:22 +00:00
extern const int CHECKSUM_DOESNT_MATCH ;
2018-05-21 13:49:54 +00:00
}
namespace ActionLocks
{
extern const StorageActionBlockType PartsMerge ;
extern const StorageActionBlockType PartsFetch ;
extern const StorageActionBlockType PartsSend ;
extern const StorageActionBlockType ReplicationQueue ;
2019-08-01 15:36:12 +00:00
extern const StorageActionBlockType PartsTTLMerge ;
2019-09-03 14:50:49 +00:00
extern const StorageActionBlockType PartsMove ;
2016-01-11 21:46:36 +00:00
}
2014-04-03 11:48:28 +00:00
2020-04-27 16:19:04 +00:00
static const auto QUEUE_UPDATE_ERROR_SLEEP_MS = 1 * 1000 ;
static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000 ;
static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000 ;
2014-04-03 11:48:28 +00:00
2020-11-16 08:27:33 +00:00
void StorageReplicatedMergeTree : : setZooKeeper ( )
2016-01-17 08:12:48 +00:00
{
2021-02-27 08:07:14 +00:00
/// Every ReplicatedMergeTree table is using only one ZooKeeper session.
/// But if several ReplicatedMergeTree tables are using different
/// ZooKeeper sessions, some queries like ATTACH PARTITION FROM may have
/// strange effects. So we always use only one session for all tables.
/// (excluding auxiliary zookeepers)
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( current_zookeeper_mutex ) ;
2020-11-16 08:27:33 +00:00
if ( zookeeper_name = = default_zookeeper_name )
{
2021-04-10 23:33:54 +00:00
current_zookeeper = getContext ( ) - > getZooKeeper ( ) ;
2020-11-16 08:27:33 +00:00
}
else
{
2021-04-10 23:33:54 +00:00
current_zookeeper = getContext ( ) - > getAuxiliaryZooKeeper ( zookeeper_name ) ;
2020-11-16 08:27:33 +00:00
}
2016-01-17 08:12:48 +00:00
}
2019-10-28 17:27:43 +00:00
zkutil : : ZooKeeperPtr StorageReplicatedMergeTree : : tryGetZooKeeper ( ) const
2016-01-17 08:12:48 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( current_zookeeper_mutex ) ;
2017-04-01 07:20:54 +00:00
return current_zookeeper ;
2016-01-17 08:12:48 +00:00
}
2019-10-28 17:27:43 +00:00
zkutil : : ZooKeeperPtr StorageReplicatedMergeTree : : getZooKeeper ( ) const
2016-01-17 08:12:48 +00:00
{
2017-04-01 07:20:54 +00:00
auto res = tryGetZooKeeper ( ) ;
if ( ! res )
throw Exception ( " Cannot get ZooKeeper " , ErrorCodes : : NO_ZOOKEEPER ) ;
return res ;
2016-01-17 08:12:48 +00:00
}
2021-10-28 16:19:41 +00:00
static std : : string normalizeZooKeeperPath ( std : : string zookeeper_path , bool check_starts_with_slash , Poco : : Logger * log = nullptr )
2020-07-09 23:45:29 +00:00
{
if ( ! zookeeper_path . empty ( ) & & zookeeper_path . back ( ) = = ' / ' )
zookeeper_path . resize ( zookeeper_path . size ( ) - 1 ) ;
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if ( ! zookeeper_path . empty ( ) & & zookeeper_path . front ( ) ! = ' / ' )
2021-10-28 16:19:41 +00:00
{
/// Do not allow this for new tables, print warning for tables created in old versions
if ( check_starts_with_slash )
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " ZooKeeper path must starts with '/', got '{}' " , zookeeper_path ) ;
if ( log )
LOG_WARNING ( log , " ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases " ) ;
2020-07-09 23:45:29 +00:00
zookeeper_path = " / " + zookeeper_path ;
2021-10-28 16:19:41 +00:00
}
2020-07-09 23:45:29 +00:00
return zookeeper_path ;
}
2020-11-17 12:27:19 +00:00
static String extractZooKeeperName ( const String & path )
2020-11-16 12:30:54 +00:00
{
2021-10-28 16:19:41 +00:00
static constexpr auto default_zookeeper_name = " default " ;
2020-11-16 12:30:54 +00:00
if ( path . empty ( ) )
2021-10-28 16:19:41 +00:00
throw Exception ( " ZooKeeper path should not be empty " , ErrorCodes : : BAD_ARGUMENTS ) ;
if ( path [ 0 ] = = ' / ' )
return default_zookeeper_name ;
auto pos = path . find ( " :/ " ) ;
if ( pos ! = String : : npos & & pos < path . find ( ' / ' ) )
2020-11-16 12:30:54 +00:00
{
2020-11-17 12:45:21 +00:00
auto zookeeper_name = path . substr ( 0 , pos ) ;
if ( zookeeper_name . empty ( ) )
2021-10-28 16:19:41 +00:00
throw Exception ( " Zookeeper path should start with '/' or '<auxiliary_zookeeper_name>:/' " , ErrorCodes : : BAD_ARGUMENTS ) ;
2020-11-17 12:45:21 +00:00
return zookeeper_name ;
2020-11-16 12:30:54 +00:00
}
2020-11-17 12:27:19 +00:00
return default_zookeeper_name ;
}
2021-10-28 16:19:41 +00:00
static String extractZooKeeperPath ( const String & path , bool check_starts_with_slash , Poco : : Logger * log = nullptr )
2020-11-17 12:27:19 +00:00
{
if ( path . empty ( ) )
2021-10-28 16:19:41 +00:00
throw Exception ( " ZooKeeper path should not be empty " , ErrorCodes : : BAD_ARGUMENTS ) ;
if ( path [ 0 ] = = ' / ' )
2021-10-28 17:25:31 +00:00
return normalizeZooKeeperPath ( path , check_starts_with_slash , log ) ;
2021-10-28 16:19:41 +00:00
auto pos = path . find ( " :/ " ) ;
if ( pos ! = String : : npos & & pos < path . find ( ' / ' ) )
2020-11-16 12:30:54 +00:00
{
2021-10-28 16:19:41 +00:00
return normalizeZooKeeperPath ( path . substr ( pos + 1 , String : : npos ) , check_starts_with_slash , log ) ;
2020-11-16 12:30:54 +00:00
}
2021-10-28 16:19:41 +00:00
return normalizeZooKeeperPath ( path , check_starts_with_slash , log ) ;
2020-11-16 12:30:54 +00:00
}
2020-07-09 23:45:29 +00:00
2021-05-13 11:29:59 +00:00
static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom ( const String & partition_id )
{
/// NOTE We don't have special log entry type for MOVE PARTITION/ATTACH PARTITION FROM,
/// so we use REPLACE_RANGE with dummy range of one block, which means "attach, not replace".
/// It's safe to fill drop range for MOVE PARTITION/ATTACH PARTITION FROM with zeros,
/// because drop range for REPLACE PARTITION must contain at least 2 blocks,
/// so we can distinguish dummy drop range from any real or virtual part.
/// But we should never construct such part name, even for virtual part,
/// because it can be confused with real part <partition>_0_0_0.
/// TODO get rid of this.
MergeTreePartInfo drop_range ;
drop_range . partition_id = partition_id ;
drop_range . min_block = 0 ;
drop_range . max_block = 0 ;
drop_range . level = 0 ;
drop_range . mutation = 0 ;
return drop_range ;
}
2014-03-21 13:42:14 +00:00
StorageReplicatedMergeTree : : StorageReplicatedMergeTree (
2017-04-01 07:20:54 +00:00
const String & zookeeper_path_ ,
const String & replica_name_ ,
bool attach ,
2019-12-04 16:06:55 +00:00
const StorageID & table_id_ ,
2019-10-28 20:12:14 +00:00
const String & relative_data_path_ ,
2020-06-08 18:23:26 +00:00
const StorageInMemoryMetadata & metadata_ ,
2021-05-31 14:49:02 +00:00
ContextMutablePtr context_ ,
2017-09-08 18:11:09 +00:00
const String & date_column_name ,
2019-05-03 02:00:57 +00:00
const MergingParams & merging_params_ ,
2019-08-26 14:24:29 +00:00
std : : unique_ptr < MergeTreeSettings > settings_ ,
2020-09-26 19:18:28 +00:00
bool has_force_restore_data_flag ,
bool allow_renaming_ )
2019-12-30 11:08:09 +00:00
: MergeTreeData ( table_id_ ,
relative_data_path_ ,
2020-06-08 18:23:26 +00:00
metadata_ ,
2019-12-30 11:08:09 +00:00
context_ ,
date_column_name ,
merging_params_ ,
std : : move ( settings_ ) ,
true , /// require_part_metadata
attach ,
[ this ] ( const std : : string & name ) { enqueuePartForCheck ( name ) ; } )
2020-11-17 12:45:21 +00:00
, zookeeper_name ( extractZooKeeperName ( zookeeper_path_ ) )
2021-10-28 16:19:41 +00:00
, zookeeper_path ( extractZooKeeperPath ( zookeeper_path_ , /* check_starts_with_slash */ ! attach , log ) )
2020-07-10 09:19:32 +00:00
, replica_name ( replica_name_ )
2021-05-09 11:59:49 +00:00
, replica_path ( fs : : path ( zookeeper_path ) / " replicas " / replica_name_ )
2019-12-30 11:08:09 +00:00
, reader ( * this )
, writer ( * this )
2021-09-30 21:26:24 +00:00
, merger_mutator ( * this ,
getContext ( ) - > getSettingsRef ( ) . background_merges_mutations_concurrency_ratio *
getContext ( ) - > getSettingsRef ( ) . background_pool_size )
2020-09-18 10:57:33 +00:00
, merge_strategy_picker ( * this )
, queue ( * this , merge_strategy_picker )
2019-12-30 11:08:09 +00:00
, fetcher ( * this )
, cleanup_thread ( * this )
, part_check_thread ( * this )
, restarting_thread ( * this )
2020-11-24 14:24:48 +00:00
, part_moves_between_shards_orchestrator ( * this )
2020-09-26 19:18:28 +00:00
, allow_renaming ( allow_renaming_ )
2021-04-10 23:33:54 +00:00
, replicated_fetches_pool_size ( getContext ( ) - > getSettingsRef ( ) . background_fetches_pool_size )
2021-05-26 20:37:44 +00:00
, replicated_fetches_throttler ( std : : make_shared < Throttler > ( getSettings ( ) - > max_replicated_fetches_network_bandwidth , getContext ( ) - > getReplicatedFetchesThrottler ( ) ) )
, replicated_sends_throttler ( std : : make_shared < Throttler > ( getSettings ( ) - > max_replicated_sends_network_bandwidth , getContext ( ) - > getReplicatedSendsThrottler ( ) ) )
2014-03-21 13:42:14 +00:00
{
2021-04-10 23:33:54 +00:00
queue_updating_task = getContext ( ) - > getSchedulePool ( ) . createTask (
2020-06-11 03:24:52 +00:00
getStorageID ( ) . getFullTableName ( ) + " (StorageReplicatedMergeTree::queueUpdatingTask) " , [ this ] { queueUpdatingTask ( ) ; } ) ;
2018-05-31 13:05:05 +00:00
2021-04-10 23:33:54 +00:00
mutations_updating_task = getContext ( ) - > getSchedulePool ( ) . createTask (
2020-06-11 03:24:52 +00:00
getStorageID ( ) . getFullTableName ( ) + " (StorageReplicatedMergeTree::mutationsUpdatingTask) " , [ this ] { mutationsUpdatingTask ( ) ; } ) ;
2018-05-31 13:05:05 +00:00
2021-04-10 23:33:54 +00:00
merge_selecting_task = getContext ( ) - > getSchedulePool ( ) . createTask (
2020-06-11 03:24:52 +00:00
getStorageID ( ) . getFullTableName ( ) + " (StorageReplicatedMergeTree::mergeSelectingTask) " , [ this ] { mergeSelectingTask ( ) ; } ) ;
2020-07-09 23:45:29 +00:00
2018-05-31 13:05:05 +00:00
/// Will be activated if we win leader election.
merge_selecting_task - > deactivate ( ) ;
2021-04-10 23:33:54 +00:00
mutations_finalizing_task = getContext ( ) - > getSchedulePool ( ) . createTask (
2020-06-11 03:24:52 +00:00
getStorageID ( ) . getFullTableName ( ) + " (StorageReplicatedMergeTree::mutationsFinalizingTask) " , [ this ] { mutationsFinalizingTask ( ) ; } ) ;
2018-06-21 13:27:36 +00:00
2021-04-10 23:33:54 +00:00
if ( getContext ( ) - > hasZooKeeper ( ) | | getContext ( ) - > hasAuxiliaryZooKeeper ( zookeeper_name ) )
2020-09-07 23:01:49 +00:00
{
/// It's possible for getZooKeeper() to timeout if zookeeper host(s) can't
/// be reached. In such cases Poco::Exception is thrown after a connection
/// timeout - refer to src/Common/ZooKeeper/ZooKeeperImpl.cpp:866 for more info.
///
/// Side effect of this is that the CreateQuery gets interrupted and it exits.
/// But the data Directories for the tables being created aren't cleaned up.
/// This unclean state will hinder table creation on any retries and will
/// complain that the Directory for table already exists.
///
2020-09-10 16:09:48 +00:00
/// To achieve a clean state on failed table creations, catch this error and
/// call dropIfEmpty() method only if the operation isn't ATTACH then proceed
/// throwing the exception. Without this, the Directory for the tables need
/// to be manually deleted before retrying the CreateQuery.
2020-09-07 23:01:49 +00:00
try
{
2020-11-16 08:27:33 +00:00
if ( zookeeper_name = = default_zookeeper_name )
{
2021-04-10 23:33:54 +00:00
current_zookeeper = getContext ( ) - > getZooKeeper ( ) ;
2020-11-16 08:27:33 +00:00
}
else
{
2021-04-10 23:33:54 +00:00
current_zookeeper = getContext ( ) - > getAuxiliaryZooKeeper ( zookeeper_name ) ;
2020-11-16 08:27:33 +00:00
}
2020-09-07 23:01:49 +00:00
}
2020-09-10 10:21:13 +00:00
catch ( . . . )
2020-09-07 23:01:49 +00:00
{
2020-09-10 10:21:13 +00:00
if ( ! attach )
dropIfEmpty ( ) ;
throw ;
2020-09-07 23:01:49 +00:00
}
}
2018-04-21 18:41:06 +00:00
2017-04-01 07:20:54 +00:00
bool skip_sanity_checks = false ;
2018-04-09 15:49:12 +00:00
if ( current_zookeeper & & current_zookeeper - > exists ( replica_path + " /flags/force_restore_data " ) )
2017-04-01 07:20:54 +00:00
{
2018-04-09 15:49:12 +00:00
skip_sanity_checks = true ;
current_zookeeper - > remove ( replica_path + " /flags/force_restore_data " ) ;
2017-04-01 07:20:54 +00:00
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Skipping the limits on severity of changes to data parts and columns (flag {}/flags/force_restore_data). " , replica_path ) ;
2017-04-01 07:20:54 +00:00
}
2018-04-09 15:49:12 +00:00
else if ( has_force_restore_data_flag )
2017-04-01 07:20:54 +00:00
{
2018-04-09 15:49:12 +00:00
skip_sanity_checks = true ;
LOG_WARNING ( log , " Skipping the limits on severity of changes to data parts and columns (flag force_restore_data). " ) ;
2017-04-01 07:20:54 +00:00
}
2019-05-03 02:00:57 +00:00
loadDataParts ( skip_sanity_checks ) ;
2017-04-01 07:20:54 +00:00
if ( ! current_zookeeper )
{
if ( ! attach )
2020-09-10 10:21:13 +00:00
{
dropIfEmpty ( ) ;
2017-04-01 07:20:54 +00:00
throw Exception ( " Can't create replicated table without ZooKeeper " , ErrorCodes : : NO_ZOOKEEPER ) ;
2020-09-10 10:21:13 +00:00
}
2017-04-01 07:20:54 +00:00
/// Do not activate the replica. It will be readonly.
LOG_ERROR ( log , " No ZooKeeper: table will be in readonly mode. " ) ;
is_readonly = true ;
return ;
}
2019-03-14 11:49:44 +00:00
if ( attach & & ! current_zookeeper - > exists ( zookeeper_path + " /metadata " ) )
{
2021-06-20 08:24:43 +00:00
LOG_WARNING ( log , " No metadata in ZooKeeper for {}: table will be in readonly mode. " , zookeeper_path ) ;
2019-03-14 11:49:44 +00:00
is_readonly = true ;
2020-06-14 01:23:53 +00:00
has_metadata_in_zookeeper = false ;
2019-03-14 11:49:44 +00:00
return ;
}
2020-06-16 16:55:04 +00:00
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2021-06-20 08:24:43 +00:00
/// May it be ZK lost not the whole root, so the upper check passed, but only the /replicas/replica
/// folder.
if ( attach & & ! current_zookeeper - > exists ( replica_path ) )
{
LOG_WARNING ( log , " No metadata in ZooKeeper for {}: table will be in readonly mode " , replica_path ) ;
is_readonly = true ;
has_metadata_in_zookeeper = false ;
return ;
}
2017-04-01 07:20:54 +00:00
if ( ! attach )
{
2019-05-03 02:00:57 +00:00
if ( ! getDataParts ( ) . empty ( ) )
2021-06-20 08:24:43 +00:00
throw Exception ( " Data directory for table already contains data parts "
2020-08-01 22:06:09 +00:00
" - probably it was unclean DROP table or manual intervention. "
" You must either clear directory by hand or use ATTACH TABLE "
" instead of CREATE TABLE if you need to use that parts. " , ErrorCodes : : INCORRECT_DATA ) ;
2017-04-01 07:20:54 +00:00
2020-06-11 03:24:52 +00:00
try
{
2020-06-16 16:55:04 +00:00
bool is_first_replica = createTableIfNotExists ( metadata_snapshot ) ;
2017-04-01 07:20:54 +00:00
2020-08-15 12:30:17 +00:00
try
{
/// NOTE If it's the first replica, these requests to ZooKeeper look redundant, we already know everything.
2020-04-08 09:51:04 +00:00
2020-08-15 12:30:17 +00:00
/// We have to check granularity on other replicas. If it's fixed we
/// must create our new replica with fixed granularity and store this
/// information in /replica/metadata.
2021-06-28 17:02:22 +00:00
other_replicas_fixed_granularity = checkFixedGranularityInZookeeper ( ) ;
2020-02-14 10:17:04 +00:00
2020-08-15 12:30:17 +00:00
checkTableStructure ( zookeeper_path , metadata_snapshot ) ;
Coordination : : Stat metadata_stat ;
current_zookeeper - > get ( zookeeper_path + " /metadata " , & metadata_stat ) ;
metadata_version = metadata_stat . version ;
}
catch ( Coordination : : Exception & e )
{
if ( ! is_first_replica & & e . code = = Coordination : : Error : : ZNONODE )
throw Exception ( " Table " + zookeeper_path + " was suddenly removed. " , ErrorCodes : : ALL_REPLICAS_LOST ) ;
else
throw ;
}
2020-02-14 10:34:54 +00:00
2020-06-11 03:24:52 +00:00
if ( ! is_first_replica )
2020-06-16 16:55:04 +00:00
createReplica ( metadata_snapshot ) ;
2020-06-11 03:24:52 +00:00
}
catch ( . . . )
{
/// If replica was not created, rollback creation of data directory.
dropIfEmpty ( ) ;
throw ;
}
2017-04-01 07:20:54 +00:00
}
else
{
2020-04-07 15:55:23 +00:00
/// In old tables this node may missing or be empty
2020-04-07 15:28:29 +00:00
String replica_metadata ;
2021-06-20 08:24:43 +00:00
const bool replica_metadata_exists = current_zookeeper - > tryGet ( replica_path + " /metadata " , replica_metadata ) ;
2020-04-07 15:28:29 +00:00
if ( ! replica_metadata_exists | | replica_metadata . empty ( ) )
{
2020-04-08 09:51:04 +00:00
/// We have to check shared node granularity before we create ours.
2021-06-28 17:02:22 +00:00
other_replicas_fixed_granularity = checkFixedGranularityInZookeeper ( ) ;
2021-06-20 08:24:43 +00:00
2020-06-16 16:55:04 +00:00
ReplicatedMergeTreeTableMetadata current_metadata ( * this , metadata_snapshot ) ;
2021-06-20 08:24:43 +00:00
current_zookeeper - > createOrUpdate ( replica_path + " /metadata " , current_metadata . toString ( ) ,
zkutil : : CreateMode : : Persistent ) ;
2020-04-07 15:28:29 +00:00
}
2020-06-16 16:55:04 +00:00
checkTableStructure ( replica_path , metadata_snapshot ) ;
2017-04-01 07:20:54 +00:00
checkParts ( skip_sanity_checks ) ;
2017-05-31 15:01:25 +00:00
2020-02-14 10:17:04 +00:00
if ( current_zookeeper - > exists ( replica_path + " /metadata_version " ) )
{
metadata_version = parse < int > ( current_zookeeper - > get ( replica_path + " /metadata_version " ) ) ;
}
2020-04-07 15:28:29 +00:00
else
2020-02-14 10:17:04 +00:00
{
2020-04-07 15:28:29 +00:00
/// This replica was created with old clickhouse version, so we have
/// to take version of global node. If somebody will alter our
2020-04-07 17:23:26 +00:00
/// table, then we will fill /metadata_version node in zookeeper.
2020-04-07 15:28:29 +00:00
/// Otherwise on the next restart we can again use version from
/// shared metadata node because it was not changed.
2020-02-14 10:17:04 +00:00
Coordination : : Stat metadata_stat ;
current_zookeeper - > get ( zookeeper_path + " /metadata " , & metadata_stat ) ;
metadata_version = metadata_stat . version ;
}
2021-06-20 08:24:43 +00:00
/// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart),
/// don't allow to reinitialize them, delete each of them immediately.
2021-09-10 21:16:09 +00:00
clearOldTemporaryDirectories ( merger_mutator , 0 ) ;
2020-05-27 20:05:55 +00:00
clearOldWriteAheadLogs ( ) ;
2017-04-01 07:20:54 +00:00
}
createNewZooKeeperNodes ( ) ;
2020-11-24 14:24:48 +00:00
syncPinnedPartUUIDs ( ) ;
2019-08-12 13:30:29 +00:00
}
2021-06-28 17:02:22 +00:00
bool StorageReplicatedMergeTree : : checkFixedGranularityInZookeeper ( )
2019-08-12 13:30:29 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
String metadata_str = zookeeper - > get ( zookeeper_path + " /metadata " ) ;
auto metadata_from_zk = ReplicatedMergeTreeTableMetadata : : parse ( metadata_str ) ;
return metadata_from_zk . index_granularity_bytes = = 0 ;
2014-03-21 13:42:14 +00:00
}
2014-10-17 01:05:51 +00:00
2019-12-19 15:27:56 +00:00
void StorageReplicatedMergeTree : : waitMutationToFinishOnReplicas (
const Strings & replicas , const String & mutation_id ) const
2019-12-16 15:51:15 +00:00
{
2019-12-19 15:27:56 +00:00
if ( replicas . empty ( ) )
return ;
2019-12-16 15:51:15 +00:00
std : : set < String > inactive_replicas ;
for ( const String & replica : replicas )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Waiting for {} to apply mutation {} " , replica , mutation_id ) ;
2021-04-05 15:08:43 +00:00
zkutil : : EventPtr wait_event = std : : make_shared < Poco : : Event > ( ) ;
2019-12-19 15:27:56 +00:00
2019-12-16 15:51:15 +00:00
while ( ! partial_shutdown_called )
{
2019-12-19 15:27:56 +00:00
/// Mutation maybe killed or whole replica was deleted.
/// Wait event will unblock at this moment.
Coordination : : Stat exists_stat ;
2021-05-09 11:59:49 +00:00
if ( ! getZooKeeper ( ) - > exists ( fs : : path ( zookeeper_path ) / " mutations " / mutation_id , & exists_stat , wait_event ) )
2019-12-19 15:27:56 +00:00
{
2020-07-22 12:36:19 +00:00
throw Exception ( ErrorCodes : : UNFINISHED , " Mutation {} was killed, manually removed or table was dropped " , mutation_id ) ;
2019-12-19 15:27:56 +00:00
}
2019-12-16 15:51:15 +00:00
auto zookeeper = getZooKeeper ( ) ;
/// Replica could be inactive.
2021-05-09 11:59:49 +00:00
if ( ! zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) )
2019-12-16 15:51:15 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Replica {} is not active during mutation. Mutation will be done asynchronously when replica becomes active. " , replica ) ;
2019-12-16 15:51:15 +00:00
inactive_replicas . emplace ( replica ) ;
break ;
}
2021-05-09 11:59:49 +00:00
String mutation_pointer = fs : : path ( zookeeper_path ) / " replicas " / replica / " mutation_pointer " ;
2019-12-19 15:27:56 +00:00
std : : string mutation_pointer_value ;
2019-12-16 15:51:15 +00:00
/// Replica could be removed
2021-04-05 15:08:43 +00:00
if ( ! zookeeper - > tryGet ( mutation_pointer , mutation_pointer_value , nullptr , wait_event ) )
2019-12-16 15:51:15 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Replica {} was removed " , replica ) ;
2019-12-16 15:51:15 +00:00
break ;
}
2019-12-19 15:27:56 +00:00
else if ( mutation_pointer_value > = mutation_id ) /// Maybe we already processed more fresh mutation
break ; /// (numbers like 0000000000 and 0000000001)
2019-12-16 15:51:15 +00:00
2020-05-12 14:11:09 +00:00
/// Replica can become inactive, so wait with timeout and recheck it
if ( wait_event - > tryWait ( 1000 ) )
2021-04-05 15:08:43 +00:00
continue ;
2020-07-22 12:36:19 +00:00
2021-05-31 15:03:45 +00:00
/// Here we check mutation for errors on local replica. If they happen on this replica
2021-04-05 15:08:43 +00:00
/// they will happen on each replica, so we can check only in-memory info.
2020-07-22 15:19:54 +00:00
auto mutation_status = queue . getIncompleteMutationsStatus ( mutation_id ) ;
2021-05-31 15:03:45 +00:00
/// If mutation status is empty, than local replica may just not loaded it into memory.
if ( mutation_status & & ! mutation_status - > latest_fail_reason . empty ( ) )
2020-07-22 12:36:19 +00:00
break ;
2019-12-16 15:51:15 +00:00
}
2021-06-27 16:18:15 +00:00
/// This replica inactive, don't check anything
if ( ! inactive_replicas . empty ( ) & & inactive_replicas . count ( replica ) )
break ;
2020-07-22 12:36:19 +00:00
/// It maybe already removed from zk, but local in-memory mutations
2020-08-16 10:15:35 +00:00
/// state was not updated.
2021-05-09 11:59:49 +00:00
if ( ! getZooKeeper ( ) - > exists ( fs : : path ( zookeeper_path ) / " mutations " / mutation_id ) )
2020-07-22 12:36:19 +00:00
{
throw Exception ( ErrorCodes : : UNFINISHED , " Mutation {} was killed, manually removed or table was dropped " , mutation_id ) ;
}
2021-06-28 08:28:45 +00:00
if ( partial_shutdown_called )
throw Exception ( " Mutation is not finished because table shutdown was called. It will be done after table restart. " ,
ErrorCodes : : UNFINISHED ) ;
/// Replica inactive, don't check mutation status
if ( ! inactive_replicas . empty ( ) & & inactive_replicas . count ( replica ) )
continue ;
2020-07-31 12:22:32 +00:00
/// At least we have our current mutation
2020-07-31 11:37:16 +00:00
std : : set < String > mutation_ids ;
mutation_ids . insert ( mutation_id ) ;
2021-04-05 15:08:43 +00:00
/// Here we check mutation for errors or kill on local replica. If they happen on this replica
/// they will happen on each replica, so we can check only in-memory info.
2020-07-22 15:19:54 +00:00
auto mutation_status = queue . getIncompleteMutationsStatus ( mutation_id , & mutation_ids ) ;
checkMutationStatus ( mutation_status , mutation_ids ) ;
2019-12-16 15:51:15 +00:00
}
2019-12-19 15:27:56 +00:00
if ( ! inactive_replicas . empty ( ) )
2019-12-16 15:51:15 +00:00
{
2020-11-10 18:22:26 +00:00
throw Exception ( ErrorCodes : : UNFINISHED ,
" Mutation is not finished because some replicas are inactive right now: {}. Mutation will be done asynchronously " ,
boost : : algorithm : : join ( inactive_replicas , " , " ) ) ;
2019-12-16 15:51:15 +00:00
}
}
2015-11-09 20:30:54 +00:00
void StorageReplicatedMergeTree : : createNewZooKeeperNodes ( )
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2015-11-09 20:30:54 +00:00
2017-04-01 07:20:54 +00:00
/// Working with quorum.
2018-04-19 10:33:16 +00:00
zookeeper - > createIfNotExists ( zookeeper_path + " /quorum " , String ( ) ) ;
2020-10-06 21:49:48 +00:00
zookeeper - > createIfNotExists ( zookeeper_path + " /quorum/parallel " , String ( ) ) ;
2018-04-19 10:33:16 +00:00
zookeeper - > createIfNotExists ( zookeeper_path + " /quorum/last_part " , String ( ) ) ;
zookeeper - > createIfNotExists ( zookeeper_path + " /quorum/failed_parts " , String ( ) ) ;
2015-11-09 20:30:54 +00:00
2017-04-01 07:20:54 +00:00
/// Tracking lag of replicas.
2018-04-19 10:33:16 +00:00
zookeeper - > createIfNotExists ( replica_path + " /min_unprocessed_insert_time " , String ( ) ) ;
zookeeper - > createIfNotExists ( replica_path + " /max_processed_insert_time " , String ( ) ) ;
/// Mutations
zookeeper - > createIfNotExists ( zookeeper_path + " /mutations " , String ( ) ) ;
2018-06-21 13:27:36 +00:00
zookeeper - > createIfNotExists ( replica_path + " /mutation_pointer " , String ( ) ) ;
2021-03-09 08:45:41 +00:00
2021-07-05 03:32:56 +00:00
/// Nodes for remote fs zero-copy replication
2021-06-24 08:25:05 +00:00
const auto settings = getSettings ( ) ;
2021-07-05 03:32:56 +00:00
if ( settings - > allow_remote_fs_zero_copy_replication )
2021-03-09 08:45:41 +00:00
{
zookeeper - > createIfNotExists ( zookeeper_path + " /zero_copy_s3 " , String ( ) ) ;
zookeeper - > createIfNotExists ( zookeeper_path + " /zero_copy_s3/shared " , String ( ) ) ;
2021-06-24 08:25:05 +00:00
zookeeper - > createIfNotExists ( zookeeper_path + " /zero_copy_hdfs " , String ( ) ) ;
zookeeper - > createIfNotExists ( zookeeper_path + " /zero_copy_hdfs/shared " , String ( ) ) ;
}
2020-11-24 14:24:48 +00:00
/// Part movement.
zookeeper - > createIfNotExists ( zookeeper_path + " /part_moves_shard " , String ( ) ) ;
zookeeper - > createIfNotExists ( zookeeper_path + " /pinned_part_uuids " , getPinnedPartUUIDs ( ) - > toString ( ) ) ;
2021-05-17 19:23:38 +00:00
/// For ALTER PARTITION with multi-leaders
2021-05-18 10:00:34 +00:00
zookeeper - > createIfNotExists ( zookeeper_path + " /alter_partition_version " , String ( ) ) ;
2016-04-09 05:45:42 +00:00
}
2020-06-16 16:55:04 +00:00
bool StorageReplicatedMergeTree : : createTableIfNotExists ( const StorageMetadataPtr & metadata_snapshot )
2014-03-21 19:17:59 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2020-06-11 03:24:52 +00:00
zookeeper - > createAncestors ( zookeeper_path ) ;
2017-04-01 07:20:54 +00:00
2020-06-11 03:24:52 +00:00
for ( size_t i = 0 ; i < 1000 ; + + i )
{
/// Invariant: "replicas" does not exist if there is no table or if there are leftovers from incompletely dropped table.
if ( zookeeper - > exists ( zookeeper_path + " /replicas " ) )
{
LOG_DEBUG ( log , " This table {} is already created, will add new replica " , zookeeper_path ) ;
return false ;
}
2017-04-01 07:20:54 +00:00
2020-06-11 03:24:52 +00:00
/// There are leftovers from incompletely dropped table.
if ( zookeeper - > exists ( zookeeper_path + " /dropped " ) )
{
/// This condition may happen when the previous drop attempt was not completed
/// or when table is dropped by another replica right now.
/// This is Ok because another replica is definitely going to drop the table.
2017-04-01 07:20:54 +00:00
2020-06-11 03:24:52 +00:00
LOG_WARNING ( log , " Removing leftovers from table {} (this might take several minutes) " , zookeeper_path ) ;
2021-04-19 08:21:42 +00:00
String drop_lock_path = zookeeper_path + " /dropped/lock " ;
Coordination : : Error code = zookeeper - > tryCreate ( drop_lock_path , " " , zkutil : : CreateMode : : Ephemeral ) ;
2017-04-01 07:20:54 +00:00
2021-04-19 08:21:42 +00:00
if ( code = = Coordination : : Error : : ZNONODE | | code = = Coordination : : Error : : ZNODEEXISTS )
2020-06-11 03:24:52 +00:00
{
2021-04-19 08:21:42 +00:00
LOG_WARNING ( log , " The leftovers from table {} were removed by another replica " , zookeeper_path ) ;
}
else if ( code ! = Coordination : : Error : : ZOK )
{
throw Coordination : : Exception ( code , drop_lock_path ) ;
2020-06-11 03:24:52 +00:00
}
else
{
2021-04-19 10:40:20 +00:00
auto metadata_drop_lock = zkutil : : EphemeralNodeHolder : : existing ( drop_lock_path , * zookeeper ) ;
if ( ! removeTableNodesFromZooKeeper ( zookeeper , zookeeper_path , metadata_drop_lock , log ) )
2020-06-11 03:24:52 +00:00
{
2021-04-19 08:21:42 +00:00
/// Someone is recursively removing table right now, we cannot create new table until old one is removed
continue ;
2020-06-11 03:24:52 +00:00
}
}
}
2017-04-01 07:20:54 +00:00
2020-06-11 03:24:52 +00:00
LOG_DEBUG ( log , " Creating table {} " , zookeeper_path ) ;
/// We write metadata of table so that the replicas can check table parameters with them.
2020-06-16 16:55:04 +00:00
String metadata_str = ReplicatedMergeTreeTableMetadata ( * this , metadata_snapshot ) . toString ( ) ;
2020-06-11 03:24:52 +00:00
Coordination : : Requests ops ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
2020-06-11 21:03:25 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /metadata " , metadata_str ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
2020-06-17 16:39:58 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /columns " , metadata_snapshot - > getColumns ( ) . toString ( ) ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /log " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /blocks " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /block_numbers " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /nonincrement_block_numbers " , " " ,
zkutil : : CreateMode : : Persistent ) ) ; /// /nonincrement_block_numbers dir is unused, but is created nonetheless for backwards compatibility.
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /leader_election " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /temp " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /replicas " , " last added replica: " + replica_name ,
zkutil : : CreateMode : : Persistent ) ) ;
/// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas.
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /host " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /log_pointer " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /queue " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /parts " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /flags " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /is_lost " , " 0 " ,
zkutil : : CreateMode : : Persistent ) ) ;
2020-06-11 21:03:25 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /metadata " , metadata_str ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
2020-06-17 16:39:58 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /columns " , metadata_snapshot - > getColumns ( ) . toString ( ) ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /metadata_version " , std : : to_string ( metadata_version ) ,
zkutil : : CreateMode : : Persistent ) ) ;
Coordination : : Responses responses ;
auto code = zookeeper - > tryMulti ( ops , responses ) ;
2020-06-12 15:09:12 +00:00
if ( code = = Coordination : : Error : : ZNODEEXISTS )
2020-06-11 03:24:52 +00:00
{
LOG_WARNING ( log , " It looks like the table {} was created by another server at the same moment, will retry " , zookeeper_path ) ;
continue ;
}
2020-06-12 15:09:12 +00:00
else if ( code ! = Coordination : : Error : : ZOK )
2020-06-11 03:24:52 +00:00
{
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
return true ;
}
2020-09-09 17:39:49 +00:00
/// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path
throw Exception ( " Cannot create table, because it is created concurrently every time "
" or because of wrong zookeeper_path "
" or because of logical error " , ErrorCodes : : REPLICA_IS_ALREADY_EXIST ) ;
2020-06-11 03:24:52 +00:00
}
2020-06-16 16:55:04 +00:00
void StorageReplicatedMergeTree : : createReplica ( const StorageMetadataPtr & metadata_snapshot )
2020-06-11 03:24:52 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
LOG_DEBUG ( log , " Creating replica {} " , replica_path ) ;
2020-06-12 15:09:12 +00:00
Coordination : : Error code ;
2020-06-11 03:24:52 +00:00
do
{
Coordination : : Stat replicas_stat ;
2020-06-11 03:35:59 +00:00
String replicas_value ;
2020-06-11 03:24:52 +00:00
2020-06-11 19:59:15 +00:00
if ( ! zookeeper - > tryGet ( zookeeper_path + " /replicas " , replicas_value , & replicas_stat ) )
2021-06-20 08:24:43 +00:00
throw Exception ( ErrorCodes : : ALL_REPLICAS_LOST ,
" Cannot create a replica of the table {}, because the last replica of the table was dropped right now " ,
zookeeper_path ) ;
2020-06-11 03:24:52 +00:00
/// It is not the first replica, we will mark it as "lost", to immediately repair (clone) from existing replica.
/// By the way, it's possible that the replica will be first, if all previous replicas were removed concurrently.
2021-06-20 08:24:43 +00:00
const String is_lost_value = replicas_stat . numChildren ? " 1 " : " 0 " ;
2020-06-11 03:24:52 +00:00
Coordination : : Requests ops ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /host " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /log_pointer " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /queue " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /parts " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /flags " , " " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /is_lost " , is_lost_value ,
zkutil : : CreateMode : : Persistent ) ) ;
2020-06-16 16:55:04 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /metadata " , ReplicatedMergeTreeTableMetadata ( * this , metadata_snapshot ) . toString ( ) ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
2020-06-17 16:39:58 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /columns " , metadata_snapshot - > getColumns ( ) . toString ( ) ,
2020-06-11 03:24:52 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( replica_path + " /metadata_version " , std : : to_string ( metadata_version ) ,
zkutil : : CreateMode : : Persistent ) ) ;
/// Check version of /replicas to see if there are any replicas created at the same moment of time.
ops . emplace_back ( zkutil : : makeSetRequest ( zookeeper_path + " /replicas " , " last added replica: " + replica_name , replicas_stat . version ) ) ;
Coordination : : Responses responses ;
code = zookeeper - > tryMulti ( ops , responses ) ;
2021-06-20 08:24:43 +00:00
switch ( code )
2020-06-11 03:24:52 +00:00
{
2021-06-20 08:24:43 +00:00
case Coordination : : Error : : ZNODEEXISTS :
throw Exception ( ErrorCodes : : REPLICA_IS_ALREADY_EXIST , " Replica {} already exists " , replica_path ) ;
case Coordination : : Error : : ZBADVERSION :
LOG_ERROR ( log , " Retrying createReplica(), because some other replicas were created at the same time " ) ;
break ;
case Coordination : : Error : : ZNONODE :
throw Exception ( ErrorCodes : : ALL_REPLICAS_LOST , " Table {} was suddenly removed " , zookeeper_path ) ;
default :
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2020-06-11 03:24:52 +00:00
}
2020-06-12 15:09:12 +00:00
} while ( code = = Coordination : : Error : : ZBADVERSION ) ;
2020-06-11 03:24:52 +00:00
}
void StorageReplicatedMergeTree : : drop ( )
{
2020-06-23 12:01:51 +00:00
/// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster
/// or metadata of staled replica were removed manually,
2020-06-14 01:23:53 +00:00
/// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table.
if ( has_metadata_in_zookeeper )
2020-06-11 03:24:52 +00:00
{
2020-09-17 11:27:17 +00:00
/// Table can be shut down, restarting thread is not active
2021-02-24 08:33:39 +00:00
/// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice.
zkutil : : ZooKeeperPtr zookeeper ;
if ( zookeeper_name = = default_zookeeper_name )
2021-04-10 23:33:54 +00:00
zookeeper = getContext ( ) - > getZooKeeper ( ) ;
2021-02-24 08:33:39 +00:00
else
2021-04-10 23:33:54 +00:00
zookeeper = getContext ( ) - > getAuxiliaryZooKeeper ( zookeeper_name ) ;
2020-06-11 03:24:52 +00:00
2020-06-14 01:23:53 +00:00
/// If probably there is metadata in ZooKeeper, we don't allow to drop the table.
2021-02-15 12:04:30 +00:00
if ( ! zookeeper )
2020-06-11 03:24:52 +00:00
throw Exception ( " Can't drop readonly replicated table (need to drop data in ZooKeeper as well) " , ErrorCodes::TABLE_IS_READ_ONLY) ;
shutdown ( ) ;
2020-06-23 12:01:51 +00:00
dropReplica ( zookeeper , zookeeper_path , replica_name , log ) ;
2020-06-11 03:24:52 +00:00
}
dropAllData ( ) ;
2014-03-22 14:44:44 +00:00
}
2020-06-11 03:24:52 +00:00
2020-06-23 12:01:51 +00:00
void StorageReplicatedMergeTree : : dropReplica ( zkutil : : ZooKeeperPtr zookeeper , const String & zookeeper_path , const String & replica , Poco : : Logger * logger )
{
if ( zookeeper - > expired ( ) )
throw Exception ( " Table was not dropped because ZooKeeper session has expired. " , ErrorCodes : : TABLE_WAS_NOT_DROPPED ) ;
2020-06-11 03:24:52 +00:00
2020-06-23 12:01:51 +00:00
auto remote_replica_path = zookeeper_path + " /replicas/ " + replica ;
2021-10-28 18:00:33 +00:00
2020-10-07 19:47:31 +00:00
LOG_INFO ( logger , " Removing replica {}, marking it as lost " , remote_replica_path ) ;
/// Mark itself lost before removing, because the following recursive removal may fail
/// and partially dropped replica may be considered as alive one (until someone will mark it lost)
2021-10-28 18:00:33 +00:00
zookeeper - > trySet ( remote_replica_path + " /is_lost " , " 1 " ) ;
2021-10-28 18:00:33 +00:00
/// NOTE: we should check for remote_replica_path existence,
/// since otherwise DROP REPLICA will fail if the replica had been already removed.
if ( ! zookeeper - > exists ( remote_replica_path ) )
{
LOG_INFO ( logger , " Removing replica {} does not exist " , remote_replica_path ) ;
return ;
}
/// Analog of removeRecursive(remote_replica_path)
/// but it removes "metadata" firstly.
///
/// This will allow to mark table as readonly
/// and skip any checks of parts between on-disk and in the zookeeper.
///
/// Without this removeRecursive() may remove "parts" first
/// and on DETACH/ATTACH (or server restart) it will trigger the following error:
///
/// "The local set of parts of table X doesn't look like the set of parts in ZooKeeper"
///
{
Strings children = zookeeper - > getChildren ( remote_replica_path ) ;
if ( std : : find ( children . begin ( ) , children . end ( ) , " metadata " ) ! = children . end ( ) )
zookeeper - > remove ( fs : : path ( remote_replica_path ) / " metadata " ) ;
for ( const auto & child : children )
{
if ( child ! = " metadata " )
zookeeper - > removeRecursive ( fs : : path ( remote_replica_path ) / child ) ;
}
zookeeper - > remove ( remote_replica_path ) ;
}
2020-06-23 12:01:51 +00:00
/// It may left some garbage if replica_path subtree are concurrently modified
if ( zookeeper - > exists ( remote_replica_path ) )
LOG_ERROR ( logger , " Replica was not completely removed from ZooKeeper, {} still exists and may contain some garbage. " , remote_replica_path ) ;
2020-06-11 03:24:52 +00:00
2020-06-23 12:01:51 +00:00
/// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
Strings replicas ;
if ( Coordination : : Error : : ZOK ! = zookeeper - > tryGetChildren ( zookeeper_path + " /replicas " , replicas ) | | ! replicas . empty ( ) )
return ;
2020-06-11 03:24:52 +00:00
2020-06-23 12:01:51 +00:00
LOG_INFO ( logger , " {} is the last replica, will remove table " , remote_replica_path ) ;
/** At this moment, another replica can be created and we cannot remove the table.
* Try to remove / replicas node first . If we successfully removed it ,
* it guarantees that we are the only replica that proceed to remove the table
* and no new replicas can be created after that moment ( it requires the existence of / replicas node ) .
* and table cannot be recreated with new / replicas node on another servers while we are removing data ,
* because table creation is executed in single transaction that will conflict with remaining nodes .
*/
2020-06-11 03:24:52 +00:00
2021-04-19 08:21:42 +00:00
/// Node /dropped works like a lock that protects from concurrent removal of old table and creation of new table.
/// But recursive removal may fail in the middle of operation leaving some garbage in zookeeper_path, so
/// we remove it on table creation if there is /dropped node. Creating thread may remove /dropped node created by
/// removing thread, and it causes race condition if removing thread is not finished yet.
/// To avoid this we also create ephemeral child before starting recursive removal.
2021-04-21 13:01:54 +00:00
/// (The existence of child node does not allow to remove parent node).
2020-06-23 12:01:51 +00:00
Coordination : : Requests ops ;
Coordination : : Responses responses ;
2021-04-19 08:21:42 +00:00
String drop_lock_path = zookeeper_path + " /dropped/lock " ;
2020-06-23 12:01:51 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( zookeeper_path + " /replicas " , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_path + " /dropped " , " " , zkutil : : CreateMode : : Persistent ) ) ;
2021-04-19 08:21:42 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( drop_lock_path , " " , zkutil : : CreateMode : : Ephemeral ) ) ;
2020-06-23 12:01:51 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , responses ) ;
2020-06-11 03:24:52 +00:00
2020-06-23 12:01:51 +00:00
if ( code = = Coordination : : Error : : ZNONODE | | code = = Coordination : : Error : : ZNODEEXISTS )
{
LOG_WARNING ( logger , " Table {} is already started to be removing by another replica right now " , remote_replica_path ) ;
}
else if ( code = = Coordination : : Error : : ZNOTEMPTY )
{
LOG_WARNING ( logger , " Another replica was suddenly created, will keep the table {} " , remote_replica_path ) ;
}
else if ( code ! = Coordination : : Error : : ZOK )
{
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
else
{
2021-04-19 10:40:20 +00:00
auto metadata_drop_lock = zkutil : : EphemeralNodeHolder : : existing ( drop_lock_path , * zookeeper ) ;
2020-06-23 12:01:51 +00:00
LOG_INFO ( logger , " Removing table {} (this might take several minutes) " , zookeeper_path ) ;
2021-04-19 10:40:20 +00:00
removeTableNodesFromZooKeeper ( zookeeper , zookeeper_path , metadata_drop_lock , logger ) ;
2021-04-19 08:21:42 +00:00
}
}
2020-06-11 03:24:52 +00:00
2021-04-19 08:21:42 +00:00
bool StorageReplicatedMergeTree : : removeTableNodesFromZooKeeper ( zkutil : : ZooKeeperPtr zookeeper ,
2021-04-19 10:40:20 +00:00
const String & zookeeper_path , const zkutil : : EphemeralNodeHolder : : Ptr & metadata_drop_lock , Poco : : Logger * logger )
2021-04-19 08:21:42 +00:00
{
bool completely_removed = false ;
Strings children ;
Coordination : : Error code = zookeeper - > tryGetChildren ( zookeeper_path , children ) ;
if ( code = = Coordination : : Error : : ZNONODE )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is a race condition between creation and removal of replicated table. It's a bug " ) ;
2020-06-23 12:01:51 +00:00
2020-06-11 03:24:52 +00:00
2021-04-19 08:21:42 +00:00
for ( const auto & child : children )
if ( child ! = " dropped " )
2021-05-08 10:59:55 +00:00
zookeeper - > tryRemoveRecursive ( fs : : path ( zookeeper_path ) / child ) ;
2021-04-19 08:21:42 +00:00
Coordination : : Requests ops ;
Coordination : : Responses responses ;
2021-04-19 10:40:20 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( metadata_drop_lock - > getPath ( ) , - 1 ) ) ;
2021-05-08 10:59:55 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( fs : : path ( zookeeper_path ) / " dropped " , - 1 ) ) ;
2021-04-19 08:21:42 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( zookeeper_path , - 1 ) ) ;
code = zookeeper - > tryMulti ( ops , responses ) ;
if ( code = = Coordination : : Error : : ZNONODE )
{
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is a race condition between creation and removal of replicated table. It's a bug " ) ;
2020-06-11 03:24:52 +00:00
}
2021-04-19 08:21:42 +00:00
else if ( code = = Coordination : : Error : : ZNOTEMPTY )
{
LOG_ERROR ( logger , " Table was not completely removed from ZooKeeper, {} still exists and may contain some garbage, "
" but someone is removing it right now. " , zookeeper_path ) ;
}
else if ( code ! = Coordination : : Error : : ZOK )
{
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
else
{
2021-04-19 10:40:20 +00:00
metadata_drop_lock - > setAlreadyRemoved ( ) ;
2021-04-19 08:21:42 +00:00
completely_removed = true ;
LOG_INFO ( logger , " Table {} was successfully removed from ZooKeeper " , zookeeper_path ) ;
}
return completely_removed ;
2014-03-22 14:44:44 +00:00
}
2014-03-21 19:17:59 +00:00
2014-10-18 17:37:55 +00:00
2020-08-15 12:30:17 +00:00
/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/metadata).
* If not , throw an exception .
*/
2020-06-16 16:55:04 +00:00
void StorageReplicatedMergeTree : : checkTableStructure ( const String & zookeeper_prefix , const StorageMetadataPtr & metadata_snapshot )
2014-03-22 14:44:44 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2020-06-16 16:55:04 +00:00
ReplicatedMergeTreeTableMetadata old_metadata ( * this , metadata_snapshot ) ;
2018-11-02 11:53:05 +00:00
2018-11-01 13:30:38 +00:00
Coordination : : Stat metadata_stat ;
2021-05-08 10:59:55 +00:00
String metadata_str = zookeeper - > get ( fs : : path ( zookeeper_prefix ) / " metadata " , & metadata_stat ) ;
2018-11-02 15:39:19 +00:00
auto metadata_from_zk = ReplicatedMergeTreeTableMetadata : : parse ( metadata_str ) ;
2021-04-10 23:33:54 +00:00
old_metadata . checkEquals ( metadata_from_zk , metadata_snapshot - > getColumns ( ) , getContext ( ) ) ;
2017-04-01 07:20:54 +00:00
2018-11-01 13:30:38 +00:00
Coordination : : Stat columns_stat ;
2021-05-08 10:59:55 +00:00
auto columns_from_zk = ColumnsDescription : : parse ( zookeeper - > get ( fs : : path ( zookeeper_prefix ) / " columns " , & columns_stat ) ) ;
2017-04-01 07:20:54 +00:00
2020-06-17 16:39:58 +00:00
const ColumnsDescription & old_columns = metadata_snapshot - > getColumns ( ) ;
2020-02-14 10:17:04 +00:00
if ( columns_from_zk ! = old_columns )
2017-04-01 07:20:54 +00:00
{
2021-09-21 19:39:30 +00:00
throw Exception ( ErrorCodes : : INCOMPATIBLE_COLUMNS ,
" Table columns structure in ZooKeeper is different from local table structure. Local columns: \n "
" {} \n Zookeeper columns: \n {} " , old_columns . toString ( ) , columns_from_zk . toString ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2014-03-22 14:44:44 +00:00
}
2014-03-21 19:17:59 +00:00
2020-06-27 20:13:16 +00:00
void StorageReplicatedMergeTree : : setTableStructure (
2020-10-15 13:02:39 +00:00
ColumnsDescription new_columns , const ReplicatedMergeTreeTableMetadata : : Diff & metadata_diff )
2018-11-02 11:53:05 +00:00
{
2020-06-10 09:09:51 +00:00
StorageInMemoryMetadata new_metadata = getInMemoryMetadata ( ) ;
2020-06-17 10:34:23 +00:00
StorageInMemoryMetadata old_metadata = getInMemoryMetadata ( ) ;
2020-06-27 20:13:16 +00:00
2021-03-15 08:22:15 +00:00
new_metadata . columns = new_columns ;
2019-12-27 14:36:59 +00:00
2018-11-02 15:39:19 +00:00
if ( ! metadata_diff . empty ( ) )
2018-11-02 11:53:05 +00:00
{
2020-10-15 13:02:39 +00:00
auto parse_key_expr = [ ] ( const String & key_expr )
2018-11-13 12:51:55 +00:00
{
2019-02-05 14:50:25 +00:00
ParserNotEmptyExpressionList parser ( false ) ;
2020-08-27 13:10:10 +00:00
auto new_sorting_key_expr_list = parseQuery ( parser , key_expr , 0 , DBMS_DEFAULT_MAX_PARSER_DEPTH ) ;
2018-11-06 18:25:36 +00:00
2020-06-05 17:29:40 +00:00
ASTPtr order_by_ast ;
2019-02-05 14:50:25 +00:00
if ( new_sorting_key_expr_list - > children . size ( ) = = 1 )
2020-06-05 17:29:40 +00:00
order_by_ast = new_sorting_key_expr_list - > children [ 0 ] ;
2019-02-05 14:50:25 +00:00
else
{
auto tuple = makeASTFunction ( " tuple " ) ;
tuple - > arguments - > children = new_sorting_key_expr_list - > children ;
2020-06-05 17:29:40 +00:00
order_by_ast = tuple ;
2019-02-05 14:50:25 +00:00
}
2020-08-27 13:10:10 +00:00
return order_by_ast ;
} ;
if ( metadata_diff . sorting_key_changed )
{
auto order_by_ast = parse_key_expr ( metadata_diff . new_sorting_key ) ;
2020-06-12 09:37:52 +00:00
auto & sorting_key = new_metadata . sorting_key ;
auto & primary_key = new_metadata . primary_key ;
2019-02-05 14:50:25 +00:00
2021-04-10 23:33:54 +00:00
sorting_key . recalculateWithNewAST ( order_by_ast , new_metadata . columns , getContext ( ) ) ;
2019-02-05 14:50:25 +00:00
2020-06-12 09:37:52 +00:00
if ( primary_key . definition_ast = = nullptr )
2019-02-05 14:50:25 +00:00
{
/// Primary and sorting key become independent after this ALTER so we have to
/// save the old ORDER BY expression as the new primary key.
2020-06-17 11:05:11 +00:00
auto old_sorting_key_ast = old_metadata . getSortingKey ( ) . definition_ast ;
2020-06-12 09:37:52 +00:00
primary_key = KeyDescription : : getKeyFromAST (
2021-04-10 23:33:54 +00:00
old_sorting_key_ast , new_metadata . columns , getContext ( ) ) ;
2019-02-05 14:50:25 +00:00
}
2018-11-06 18:25:36 +00:00
}
2018-11-02 11:53:05 +00:00
2020-08-27 13:10:10 +00:00
if ( metadata_diff . sampling_expression_changed )
{
2021-10-14 13:44:28 +00:00
if ( ! metadata_diff . new_sampling_expression . empty ( ) )
{
auto sample_by_ast = parse_key_expr ( metadata_diff . new_sampling_expression ) ;
new_metadata . sampling_key . recalculateWithNewAST ( sample_by_ast , new_metadata . columns , getContext ( ) ) ;
}
else /// SAMPLE BY was removed
{
new_metadata . sampling_key = { } ;
}
2020-08-27 13:10:10 +00:00
}
2019-02-05 14:50:25 +00:00
if ( metadata_diff . skip_indices_changed )
2021-04-10 23:33:54 +00:00
new_metadata . secondary_indices = IndicesDescription : : parse ( metadata_diff . new_skip_indices , new_columns , getContext ( ) ) ;
2019-02-05 14:50:25 +00:00
2019-06-02 14:41:12 +00:00
if ( metadata_diff . constraints_changed )
2020-06-10 09:09:51 +00:00
new_metadata . constraints = ConstraintsDescription : : parse ( metadata_diff . new_constraints ) ;
2019-06-02 14:41:12 +00:00
2021-02-10 14:12:49 +00:00
if ( metadata_diff . projections_changed )
new_metadata . projections = ProjectionsDescription : : parse ( metadata_diff . new_projections , new_columns , getContext ( ) ) ;
2019-04-15 09:30:45 +00:00
if ( metadata_diff . ttl_table_changed )
{
2020-09-20 13:27:33 +00:00
if ( ! metadata_diff . new_ttl_table . empty ( ) )
{
ParserTTLExpressionList parser ;
auto ttl_for_table_ast = parseQuery ( parser , metadata_diff . new_ttl_table , 0 , DBMS_DEFAULT_MAX_PARSER_DEPTH ) ;
new_metadata . table_ttl = TTLTableDescription : : getTTLForTableFromAST (
2021-04-10 23:33:54 +00:00
ttl_for_table_ast , new_metadata . columns , getContext ( ) , new_metadata . primary_key ) ;
2020-09-20 13:27:33 +00:00
}
else /// TTL was removed
{
new_metadata . table_ttl = TTLTableDescription { } ;
}
2019-04-15 09:30:45 +00:00
}
2018-11-02 11:53:05 +00:00
}
2021-03-01 09:59:19 +00:00
/// Changes in columns may affect following metadata fields
2021-03-15 08:22:15 +00:00
new_metadata . column_ttls_by_name . clear ( ) ;
for ( const auto & [ name , ast ] : new_metadata . columns . getColumnTTLs ( ) )
2021-03-01 09:59:19 +00:00
{
2021-04-10 23:33:54 +00:00
auto new_ttl_entry = TTLDescription : : getTTLFromAST ( ast , new_metadata . columns , getContext ( ) , new_metadata . primary_key ) ;
2021-03-15 08:22:15 +00:00
new_metadata . column_ttls_by_name [ name ] = new_ttl_entry ;
}
2021-03-01 09:59:19 +00:00
2021-03-15 08:22:15 +00:00
if ( new_metadata . partition_key . definition_ast ! = nullptr )
2021-04-10 23:33:54 +00:00
new_metadata . partition_key . recalculateWithNewColumns ( new_metadata . columns , getContext ( ) ) ;
2021-03-01 09:59:19 +00:00
2021-03-15 08:22:15 +00:00
if ( ! metadata_diff . sorting_key_changed ) /// otherwise already updated
2021-04-10 23:33:54 +00:00
new_metadata . sorting_key . recalculateWithNewColumns ( new_metadata . columns , getContext ( ) ) ;
2021-03-01 09:59:19 +00:00
2021-03-15 08:22:15 +00:00
/// Primary key is special, it exists even if not defined
if ( new_metadata . primary_key . definition_ast ! = nullptr )
{
2021-04-10 23:33:54 +00:00
new_metadata . primary_key . recalculateWithNewColumns ( new_metadata . columns , getContext ( ) ) ;
2021-03-15 08:22:15 +00:00
}
else
{
2021-04-10 23:33:54 +00:00
new_metadata . primary_key = KeyDescription : : getKeyFromAST ( new_metadata . sorting_key . definition_ast , new_metadata . columns , getContext ( ) ) ;
2021-03-15 08:22:15 +00:00
new_metadata . primary_key . definition_ast = nullptr ;
}
2021-03-01 09:59:19 +00:00
2021-03-15 08:22:15 +00:00
if ( ! metadata_diff . sampling_expression_changed & & new_metadata . sampling_key . definition_ast ! = nullptr )
2021-04-10 23:33:54 +00:00
new_metadata . sampling_key . recalculateWithNewColumns ( new_metadata . columns , getContext ( ) ) ;
2021-03-01 09:59:19 +00:00
2021-03-15 08:22:15 +00:00
if ( ! metadata_diff . skip_indices_changed ) /// otherwise already updated
{
for ( auto & index : new_metadata . secondary_indices )
2021-04-10 23:33:54 +00:00
index . recalculateWithNewColumns ( new_metadata . columns , getContext ( ) ) ;
2021-03-01 09:59:19 +00:00
}
2021-03-15 08:22:15 +00:00
if ( ! metadata_diff . ttl_table_changed & & new_metadata . table_ttl . definition_ast ! = nullptr )
new_metadata . table_ttl = TTLTableDescription : : getTTLForTableFromAST (
2021-04-10 23:33:54 +00:00
new_metadata . table_ttl . definition_ast , new_metadata . columns , getContext ( ) , new_metadata . primary_key ) ;
2021-03-15 08:22:15 +00:00
2020-06-27 20:13:16 +00:00
/// Even if the primary/sorting/partition keys didn't change we must reinitialize it
/// because primary/partition key column types might have changed.
2020-06-18 17:09:06 +00:00
checkTTLExpressions ( new_metadata , old_metadata ) ;
2020-06-17 10:34:23 +00:00
setProperties ( new_metadata , old_metadata ) ;
2020-06-18 17:03:42 +00:00
auto table_id = getStorageID ( ) ;
2021-04-10 23:33:54 +00:00
DatabaseCatalog : : instance ( ) . getDatabase ( table_id . database_name ) - > alterTable ( getContext ( ) , table_id , new_metadata ) ;
2018-11-02 11:53:05 +00:00
}
2017-03-12 19:18:07 +00:00
/** If necessary, restore a part, replica itself adds a record for its receipt.
* What time should I put for this entry in the queue ? Time is taken into account when calculating lag of replica .
* For these purposes , it makes sense to use creation time of missing part
* ( that is , in calculating lag , it will be taken into account how old is the part we need to recover ) .
2015-09-20 05:50:15 +00:00
*/
static time_t tryGetPartCreateTime ( zkutil : : ZooKeeperPtr & zookeeper , const String & replica_path , const String & part_name )
{
2017-04-01 07:20:54 +00:00
time_t res = 0 ;
2015-09-20 05:50:15 +00:00
2017-04-01 07:20:54 +00:00
/// We get creation time of part, if it still exists (was not merged, for example).
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2017-04-01 07:20:54 +00:00
String unused ;
2021-05-08 10:59:55 +00:00
if ( zookeeper - > tryGet ( fs : : path ( replica_path ) / " parts " / part_name , unused , & stat ) )
2017-04-01 07:20:54 +00:00
res = stat . ctime / 1000 ;
2015-09-20 05:50:15 +00:00
2017-04-01 07:20:54 +00:00
return res ;
2015-09-20 05:50:15 +00:00
}
2014-07-10 08:40:59 +00:00
void StorageReplicatedMergeTree : : checkParts ( bool skip_sanity_checks )
2014-04-02 07:59:43 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
Strings expected_parts_vec = zookeeper - > getChildren ( fs : : path ( replica_path ) / " parts " ) ;
2017-04-01 07:20:54 +00:00
/// Parts in ZK.
NameSet expected_parts ( expected_parts_vec . begin ( ) , expected_parts_vec . end ( ) ) ;
2017-12-19 14:55:50 +00:00
/// There are no PreCommitted parts at startup.
2019-05-03 02:00:57 +00:00
auto parts = getDataParts ( { MergeTreeDataPartState : : Committed , MergeTreeDataPartState : : Outdated } ) ;
2017-04-01 07:20:54 +00:00
2019-03-14 14:46:28 +00:00
/** Local parts that are not in ZK.
* In very rare cases they may cover missing parts
* and someone may think that pushing them to zookeeper is good idea .
* But actually we can ' t precisely determine that ALL missing parts
* covered by this unexpected part . So missing parts will be downloaded .
*/
2019-05-03 02:00:57 +00:00
DataParts unexpected_parts ;
2017-04-01 07:20:54 +00:00
2019-03-14 18:22:04 +00:00
/// Collect unexpected parts
2019-03-14 14:46:28 +00:00
for ( const auto & part : parts )
2019-03-14 18:22:04 +00:00
if ( ! expected_parts . count ( part - > name ) )
2019-03-14 14:46:28 +00:00
unexpected_parts . insert ( part ) ; /// this parts we will place to detached with ignored_ prefix
2017-04-01 07:20:54 +00:00
2019-03-14 18:22:04 +00:00
/// Which parts should be taken from other replicas.
Strings parts_to_fetch ;
for ( const String & missing_name : expected_parts )
2019-05-03 02:00:57 +00:00
if ( ! getActiveContainingPart ( missing_name ) )
2019-03-14 18:22:04 +00:00
parts_to_fetch . push_back ( missing_name ) ;
2018-02-21 17:06:29 +00:00
2017-04-01 07:20:54 +00:00
/** To check the adequacy, for the parts that are in the FS, but not in ZK, we will only consider not the most recent parts.
* Because unexpected new parts usually arise only because they did not have time to enroll in ZK with a rough restart of the server .
* It also occurs from deduplicated parts that did not have time to retire .
*/
size_t unexpected_parts_nonnew = 0 ;
2018-02-21 17:06:29 +00:00
UInt64 unexpected_parts_nonnew_rows = 0 ;
UInt64 unexpected_parts_rows = 0 ;
2021-06-20 08:24:43 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & part : unexpected_parts )
2018-02-21 17:06:29 +00:00
{
2017-08-14 18:16:11 +00:00
if ( part - > info . level > 0 )
2018-02-21 17:06:29 +00:00
{
2017-04-01 07:20:54 +00:00
+ + unexpected_parts_nonnew ;
2018-02-21 17:06:29 +00:00
unexpected_parts_nonnew_rows + = part - > rows_count ;
}
unexpected_parts_rows + = part - > rows_count ;
}
2021-06-20 08:24:43 +00:00
const UInt64 parts_to_fetch_blocks = std : : accumulate ( parts_to_fetch . cbegin ( ) , parts_to_fetch . cend ( ) , 0 ,
[ & ] ( UInt64 acc , const String & part_name )
{
2021-08-24 12:57:49 +00:00
if ( const auto part_info = MergeTreePartInfo : : tryParsePartName ( part_name , format_version ) )
return acc + part_info - > getBlocksCount ( ) ;
2018-02-21 17:06:29 +00:00
2021-06-20 08:24:43 +00:00
LOG_ERROR ( log , " Unexpected part name: {} " , part_name ) ;
return acc ;
} ) ;
2018-02-21 17:06:29 +00:00
2017-06-21 19:07:08 +00:00
/** We can automatically synchronize data,
* if the ratio of the total number of errors to the total number of parts ( minimum - on the local filesystem or in ZK )
* is no more than some threshold ( for example 50 % ) .
2017-04-01 07:20:54 +00:00
*
2017-06-21 19:07:08 +00:00
* A large ratio of mismatches in the data on the filesystem and the expected data
* may indicate a configuration error ( the server accidentally connected as a replica not from right shard ) .
2017-04-01 07:20:54 +00:00
* In this case , the protection mechanism does not allow the server to start .
*/
2018-02-21 17:06:29 +00:00
UInt64 total_rows_on_filesystem = 0 ;
for ( const auto & part : parts )
total_rows_on_filesystem + = part - > rows_count ;
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
bool insane = unexpected_parts_rows > total_rows_on_filesystem * storage_settings_ptr - > replicated_max_ratio_of_wrong_parts ;
2017-04-01 07:20:54 +00:00
2020-11-10 18:22:26 +00:00
constexpr const char * sanity_report_fmt = " The local set of parts of table {} doesn't look like the set of parts in ZooKeeper: "
" {} rows of {} total rows in filesystem are suspicious. "
" There are {} unexpected parts with {} rows ({} of them is not just-written with {} rows), "
" {} missing parts (with {} blocks). " ;
2017-04-01 07:20:54 +00:00
if ( insane & & ! skip_sanity_checks )
2018-02-21 17:06:29 +00:00
{
2020-11-10 18:22:26 +00:00
throw Exception ( ErrorCodes : : TOO_MANY_UNEXPECTED_DATA_PARTS , sanity_report_fmt , getStorageID ( ) . getNameForLogs ( ) ,
formatReadableQuantity ( unexpected_parts_rows ) , formatReadableQuantity ( total_rows_on_filesystem ) ,
unexpected_parts . size ( ) , unexpected_parts_rows , unexpected_parts_nonnew , unexpected_parts_nonnew_rows ,
parts_to_fetch . size ( ) , parts_to_fetch_blocks ) ;
2018-02-21 17:06:29 +00:00
}
2017-04-01 07:20:54 +00:00
2019-03-14 14:46:28 +00:00
if ( unexpected_parts_nonnew_rows > 0 )
2020-11-10 18:22:26 +00:00
{
LOG_WARNING ( log , sanity_report_fmt , getStorageID ( ) . getNameForLogs ( ) ,
formatReadableQuantity ( unexpected_parts_rows ) , formatReadableQuantity ( total_rows_on_filesystem ) ,
unexpected_parts . size ( ) , unexpected_parts_rows , unexpected_parts_nonnew , unexpected_parts_nonnew_rows ,
parts_to_fetch . size ( ) , parts_to_fetch_blocks ) ;
}
2017-04-01 07:20:54 +00:00
2018-12-11 13:30:20 +00:00
/// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them.
2021-10-18 20:16:02 +00:00
queue . setBrokenPartsToEnqueueFetchesOnLoading ( std : : move ( parts_to_fetch ) ) ;
2018-12-11 13:30:20 +00:00
2017-04-01 07:20:54 +00:00
/// Remove extra local parts.
2019-05-03 02:00:57 +00:00
for ( const DataPartPtr & part : unexpected_parts )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Renaming unexpected part {} to ignored_{} " , part - > name , part - > name ) ;
2019-05-20 16:24:36 +00:00
forgetPartAndMoveToDetached ( part , " ignored " , true ) ;
2017-04-01 07:20:54 +00:00
}
2014-04-02 07:59:43 +00:00
}
2014-03-21 19:17:59 +00:00
2014-10-18 17:37:55 +00:00
2020-11-24 14:24:48 +00:00
void StorageReplicatedMergeTree : : syncPinnedPartUUIDs ( )
{
auto zookeeper = getZooKeeper ( ) ;
Coordination : : Stat stat ;
String s = zookeeper - > get ( zookeeper_path + " /pinned_part_uuids " , & stat ) ;
std : : lock_guard lock ( pinned_part_uuids_mutex ) ;
/// Unsure whether or not this can be called concurrently.
if ( pinned_part_uuids - > stat . version < stat . version )
{
auto new_pinned_part_uuids = std : : make_shared < PinnedPartUUIDs > ( ) ;
new_pinned_part_uuids - > fromString ( s ) ;
new_pinned_part_uuids - > stat = stat ;
pinned_part_uuids = new_pinned_part_uuids ;
}
}
2018-03-21 23:30:20 +00:00
void StorageReplicatedMergeTree : : checkPartChecksumsAndAddCommitOps ( const zkutil : : ZooKeeperPtr & zookeeper ,
2019-05-03 02:00:57 +00:00
const DataPartPtr & part , Coordination : : Requests & ops , String part_name , NameSet * absent_replicas_paths )
2014-04-08 17:45:21 +00:00
{
2017-04-01 07:20:54 +00:00
if ( part_name . empty ( ) )
part_name = part - > name ;
2018-12-11 13:30:20 +00:00
auto local_part_header = ReplicatedMergeTreePartHeader : : fromColumnsAndChecksums (
2020-01-16 16:15:01 +00:00
part - > getColumns ( ) , part - > checksums ) ;
2018-12-11 13:30:20 +00:00
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2019-07-28 15:30:38 +00:00
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
2018-12-11 13:30:20 +00:00
bool has_been_already_added = false ;
2017-04-01 07:20:54 +00:00
for ( const String & replica : replicas )
{
2021-05-08 10:59:55 +00:00
String current_part_path = fs : : path ( zookeeper_path ) / " replicas " / replica / " parts " / part_name ;
2018-03-21 23:30:20 +00:00
2018-12-11 13:30:20 +00:00
String part_zk_str ;
if ( ! zookeeper - > tryGet ( current_part_path , part_zk_str ) )
2018-03-21 23:30:20 +00:00
{
if ( absent_replicas_paths )
absent_replicas_paths - > emplace ( current_part_path ) ;
2017-04-01 07:20:54 +00:00
continue ;
2018-03-21 23:30:20 +00:00
}
2018-12-11 13:30:20 +00:00
ReplicatedMergeTreePartHeader replica_part_header ;
2021-08-18 09:49:22 +00:00
if ( part_zk_str . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2018-12-11 13:30:20 +00:00
String columns_str ;
String checksums_str ;
2021-08-18 09:49:22 +00:00
if ( zookeeper - > tryGet ( fs : : path ( current_part_path ) / " columns " , columns_str ) & &
zookeeper - > tryGet ( fs : : path ( current_part_path ) / " checksums " , checksums_str ) )
{
replica_part_header = ReplicatedMergeTreePartHeader : : fromColumnsAndChecksumsZNodes ( columns_str , checksums_str ) ;
}
else
2018-12-11 13:30:20 +00:00
{
2021-08-18 09:49:22 +00:00
if ( zookeeper - > exists ( current_part_path ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Part {} has empty header and does not have columns and checksums. "
" Looks like a bug. " , current_part_path ) ;
LOG_INFO ( log , " Not checking checksums of part {} with replica {} because part was removed from ZooKeeper " , part_name , replica ) ;
2018-12-11 13:30:20 +00:00
continue ;
}
2021-08-18 09:49:22 +00:00
}
else
{
replica_part_header = ReplicatedMergeTreePartHeader : : fromString ( part_zk_str ) ;
2017-04-01 07:20:54 +00:00
}
2018-03-21 23:30:20 +00:00
2018-12-11 13:30:20 +00:00
if ( replica_part_header . getColumnsHash ( ) ! = local_part_header . getColumnsHash ( ) )
2017-04-01 07:20:54 +00:00
{
2021-08-18 09:49:22 +00:00
/// Either it's a bug or ZooKeeper contains broken data.
/// TODO Fix KILL MUTATION and replace CHECKSUM_DOESNT_MATCH with LOGICAL_ERROR
/// (some replicas may skip killed mutation even if it was executed on other replicas)
throw Exception ( ErrorCodes : : CHECKSUM_DOESNT_MATCH , " Part {} from {} has different columns hash " , part_name , replica ) ;
2017-04-01 07:20:54 +00:00
}
2018-12-11 13:30:20 +00:00
replica_part_header . getChecksums ( ) . checkEqual ( local_part_header . getChecksums ( ) , true ) ;
2018-03-21 23:30:20 +00:00
if ( replica = = replica_name )
2018-12-11 13:30:20 +00:00
has_been_already_added = true ;
2018-05-17 22:57:31 +00:00
/// If we verify checksums in "sequential manner" (i.e. recheck absence of checksums on other replicas when commit)
/// then it is enough to verify checksums on at least one replica since checksums on other replicas must be the same.
if ( absent_replicas_paths )
{
absent_replicas_paths - > clear ( ) ;
break ;
}
2017-04-01 07:20:54 +00:00
}
2018-12-11 13:30:20 +00:00
if ( ! has_been_already_added )
2017-04-01 07:20:54 +00:00
{
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2018-03-21 23:30:20 +00:00
2020-01-30 12:54:52 +00:00
//ops.emplace_back(zkutil::makeCheckRequest(
// zookeeper_path + "/columns", expected_columns_version));
2018-12-11 13:30:20 +00:00
2019-08-26 18:08:58 +00:00
if ( storage_settings_ptr - > use_minimalistic_part_header_in_zookeeper )
2018-12-11 13:30:20 +00:00
{
ops . emplace_back ( zkutil : : makeCreateRequest (
part_path , local_part_header . toString ( ) , zkutil : : CreateMode : : Persistent ) ) ;
}
else
{
ops . emplace_back ( zkutil : : makeCreateRequest (
part_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( part_path ) / " columns " , part - > getColumns ( ) . toString ( ) , zkutil : : CreateMode : : Persistent ) ) ;
2018-12-11 13:30:20 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( part_path ) / " checksums " , getChecksumsForZooKeeper ( part - > checksums ) , zkutil : : CreateMode : : Persistent ) ) ;
2018-12-11 13:30:20 +00:00
}
2018-03-21 23:30:20 +00:00
}
else
{
2021-05-08 10:59:55 +00:00
LOG_WARNING ( log , " checkPartAndAddToZooKeeper: node {} already exists. Will not commit any nodes. " ,
( fs : : path ( replica_path ) / " parts " / part_name ) . string ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-03-21 23:30:20 +00:00
}
2017-04-01 07:20:54 +00:00
2019-05-03 02:00:57 +00:00
MergeTreeData : : DataPartsVector StorageReplicatedMergeTree : : checkPartChecksumsAndCommit ( Transaction & transaction ,
const DataPartPtr & part )
2018-03-21 23:30:20 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
2018-03-21 23:30:20 +00:00
while ( true )
{
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2018-03-21 23:30:20 +00:00
NameSet absent_part_paths_on_replicas ;
/// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`.
checkPartChecksumsAndAddCommitOps ( zookeeper , part , ops , part - > name , & absent_part_paths_on_replicas ) ;
/// Do not commit if the part is obsolete, we have just briefly checked its checksums
if ( transaction . isEmpty ( ) )
return { } ;
/// Will check that the part did not suddenly appear on skipped replicas
if ( ! absent_part_paths_on_replicas . empty ( ) )
{
2018-08-25 01:58:14 +00:00
Coordination : : Requests new_ops ;
2018-03-21 23:30:20 +00:00
for ( const String & part_path : absent_part_paths_on_replicas )
{
2018-03-24 00:45:04 +00:00
new_ops . emplace_back ( zkutil : : makeCreateRequest ( part_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
new_ops . emplace_back ( zkutil : : makeRemoveRequest ( part_path , - 1 ) ) ;
2018-03-21 23:30:20 +00:00
}
/// Add check ops at the beginning
new_ops . insert ( new_ops . end ( ) , ops . begin ( ) , ops . end ( ) ) ;
ops = std : : move ( new_ops ) ;
}
try
{
zookeeper - > multi ( ops ) ;
return transaction . commit ( ) ;
}
2018-03-24 20:00:16 +00:00
catch ( const zkutil : : KeeperMultiException & e )
2018-03-21 23:30:20 +00:00
{
size_t num_check_ops = 2 * absent_part_paths_on_replicas . size ( ) ;
2018-03-24 20:00:16 +00:00
size_t failed_op_index = e . failed_op_index ;
2018-03-21 23:30:20 +00:00
2020-06-12 15:09:12 +00:00
if ( failed_op_index < num_check_ops & & e . code = = Coordination : : Error : : ZNODEEXISTS )
2018-03-21 23:30:20 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " The part {} on a replica suddenly appeared, will recheck checksums " , e . getPathForFirstFailedOp ( ) ) ;
2018-03-21 23:30:20 +00:00
}
else
throw ;
}
}
}
2018-05-21 13:49:54 +00:00
String StorageReplicatedMergeTree : : getChecksumsForZooKeeper ( const MergeTreeDataPartChecksums & checksums ) const
2018-03-21 23:30:20 +00:00
{
return MinimalisticDataPartChecksums : : getSerializedString ( checksums ,
2019-08-26 14:24:29 +00:00
getSettings ( ) - > use_minimalistic_checksums_in_zookeeper ) ;
2014-04-08 17:45:21 +00:00
}
2021-02-16 13:19:21 +00:00
MergeTreeData : : MutableDataPartPtr StorageReplicatedMergeTree : : attachPartHelperFoundValidPart ( const LogEntry & entry ) const
2021-02-15 15:06:48 +00:00
{
2021-03-16 21:59:12 +00:00
const MergeTreePartInfo actual_part_info = MergeTreePartInfo : : fromPartName ( entry . new_part_name , format_version ) ;
const String part_new_name = actual_part_info . getPartName ( ) ;
2021-02-15 15:06:48 +00:00
2021-03-16 21:59:12 +00:00
for ( const DiskPtr & disk : getStoragePolicy ( ) - > getDisks ( ) )
2021-05-08 10:59:55 +00:00
for ( const auto it = disk - > iterateDirectory ( fs : : path ( relative_data_path ) / " detached/ " ) ; it - > isValid ( ) ; it - > next ( ) )
2021-02-15 15:06:48 +00:00
{
2021-08-24 12:57:49 +00:00
const auto part_info = MergeTreePartInfo : : tryParsePartName ( it - > name ( ) , format_version ) ;
2021-02-19 14:28:29 +00:00
2021-08-24 12:57:49 +00:00
if ( ! part_info | | part_info - > partition_id ! = actual_part_info . partition_id )
2021-02-15 15:06:48 +00:00
continue ;
2021-08-24 12:57:49 +00:00
const String part_old_name = part_info - > getPartName ( ) ;
2021-05-08 10:59:55 +00:00
const String part_path = fs : : path ( " detached " ) / part_old_name ;
2021-03-15 15:44:15 +00:00
2021-03-16 21:59:12 +00:00
const VolumePtr volume = std : : make_shared < SingleDiskVolume > ( " volume_ " + part_old_name , disk ) ;
2021-03-30 22:49:15 +00:00
/// actual_part_info is more recent than part_info so we use it
MergeTreeData : : MutableDataPartPtr part = createPart ( part_new_name , actual_part_info , volume , part_path ) ;
2021-03-15 15:44:15 +00:00
2021-03-22 14:52:21 +00:00
try
{
2021-03-22 15:44:44 +00:00
part - > loadColumnsChecksumsIndexes ( true , true ) ;
2021-03-22 14:52:21 +00:00
}
2021-03-22 15:44:44 +00:00
catch ( const Exception & )
2021-03-22 14:52:21 +00:00
{
2021-03-22 15:44:44 +00:00
/// This method throws if the part data is corrupted or partly missing. In this case, we simply don't
/// process the part.
2021-03-22 14:52:21 +00:00
continue ;
}
2021-03-17 16:22:56 +00:00
2021-03-16 21:59:12 +00:00
if ( entry . part_checksum = = part - > checksums . getTotalChecksumHex ( ) )
2021-03-17 15:49:04 +00:00
{
2021-03-22 13:27:35 +00:00
part - > modification_time = disk - > getLastModified ( part - > getFullRelativePath ( ) ) . epochTime ( ) ;
2021-03-16 21:59:12 +00:00
return part ;
2021-03-17 15:49:04 +00:00
}
2021-02-15 15:06:48 +00:00
}
return { } ;
}
2018-05-23 14:33:55 +00:00
bool StorageReplicatedMergeTree : : executeLogEntry ( LogEntry & entry )
2014-04-03 11:48:28 +00:00
{
2017-04-01 07:20:54 +00:00
if ( entry . type = = LogEntry : : DROP_RANGE )
{
executeDropRange ( entry ) ;
return true ;
}
2018-05-21 13:49:54 +00:00
if ( entry . type = = LogEntry : : REPLACE_RANGE )
{
executeReplaceRange ( entry ) ;
return true ;
}
2021-02-16 15:36:30 +00:00
const bool is_get_or_attach = entry . type = = LogEntry : : GET_PART | | entry . type = = LogEntry : : ATTACH_PART ;
2021-02-19 00:24:53 +00:00
if ( is_get_or_attach | | entry . type = = LogEntry : : MERGE_PARTS | | entry . type = = LogEntry : : MUTATE_PART )
2017-04-01 07:20:54 +00:00
{
/// If we already have this part or a part covering it, we do not need to do anything.
2017-12-20 17:40:36 +00:00
/// The part may be still in the PreCommitted -> Committed transition so we first search
/// among PreCommitted parts to definitely find the desired part if it exists.
2019-05-03 02:00:57 +00:00
DataPartPtr existing_part = getPartIfExists ( entry . new_part_name , { MergeTreeDataPartState : : PreCommitted } ) ;
2021-02-16 15:36:30 +00:00
2017-12-19 14:55:50 +00:00
if ( ! existing_part )
2019-05-03 02:00:57 +00:00
existing_part = getActiveContainingPart ( entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
2021-02-14 22:59:13 +00:00
/// Even if the part is local, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there.
2021-05-08 10:59:55 +00:00
if ( existing_part & & getZooKeeper ( ) - > exists ( fs : : path ( replica_path ) / " parts " / existing_part - > name ) )
2017-04-01 07:20:54 +00:00
{
2021-02-16 15:36:30 +00:00
if ( ! is_get_or_attach | | entry . source_replica ! = replica_name )
2021-02-15 15:06:48 +00:00
LOG_DEBUG ( log , " Skipping action for part {} because part {} already exists. " ,
entry . new_part_name , existing_part - > name ) ;
2021-02-16 15:36:30 +00:00
2017-04-01 07:20:54 +00:00
return true ;
}
}
2021-03-15 15:44:15 +00:00
if ( entry . type = = LogEntry : : ATTACH_PART )
{
if ( MutableDataPartPtr part = attachPartHelperFoundValidPart ( entry ) ; part )
{
2021-06-20 08:24:43 +00:00
LOG_TRACE ( log , " Found valid local part for {}, preparing the transaction " , part - > name ) ;
2021-03-15 15:44:15 +00:00
Transaction transaction ( * this ) ;
2021-03-30 22:49:15 +00:00
renameTempPartAndReplace ( part , nullptr , & transaction ) ;
2021-03-15 15:44:15 +00:00
checkPartChecksumsAndCommit ( transaction , part ) ;
2021-03-22 13:27:35 +00:00
writePartLog ( PartLogElement : : Type : : NEW_PART , { } , 0 /** log entry is fake so we don't measure the time */ ,
part - > name , part , { } /** log entry is fake so there are no initial parts */ , nullptr ) ;
2021-03-17 15:49:04 +00:00
2021-03-15 15:44:15 +00:00
return true ;
}
2021-06-20 08:24:43 +00:00
LOG_TRACE ( log , " Didn't find valid local part for {} ({}), will fetch it from other replica " ,
entry . new_part_name ,
entry . actual_new_part_name ) ;
2021-03-15 15:44:15 +00:00
}
2021-02-16 15:36:30 +00:00
if ( is_get_or_attach & & entry . source_replica = = replica_name )
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Part {} from own log doesn't exist. " , entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
2021-02-15 15:06:48 +00:00
/// Perhaps we don't need this part, because during write with quorum, the quorum has failed
/// (see below about `/quorum/failed_parts`).
2021-05-08 10:59:55 +00:00
if ( entry . quorum & & getZooKeeper ( ) - > exists ( fs : : path ( zookeeper_path ) / " quorum " / " failed_parts " / entry . new_part_name ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Skipping action for part {} because quorum for that part was failed. " , entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
return true ; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge.
}
2021-09-16 21:19:58 +00:00
// bool do_fetch = false;
2021-01-12 18:46:03 +00:00
2021-02-14 22:59:13 +00:00
switch ( entry . type )
{
2021-02-15 15:06:48 +00:00
case LogEntry : : ATTACH_PART :
2021-02-14 22:59:13 +00:00
/// We surely don't have this part locally as we've checked it before, so download it.
2021-02-15 15:06:48 +00:00
[[fallthrough]] ;
case LogEntry : : GET_PART :
2021-09-16 21:19:58 +00:00
return executeFetch ( entry ) ;
// do_fetch = true;
2021-02-14 22:59:13 +00:00
case LogEntry : : MERGE_PARTS :
2021-09-16 21:19:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Merge has to be executed by another function " ) ;
2021-02-14 22:59:13 +00:00
case LogEntry : : MUTATE_PART :
2021-09-16 21:19:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mutation has to be executed by another function " ) ;
2021-02-14 22:59:13 +00:00
case LogEntry : : ALTER_METADATA :
return executeMetadataAlter ( entry ) ;
2020-11-24 14:24:48 +00:00
case LogEntry : : SYNC_PINNED_PART_UUIDS :
syncPinnedPartUUIDs ( ) ;
return true ;
case LogEntry : : CLONE_PART_FROM_SHARD :
executeClonePartFromShard ( entry ) ;
return true ;
2021-02-14 22:59:13 +00:00
default :
2021-03-22 13:27:35 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Unexpected log entry type: {} " , static_cast < int > ( entry . type ) ) ;
2021-02-14 22:59:13 +00:00
}
2018-01-23 22:56:46 +00:00
2021-09-16 21:19:58 +00:00
// return true;
2018-01-23 22:56:46 +00:00
}
2017-04-01 07:20:54 +00:00
2018-05-23 14:33:55 +00:00
bool StorageReplicatedMergeTree : : executeFetch ( LogEntry & entry )
2018-01-23 22:56:46 +00:00
{
2020-02-17 16:33:05 +00:00
/// Looking for covering part. After that entry.actual_new_part_name may be filled.
2018-01-23 22:56:46 +00:00
String replica = findReplicaHavingCoveringPart ( entry , true ) ;
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2020-06-26 11:30:23 +00:00
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
try
{
if ( replica . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2018-01-23 22:56:46 +00:00
/** If a part is to be written with a quorum and the quorum is not reached yet,
* then ( due to the fact that a part is impossible to download right now ) ,
* the quorum entry should be considered unsuccessful .
* TODO Complex code , extract separately .
*/
if ( entry . quorum )
2017-04-01 07:20:54 +00:00
{
2018-01-23 22:56:46 +00:00
if ( entry . type ! = LogEntry : : GET_PART )
throw Exception ( " Logical error: log entry with quorum but type is not GET_PART " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " No active replica has part {} which needs to be written with quorum. Will try to mark that quorum as failed. " , entry . new_part_name ) ;
2018-01-23 22:56:46 +00:00
/** Atomically:
* - if replicas do not become active ;
* - if there is a ` quorum ` node with this part ;
* - delete ` quorum ` node ;
* - add a part to the list ` quorum / failed_parts ` ;
* - if the part is not already removed from the list for deduplication ` blocks / block_num ` , then delete it ;
*
* If something changes , then we will nothing - we ' ll get here again next time .
2017-04-01 07:20:54 +00:00
*/
2018-01-23 22:56:46 +00:00
/** We collect the `host` node versions from the replicas.
* When the replica becomes active , it changes the value of host in the same transaction ( with the creation of ` is_active ` ) .
* This will ensure that the replicas do not become active .
*/
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
auto zookeeper = getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2017-04-01 07:20:54 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2017-04-01 07:20:54 +00:00
2020-03-09 01:22:33 +00:00
for ( const auto & path_part : replicas )
2018-01-23 22:56:46 +00:00
{
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2021-05-08 10:59:55 +00:00
String path = fs : : path ( zookeeper_path ) / " replicas " / path_part / " host " ;
2018-01-23 22:56:46 +00:00
zookeeper - > get ( path , & stat ) ;
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeCheckRequest ( path , stat . version ) ) ;
2018-01-23 22:56:46 +00:00
}
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
/// We verify that while we were collecting versions, the replica with the necessary part did not come alive.
replica = findReplicaHavingPart ( entry . new_part_name , true ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
/// Also during this time a completely new replica could be created.
/// But if a part does not appear on the old, then it can not be on the new one either.
if ( replica . empty ( ) )
{
2018-08-25 01:58:14 +00:00
Coordination : : Stat quorum_stat ;
2021-05-08 10:59:55 +00:00
const String quorum_unparallel_path = fs : : path ( zookeeper_path ) / " quorum " / " status " ;
const String quorum_parallel_path = fs : : path ( zookeeper_path ) / " quorum " / " parallel " / entry . new_part_name ;
2020-10-06 21:49:48 +00:00
String quorum_str , quorum_path ;
2018-01-23 22:56:46 +00:00
ReplicatedMergeTreeQuorumEntry quorum_entry ;
2020-10-06 21:49:48 +00:00
if ( zookeeper - > tryGet ( quorum_unparallel_path , quorum_str , & quorum_stat ) )
quorum_path = quorum_unparallel_path ;
2020-10-06 22:36:42 +00:00
else
{
2020-10-06 21:49:48 +00:00
quorum_str = zookeeper - > get ( quorum_parallel_path , & quorum_stat ) ;
quorum_path = quorum_parallel_path ;
}
2018-01-23 22:56:46 +00:00
quorum_entry . fromString ( quorum_str ) ;
if ( quorum_entry . part_name = = entry . new_part_name )
2017-04-01 07:20:54 +00:00
{
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( quorum_path , quorum_stat . version ) ) ;
2019-05-03 02:00:57 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( entry . new_part_name , format_version ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
if ( part_info . min_block ! = part_info . max_block )
throw Exception ( " Logical error: log entry with quorum for part covering more than one block number " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +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 ( zookeeper_path ) / " quorum " / " failed_parts " / entry . new_part_name ,
2018-01-23 22:56:46 +00:00
" " ,
zkutil : : CreateMode : : Persistent ) ) ;
/// Deleting from `blocks`.
2021-05-08 10:59:55 +00:00
if ( ! entry . block_id . empty ( ) & & zookeeper - > exists ( fs : : path ( zookeeper_path ) / " blocks " / entry . block_id ) )
ops . emplace_back ( zkutil : : makeRemoveRequest ( fs : : path ( zookeeper_path ) / " blocks " / entry . block_id , - 1 ) ) ;
2018-01-23 22:56:46 +00:00
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 ) ;
2018-01-23 22:56:46 +00:00
2020-06-12 15:09:12 +00:00
if ( code = = Coordination : : Error : : ZOK )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Marked quorum for part {} as failed. " , entry . new_part_name ) ;
2021-05-30 21:29:37 +00:00
queue . removeFailedQuorumPart ( part_info ) ;
2018-05-10 15:01:10 +00:00
return true ;
2017-04-01 07:20:54 +00:00
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZBADVERSION | | code = = Coordination : : Error : : ZNONODE | | code = = Coordination : : Error : : ZNODEEXISTS )
2017-04-01 07:20:54 +00:00
{
2021-05-08 10:59:55 +00:00
LOG_DEBUG ( log , " State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {} " ,
entry . new_part_name , Coordination : : errorMessage ( code ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-01-23 22:56:46 +00:00
else
2018-08-25 01:58:14 +00:00
throw Coordination : : Exception ( code ) ;
2018-01-23 22:56:46 +00:00
}
else
{
2021-05-30 21:29:37 +00:00
LOG_WARNING ( log , " No active replica has part {}, "
" but that part needs quorum and /quorum/status contains entry about another part {}. "
" It means that part was successfully written to {} replicas, but then all of them goes offline. "
" Or it is a bug. " , entry . new_part_name , quorum_entry . part_name , entry . quorum ) ;
2017-04-01 07:20:54 +00:00
}
}
}
2018-01-23 22:56:46 +00:00
if ( replica . empty ( ) )
2017-04-06 13:03:23 +00:00
{
2018-01-23 22:56:46 +00:00
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartFailedFetches ) ;
throw Exception ( " No active replica has part " + entry . new_part_name + " or covering part " , ErrorCodes : : NO_REPLICA_HAS_PART ) ;
2017-04-06 13:03:23 +00:00
}
2018-01-23 22:56:46 +00:00
}
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
try
{
2020-02-13 20:09:48 +00:00
String part_name = entry . actual_new_part_name . empty ( ) ? entry . new_part_name : entry . actual_new_part_name ;
2021-07-05 19:58:36 +00:00
if ( ! entry . actual_new_part_name . empty ( ) )
LOG_DEBUG ( log , " Will fetch part {} instead of {} " , entry . actual_new_part_name , entry . new_part_name ) ;
2021-05-08 10:59:55 +00:00
if ( ! fetchPart ( part_name , metadata_snapshot , fs : : path ( zookeeper_path ) / " replicas " / replica , false , entry . quorum ) )
2018-01-23 22:56:46 +00:00
return false ;
2017-04-01 07:20:54 +00:00
}
2018-01-23 22:56:46 +00:00
catch ( Exception & e )
2017-04-01 07:20:54 +00:00
{
2018-01-23 22:56:46 +00:00
/// No stacktrace, just log message
if ( e . code ( ) = = ErrorCodes : : RECEIVED_ERROR_TOO_MANY_REQUESTS )
e . addMessage ( " Too busy replica. Will try later. " ) ;
throw ;
}
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
if ( entry . type = = LogEntry : : MERGE_PARTS )
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartFetchesOfMerged ) ;
}
catch ( . . . )
{
2020-04-20 04:19:50 +00:00
/** If we can not download the part we need for some merge, it's better not to try to get other parts for this merge,
2018-01-23 22:56:46 +00:00
* but try to get already merged part . To do this , move the action to get the remaining parts
* for this merge at the end of the queue .
*/
try
{
auto parts_for_merge = queue . moveSiblingPartsForMergeToEndOfQueue ( entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
if ( ! parts_for_merge . empty ( ) & & replica . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " No active replica has part {}. Will fetch merged part instead. " , entry . new_part_name ) ;
2021-08-18 09:49:22 +00:00
/// We should enqueue it for check, because merged part may never appear if source part is lost
enqueuePartForCheck ( entry . new_part_name ) ;
2018-01-23 22:56:46 +00:00
return false ;
2017-04-01 07:20:54 +00:00
}
2018-01-23 22:56:46 +00:00
/** If no active replica has a part, and there is no merge in the queue with its participation,
* check to see if any ( active or inactive ) replica has such a part or covering it .
*/
if ( replica . empty ( ) )
enqueuePartForCheck ( entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
}
2018-01-23 22:56:46 +00:00
catch ( . . . )
{
2018-04-06 21:46:57 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2018-01-23 22:56:46 +00:00
}
throw ;
2017-04-01 07:20:54 +00:00
}
return true ;
2014-04-03 11:48:28 +00:00
}
2014-10-18 17:37:55 +00:00
2021-03-04 23:10:20 +00:00
bool StorageReplicatedMergeTree : : executeFetchShared (
const String & source_replica ,
const String & new_part_name ,
const DiskPtr & disk ,
const String & path )
2021-01-14 16:26:56 +00:00
{
2021-03-04 23:10:20 +00:00
if ( source_replica . empty ( ) )
2021-01-14 16:26:56 +00:00
{
2021-07-05 03:32:56 +00:00
LOG_INFO ( log , " No active replica has part {} on shared storage. " , new_part_name ) ;
2021-01-14 16:26:56 +00:00
return false ;
}
const auto storage_settings_ptr = getSettings ( ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
try
{
2021-05-08 10:59:55 +00:00
if ( ! fetchExistsPart ( new_part_name , metadata_snapshot , fs : : path ( zookeeper_path ) / " replicas " / source_replica , disk , path ) )
2021-01-14 16:26:56 +00:00
return false ;
}
catch ( Exception & e )
{
if ( e . code ( ) = = ErrorCodes : : RECEIVED_ERROR_TOO_MANY_REQUESTS )
e . addMessage ( " Too busy replica. Will try later. " ) ;
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
throw ;
}
return true ;
}
2018-05-23 14:33:55 +00:00
void StorageReplicatedMergeTree : : executeDropRange ( const LogEntry & entry )
2014-08-05 13:49:44 +00:00
{
2021-10-27 12:59:26 +00:00
LOG_TRACE ( log , " Executing DROP_RANGE {} " , entry . new_part_name ) ;
2019-05-03 02:00:57 +00:00
auto drop_range_info = MergeTreePartInfo : : fromPartName ( entry . new_part_name , format_version ) ;
2021-06-24 14:07:43 +00:00
getContext ( ) - > getMergeList ( ) . cancelInPartition ( getStorageID ( ) , drop_range_info . partition_id , drop_range_info . max_block ) ;
2021-10-27 12:59:26 +00:00
part_check_thread . cancelRemovedPartsCheck ( drop_range_info ) ;
2021-06-23 19:24:43 +00:00
queue . removePartProducingOpsInRange ( getZooKeeper ( ) , drop_range_info , entry ) ;
2017-11-20 19:33:12 +00:00
2021-06-23 20:57:49 +00:00
/// Delete the parts contained in the range to be deleted.
/// It's important that no old parts remain (after the merge), because otherwise,
/// after adding a new replica, this new replica downloads them, but does not delete them.
/// And, if you do not, the parts will come to life after the server is restarted.
/// Therefore, we use all data parts.
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2019-05-03 02:00:57 +00:00
DataPartsVector parts_to_remove ;
2017-04-01 07:20:54 +00:00
{
2019-05-03 02:00:57 +00:00
auto data_parts_lock = lockParts ( ) ;
2021-05-13 14:04:36 +00:00
parts_to_remove = removePartsInRangeFromWorkingSet ( drop_range_info , true , data_parts_lock ) ;
2021-06-23 20:57:49 +00:00
if ( parts_to_remove . empty ( ) )
2021-09-10 14:32:45 +00:00
{
if ( ! drop_range_info . isFakeDropRangePart ( ) )
LOG_INFO ( log , " Log entry {} tried to drop single part {}, but part does not exist " , entry . znode_name , entry . new_part_name ) ;
2021-06-23 20:57:49 +00:00
return ;
2021-09-10 14:32:45 +00:00
}
2018-05-21 13:49:54 +00:00
}
2017-04-01 07:20:54 +00:00
2021-06-23 20:57:49 +00:00
if ( entry . detach )
LOG_DEBUG ( log , " Detaching parts. " ) ;
else
LOG_DEBUG ( log , " Removing parts. " ) ;
2018-05-21 13:49:54 +00:00
if ( entry . detach )
{
/// If DETACH clone parts to detached/ directory
for ( const auto & part : parts_to_remove )
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Detaching {} " , part - > relative_path ) ;
2020-06-26 11:30:23 +00:00
part - > makeCloneInDetached ( " " , metadata_snapshot ) ;
2018-05-21 13:49:54 +00:00
}
}
2017-04-01 07:20:54 +00:00
2018-05-21 13:49:54 +00:00
/// Forcibly remove parts from ZooKeeper
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove ) ;
2017-04-01 07:20:54 +00:00
2020-05-23 20:37:37 +00:00
if ( entry . detach )
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Detached {} parts inside {}. " , parts_to_remove . size ( ) , entry . new_part_name ) ;
2020-05-23 20:37:37 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed {} parts inside {}. " , parts_to_remove . size ( ) , entry . new_part_name ) ;
2017-04-01 07:20:54 +00:00
2018-05-21 13:49:54 +00:00
/// We want to remove dropped parts from disk as soon as possible
/// To be removed a partition should have zero refcount, therefore call the cleanup thread at exit
parts_to_remove . clear ( ) ;
2018-07-30 17:34:55 +00:00
cleanup_thread . wakeup ( ) ;
2014-08-08 08:28:13 +00:00
}
2014-10-18 17:37:55 +00:00
2018-05-23 14:33:55 +00:00
bool StorageReplicatedMergeTree : : executeReplaceRange ( const LogEntry & entry )
2014-04-03 11:48:28 +00:00
{
2018-05-21 13:49:54 +00:00
Stopwatch watch ;
auto & entry_replace = * entry . replace_range_entry ;
2021-05-13 11:29:59 +00:00
LOG_DEBUG ( log , " Executing log entry {} to replace parts range {} with {} parts from {}.{} " ,
entry . znode_name , entry_replace . drop_range_part_name , entry_replace . new_part_names . size ( ) ,
entry_replace . from_database , entry_replace . from_table ) ;
2020-06-26 11:30:23 +00:00
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2017-04-01 07:20:54 +00:00
2019-05-03 02:00:57 +00:00
MergeTreePartInfo drop_range = MergeTreePartInfo : : fromPartName ( entry_replace . drop_range_part_name , format_version ) ;
2021-05-13 14:23:00 +00:00
/// Range with only one block has special meaning: it's ATTACH PARTITION or MOVE PARTITION, so there is no drop range
2021-05-14 16:11:40 +00:00
bool replace = ! LogEntry : : ReplaceRangeEntry : : isMovePartitionOrAttachFrom ( drop_range ) ;
2018-05-21 13:49:54 +00:00
2021-05-13 14:23:00 +00:00
if ( replace )
2021-06-24 14:07:43 +00:00
{
getContext ( ) - > getMergeList ( ) . cancelInPartition ( getStorageID ( ) , drop_range . partition_id , drop_range . max_block ) ;
2021-10-27 12:59:26 +00:00
part_check_thread . cancelRemovedPartsCheck ( drop_range ) ;
2021-05-13 14:23:00 +00:00
queue . removePartProducingOpsInRange ( getZooKeeper ( ) , drop_range , entry ) ;
2021-06-24 14:07:43 +00:00
}
2021-05-13 14:23:00 +00:00
else
2021-06-24 14:07:43 +00:00
{
2021-05-13 14:23:00 +00:00
drop_range = { } ;
2021-06-24 14:07:43 +00:00
}
2018-05-21 13:49:54 +00:00
struct PartDescription
2017-11-17 08:58:35 +00:00
{
2021-04-10 23:33:54 +00:00
PartDescription (
size_t index_ ,
const String & src_part_name_ ,
const String & new_part_name_ ,
const String & checksum_hex_ ,
MergeTreeDataFormatVersion format_version )
: index ( index_ )
, src_part_name ( src_part_name_ )
, src_part_info ( MergeTreePartInfo : : fromPartName ( src_part_name_ , format_version ) )
, new_part_name ( new_part_name_ )
, new_part_info ( MergeTreePartInfo : : fromPartName ( new_part_name_ , format_version ) )
, checksum_hex ( checksum_hex_ )
{
}
2018-05-21 13:49:54 +00:00
size_t index ; // in log entry arrays
String src_part_name ;
MergeTreePartInfo src_part_info ;
String new_part_name ;
MergeTreePartInfo new_part_info ;
String checksum_hex ;
2019-01-22 19:56:53 +00:00
/// Part which will be committed
2019-05-03 02:00:57 +00:00
MutableDataPartPtr res_part ;
2018-05-21 13:49:54 +00:00
/// We could find a covering part
MergeTreePartInfo found_new_part_info ;
String found_new_part_name ;
/// Hold pointer to part in source table if will clone it from local table
2019-05-03 02:00:57 +00:00
DataPartPtr src_table_part ;
2018-05-21 13:49:54 +00:00
/// A replica that will be used to fetch part
String replica ;
} ;
using PartDescriptionPtr = std : : shared_ptr < PartDescription > ;
using PartDescriptions = std : : vector < PartDescriptionPtr > ;
PartDescriptions all_parts ;
PartDescriptions parts_to_add ;
2019-05-03 02:00:57 +00:00
DataPartsVector parts_to_remove ;
2018-05-21 13:49:54 +00:00
2020-06-18 16:10:47 +00:00
auto table_lock_holder_dst_table = lockForShare (
RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2020-06-17 10:34:23 +00:00
auto dst_metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2018-05-21 13:49:54 +00:00
for ( size_t i = 0 ; i < entry_replace . new_part_names . size ( ) ; + + i )
{
all_parts . emplace_back ( std : : make_shared < PartDescription > ( i ,
entry_replace . src_part_names . at ( i ) ,
entry_replace . new_part_names . at ( i ) ,
entry_replace . part_names_checksums . at ( i ) ,
2019-05-03 02:00:57 +00:00
format_version ) ) ;
2018-05-21 13:49:54 +00:00
}
2019-01-22 19:56:53 +00:00
/// What parts we should add? Or we have already added all required parts (we an replica-initializer)
2018-05-21 13:49:54 +00:00
{
2019-05-03 02:00:57 +00:00
auto data_parts_lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
for ( const PartDescriptionPtr & part_desc : all_parts )
2017-12-21 18:17:06 +00:00
{
2019-05-03 02:00:57 +00:00
if ( ! getActiveContainingPart ( part_desc - > new_part_info , MergeTreeDataPartState : : Committed , data_parts_lock ) )
2018-05-21 13:49:54 +00:00
parts_to_add . emplace_back ( part_desc ) ;
2017-12-21 18:17:06 +00:00
}
2018-05-21 13:49:54 +00:00
if ( parts_to_add . empty ( ) & & replace )
2021-05-13 11:29:59 +00:00
{
2021-05-13 14:04:36 +00:00
parts_to_remove = removePartsInRangeFromWorkingSet ( drop_range , true , data_parts_lock ) ;
2021-05-13 11:29:59 +00:00
String parts_to_remove_str ;
for ( const auto & part : parts_to_remove )
{
parts_to_remove_str + = part - > name ;
parts_to_remove_str + = " " ;
}
LOG_TRACE ( log , " Replacing {} parts {}with empty set " , parts_to_remove . size ( ) , parts_to_remove_str ) ;
}
2018-05-21 13:49:54 +00:00
}
if ( parts_to_add . empty ( ) )
{
LOG_INFO ( log , " All parts from REPLACE PARTITION command have been already attached " ) ;
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove ) ;
return true ;
}
2018-06-04 19:46:47 +00:00
if ( parts_to_add . size ( ) < all_parts . size ( ) )
2018-05-21 13:49:54 +00:00
{
LOG_WARNING ( log , " Some (but not all) parts from REPLACE PARTITION command already exist. REPLACE PARTITION will not be atomic. " ) ;
}
StoragePtr source_table ;
2020-06-18 16:10:47 +00:00
TableLockHolder table_lock_holder_src_table ;
2020-02-17 19:28:25 +00:00
StorageID source_table_id { entry_replace . from_database , entry_replace . from_table } ;
2018-05-21 13:49:54 +00:00
auto clone_data_parts_from_source_table = [ & ] ( ) - > size_t
{
2021-04-10 23:33:54 +00:00
source_table = DatabaseCatalog : : instance ( ) . tryGetTable ( source_table_id , getContext ( ) ) ;
2018-05-21 13:49:54 +00:00
if ( ! source_table )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Can't use {} as source table for REPLACE PARTITION command. It does not exist. " , source_table_id . getNameForLogs ( ) ) ;
2018-05-21 13:49:54 +00:00
return 0 ;
}
2020-06-17 10:34:23 +00:00
auto src_metadata_snapshot = source_table - > getInMemoryMetadataPtr ( ) ;
2018-05-21 13:49:54 +00:00
MergeTreeData * src_data = nullptr ;
2017-12-21 18:17:06 +00:00
try
{
2020-06-17 10:34:23 +00:00
src_data = & checkStructureAndGetMergeTreeData ( source_table , src_metadata_snapshot , dst_metadata_snapshot ) ;
2017-12-21 18:17:06 +00:00
}
2018-08-10 04:02:56 +00:00
catch ( Exception & )
2018-04-19 18:01:50 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Can't use {} as source table for REPLACE PARTITION command. Will fetch all parts. Reason: {} " , source_table_id . getNameForLogs ( ) , getCurrentExceptionMessage ( false ) ) ;
2018-05-21 13:49:54 +00:00
return 0 ;
}
2018-04-19 18:01:50 +00:00
2020-06-18 16:10:47 +00:00
table_lock_holder_src_table = source_table - > lockForShare (
RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2018-05-21 13:49:54 +00:00
2020-06-18 16:10:47 +00:00
DataPartStates valid_states {
MergeTreeDataPartState : : PreCommitted , MergeTreeDataPartState : : Committed , MergeTreeDataPartState : : Outdated } ;
2018-05-21 13:49:54 +00:00
size_t num_clonable_parts = 0 ;
for ( PartDescriptionPtr & part_desc : parts_to_add )
{
auto src_part = src_data - > getPartIfExists ( part_desc - > src_part_info , valid_states ) ;
if ( ! src_part )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " There is no part {} in {} " , part_desc - > src_part_name , source_table_id . getNameForLogs ( ) ) ;
2018-05-21 13:49:54 +00:00
continue ;
}
2020-03-17 15:10:56 +00:00
String checksum_hex = src_part - > checksums . getTotalChecksumHex ( ) ;
2018-05-21 13:49:54 +00:00
if ( checksum_hex ! = part_desc - > checksum_hex )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Part {} of {} has inappropriate checksum " , part_desc - > src_part_name , source_table_id . getNameForLogs ( ) ) ;
2018-05-21 13:49:54 +00:00
/// TODO: check version
continue ;
}
part_desc - > found_new_part_name = part_desc - > new_part_name ;
part_desc - > found_new_part_info = part_desc - > new_part_info ;
part_desc - > src_table_part = src_part ;
+ + num_clonable_parts ;
2018-04-19 18:01:50 +00:00
}
2018-05-21 13:49:54 +00:00
return num_clonable_parts ;
} ;
size_t num_clonable_parts = clone_data_parts_from_source_table ( ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Found {} parts that could be cloned (of {} required parts) " , num_clonable_parts , parts_to_add . size ( ) ) ;
2018-05-21 13:49:54 +00:00
2019-05-03 02:00:57 +00:00
ActiveDataPartSet adding_parts_active_set ( format_version ) ;
2018-05-21 13:49:54 +00:00
std : : unordered_map < String , PartDescriptionPtr > part_name_to_desc ;
for ( PartDescriptionPtr & part_desc : parts_to_add )
{
if ( part_desc - > src_table_part )
2017-12-21 18:17:06 +00:00
{
2018-05-21 13:49:54 +00:00
/// It is clonable part
2018-05-28 15:37:30 +00:00
adding_parts_active_set . add ( part_desc - > new_part_name ) ;
2018-05-21 13:49:54 +00:00
part_name_to_desc . emplace ( part_desc - > new_part_name , part_desc ) ;
continue ;
2017-12-21 18:17:06 +00:00
}
2018-05-21 13:49:54 +00:00
/// Firstly, try find exact part to produce more accurate part set
String replica = findReplicaHavingPart ( part_desc - > new_part_name , true ) ;
String found_part_name ;
/// TODO: check version
if ( replica . empty ( ) )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Part {} is not found on remote replicas " , part_desc - > new_part_name ) ;
2018-05-21 13:49:54 +00:00
/// Fallback to covering part
replica = findReplicaHavingCoveringPart ( part_desc - > new_part_name , true , found_part_name ) ;
if ( replica . empty ( ) )
{
/// It is not fail, since adjacent parts could cover current part
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Parts covering {} are not found on remote replicas " , part_desc - > new_part_name ) ;
2018-05-21 13:49:54 +00:00
continue ;
}
}
else
{
found_part_name = part_desc - > new_part_name ;
}
part_desc - > found_new_part_name = found_part_name ;
2019-05-03 02:00:57 +00:00
part_desc - > found_new_part_info = MergeTreePartInfo : : fromPartName ( found_part_name , format_version ) ;
2018-05-21 13:49:54 +00:00
part_desc - > replica = replica ;
2018-05-28 15:37:30 +00:00
adding_parts_active_set . add ( part_desc - > found_new_part_name ) ;
2018-05-21 13:49:54 +00:00
part_name_to_desc . emplace ( part_desc - > found_new_part_name , part_desc ) ;
2017-11-17 08:58:35 +00:00
}
2017-12-21 18:17:06 +00:00
2018-05-21 13:49:54 +00:00
/// Check that we could cover whole range
for ( PartDescriptionPtr & part_desc : parts_to_add )
{
2018-05-28 15:37:30 +00:00
if ( adding_parts_active_set . getContainingPart ( part_desc - > new_part_info ) . empty ( ) )
2018-05-21 13:49:54 +00:00
{
throw Exception ( " Not found part " + part_desc - > new_part_name +
" (or part covering it) neither source table neither remote replicas " , ErrorCodes : : NO_REPLICA_HAS_PART ) ;
}
}
2014-03-21 19:17:59 +00:00
2018-05-21 13:49:54 +00:00
/// Filter covered parts
PartDescriptions final_parts ;
2021-05-13 11:29:59 +00:00
Strings final_part_names ;
2018-05-21 13:49:54 +00:00
{
2021-05-13 11:29:59 +00:00
final_part_names = adding_parts_active_set . getParts ( ) ;
2014-10-18 17:37:55 +00:00
2018-05-21 13:49:54 +00:00
for ( const String & final_part_name : final_part_names )
{
auto part_desc = part_name_to_desc [ final_part_name ] ;
if ( ! part_desc )
throw Exception ( " There is no final part " + final_part_name + " . This is a bug " , ErrorCodes : : LOGICAL_ERROR ) ;
final_parts . emplace_back ( part_desc ) ;
2018-04-19 14:20:18 +00:00
2018-05-21 13:49:54 +00:00
if ( final_parts . size ( ) > 1 )
{
auto & prev = * final_parts [ final_parts . size ( ) - 2 ] ;
auto & curr = * final_parts [ final_parts . size ( ) - 1 ] ;
if ( ! prev . found_new_part_info . isDisjoint ( curr . found_new_part_info ) )
{
throw Exception ( " Intersected final parts detected: " + prev . found_new_part_name
2021-05-13 11:29:59 +00:00
+ " and " + curr . found_new_part_name + " . It should be investigated. " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-05-21 13:49:54 +00:00
}
}
}
}
static const String TMP_PREFIX = " tmp_replace_from_ " ;
auto obtain_part = [ & ] ( PartDescriptionPtr & part_desc )
2018-04-19 14:20:18 +00:00
{
2018-05-21 13:49:54 +00:00
if ( part_desc - > src_table_part )
2018-04-19 14:20:18 +00:00
{
2018-05-21 13:49:54 +00:00
if ( part_desc - > checksum_hex ! = part_desc - > src_table_part - > checksums . getTotalChecksumHex ( ) )
throw Exception ( " Checksums of " + part_desc - > src_table_part - > name + " is suddenly changed " , ErrorCodes : : UNFINISHED ) ;
2019-11-03 22:27:05 +00:00
part_desc - > res_part = cloneAndLoadDataPartOnSameDisk (
2020-06-26 11:30:23 +00:00
part_desc - > src_table_part , TMP_PREFIX + " clone_ " , part_desc - > new_part_info , metadata_snapshot ) ;
2018-04-19 14:20:18 +00:00
}
2018-05-21 13:49:54 +00:00
else if ( ! part_desc - > replica . empty ( ) )
2018-04-19 14:20:18 +00:00
{
2021-05-08 10:59:55 +00:00
String source_replica_path = fs : : path ( zookeeper_path ) / " replicas " / part_desc - > replica ;
ReplicatedMergeTreeAddress address ( getZooKeeper ( ) - > get ( fs : : path ( source_replica_path ) / " host " ) ) ;
2021-04-10 23:33:54 +00:00
auto timeouts = getFetchPartHTTPTimeouts ( getContext ( ) ) ;
2021-02-04 17:25:10 +00:00
2021-04-10 23:33:54 +00:00
auto credentials = getContext ( ) - > getInterserverCredentials ( ) ;
String interserver_scheme = getContext ( ) - > getInterserverScheme ( ) ;
2018-05-21 13:49:54 +00:00
2018-07-31 10:34:35 +00:00
if ( interserver_scheme ! = address . scheme )
2019-05-11 18:00:43 +00:00
throw Exception ( " Interserver schemas are different ' " + interserver_scheme + " ' != ' " + address . scheme + " ', can't fetch part from " + address . host , ErrorCodes : : LOGICAL_ERROR ) ;
2018-05-21 13:49:54 +00:00
2020-06-26 11:30:23 +00:00
part_desc - > res_part = fetcher . fetchPart (
2021-05-21 16:14:01 +00:00
metadata_snapshot , getContext ( ) , part_desc - > found_new_part_name , source_replica_path ,
2021-05-26 20:37:44 +00:00
address . host , address . replication_port , timeouts , credentials - > getUser ( ) , credentials - > getPassword ( ) ,
interserver_scheme , replicated_fetches_throttler , false , TMP_PREFIX + " fetch_ " ) ;
2018-05-21 13:49:54 +00:00
/// TODO: check columns_version of fetched part
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartFetches ) ;
}
else
throw Exception ( " There is no receipt to produce part " + part_desc - > new_part_name + " . This is bug " , ErrorCodes : : LOGICAL_ERROR ) ;
} ;
/// Download or clone parts
/// TODO: make it in parallel
for ( PartDescriptionPtr & part_desc : final_parts )
obtain_part ( part_desc ) ;
2019-05-03 02:00:57 +00:00
MutableDataPartsVector res_parts ;
2018-05-21 13:49:54 +00:00
for ( PartDescriptionPtr & part_desc : final_parts )
res_parts . emplace_back ( part_desc - > res_part ) ;
try
{
/// Commit parts
auto zookeeper = getZooKeeper ( ) ;
2019-05-03 02:00:57 +00:00
Transaction transaction ( * this ) ;
2018-05-21 13:49:54 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2018-05-21 13:49:54 +00:00
for ( PartDescriptionPtr & part_desc : final_parts )
{
2019-05-03 02:00:57 +00:00
renameTempPartAndReplace ( part_desc - > res_part , nullptr , & transaction ) ;
2018-05-21 13:49:54 +00:00
getCommitPartOps ( ops , part_desc - > res_part ) ;
2018-04-19 14:20:18 +00:00
}
2018-05-21 13:49:54 +00:00
if ( ! ops . empty ( ) )
zookeeper - > multi ( ops ) ;
{
2019-05-03 02:00:57 +00:00
auto data_parts_lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
transaction . commit ( & data_parts_lock ) ;
if ( replace )
2021-05-13 11:29:59 +00:00
{
2021-05-13 14:04:36 +00:00
parts_to_remove = removePartsInRangeFromWorkingSet ( drop_range , true , data_parts_lock ) ;
2021-05-13 11:29:59 +00:00
String parts_to_remove_str ;
for ( const auto & part : parts_to_remove )
{
parts_to_remove_str + = part - > name ;
parts_to_remove_str + = " " ;
}
2021-05-14 12:55:30 +00:00
LOG_TRACE ( log , " Replacing {} parts {}with {} parts {} " , parts_to_remove . size ( ) , parts_to_remove_str ,
2021-05-13 11:29:59 +00:00
final_parts . size ( ) , boost : : algorithm : : join ( final_part_names , " , " ) ) ;
}
2018-05-21 13:49:54 +00:00
}
2021-04-10 23:33:54 +00:00
PartLog : : addNewParts ( getContext ( ) , res_parts , watch . elapsed ( ) ) ;
2018-05-21 13:49:54 +00:00
}
catch ( . . . )
{
2021-04-10 23:33:54 +00:00
PartLog : : addNewParts ( getContext ( ) , res_parts , watch . elapsed ( ) , ExecutionStatus : : fromCurrentException ( ) ) ;
2018-05-21 13:49:54 +00:00
throw ;
}
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove ) ;
res_parts . clear ( ) ;
parts_to_remove . clear ( ) ;
2018-07-30 17:34:55 +00:00
cleanup_thread . wakeup ( ) ;
2018-05-21 13:49:54 +00:00
return true ;
}
2020-11-24 14:24:48 +00:00
void StorageReplicatedMergeTree : : executeClonePartFromShard ( const LogEntry & entry )
{
auto zookeeper = getZooKeeper ( ) ;
Strings replicas = zookeeper - > getChildren ( entry . source_shard + " /replicas " ) ;
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
2021-05-21 09:30:49 +00:00
String replica ;
for ( const String & candidate : replicas )
{
if ( zookeeper - > exists ( entry . source_shard + " /replicas/ " + candidate + " /is_active " ) )
{
replica = candidate ;
break ;
}
}
if ( replica . empty ( ) )
throw Exception ( ErrorCodes : : NO_REPLICA_HAS_PART , " Not found active replica on shard {} to clone part {} " , entry . source_shard , entry . new_part_name ) ;
2020-11-24 14:24:48 +00:00
LOG_INFO ( log , " Will clone part from shard " + entry . source_shard + " and replica " + replica ) ;
MutableDataPartPtr part ;
{
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
String source_replica_path = entry . source_shard + " /replicas/ " + replica ;
ReplicatedMergeTreeAddress address ( getZooKeeper ( ) - > get ( source_replica_path + " /host " ) ) ;
2021-04-20 12:26:05 +00:00
auto timeouts = ConnectionTimeouts : : getHTTPTimeouts ( getContext ( ) ) ;
auto credentials = getContext ( ) - > getInterserverCredentials ( ) ;
String interserver_scheme = getContext ( ) - > getInterserverScheme ( ) ;
2020-11-24 14:24:48 +00:00
2021-04-20 12:26:05 +00:00
auto get_part = [ & , address , timeouts , credentials , interserver_scheme ] ( )
2020-11-24 14:24:48 +00:00
{
if ( interserver_scheme ! = address . scheme )
throw Exception ( " Interserver schemes are different: ' " + interserver_scheme
+ " ' != ' " + address . scheme + " ', can't fetch part from " + address . host ,
ErrorCodes : : LOGICAL_ERROR ) ;
return fetcher . fetchPart (
2021-05-23 21:54:22 +00:00
metadata_snapshot , getContext ( ) , entry . new_part_name , source_replica_path ,
2020-11-24 14:24:48 +00:00
address . host , address . replication_port ,
2021-05-26 20:37:44 +00:00
timeouts , credentials - > getUser ( ) , credentials - > getPassword ( ) , interserver_scheme ,
replicated_fetches_throttler , true ) ;
2020-11-24 14:24:48 +00:00
} ;
part = get_part ( ) ;
2021-05-21 09:30:49 +00:00
// The fetched part is valuable and should not be cleaned like a temp part.
part - > is_temp = false ;
part - > renameTo ( " detached/ " + entry . new_part_name , true ) ;
LOG_INFO ( log , " Cloned part {} to detached directory " , part - > name ) ;
2020-11-24 14:24:48 +00:00
}
}
2018-08-27 13:51:22 +00:00
void StorageReplicatedMergeTree : : cloneReplica ( const String & source_replica , Coordination : : Stat source_is_lost_stat , zkutil : : ZooKeeperPtr & zookeeper )
2018-08-27 19:06:32 +00:00
{
2021-05-08 10:59:55 +00:00
String source_path = fs : : path ( zookeeper_path ) / " replicas " / source_replica ;
2018-08-27 19:06:32 +00:00
2020-03-10 13:15:12 +00:00
/// The order of the following three actions is important.
2018-08-27 19:06:32 +00:00
2020-03-10 13:15:12 +00:00
Strings source_queue_names ;
/// We are trying to get consistent /log_pointer and /queue state. Otherwise
/// we can possibly duplicate entries in queue of cloned replica.
while ( true )
{
Coordination : : Stat log_pointer_stat ;
2021-05-08 10:59:55 +00:00
String raw_log_pointer = zookeeper - > get ( fs : : path ( source_path ) / " log_pointer " , & log_pointer_stat ) ;
2018-08-20 17:15:04 +00:00
2020-03-10 13:15:12 +00:00
Coordination : : Requests ops ;
2021-05-08 10:59:55 +00:00
ops . push_back ( zkutil : : makeSetRequest ( fs : : path ( replica_path ) / " log_pointer " , raw_log_pointer , - 1 ) ) ;
2018-08-27 19:06:32 +00:00
2020-03-10 13:15:12 +00:00
/// For support old versions CH.
if ( source_is_lost_stat . version = = - 1 )
{
/// We check that it was not suddenly upgraded to new version.
/// Otherwise it can be upgraded and instantly become lost, but we cannot notice that.
2021-05-08 10:59:55 +00:00
ops . push_back ( zkutil : : makeCreateRequest ( fs : : path ( source_path ) / " is_lost " , " 0 " , zkutil : : CreateMode : : Persistent ) ) ;
ops . push_back ( zkutil : : makeRemoveRequest ( fs : : path ( source_path ) / " is_lost " , - 1 ) ) ;
2020-03-10 13:15:12 +00:00
}
else /// The replica we clone should not suddenly become lost.
2021-05-08 10:59:55 +00:00
ops . push_back ( zkutil : : makeCheckRequest ( fs : : path ( source_path ) / " is_lost " , source_is_lost_stat . version ) ) ;
2018-08-27 19:06:32 +00:00
2020-03-10 13:15:12 +00:00
Coordination : : Responses responses ;
2018-08-27 19:06:32 +00:00
2020-03-10 13:15:12 +00:00
/// Let's remember the queue of the reference/master replica.
2021-05-08 10:59:55 +00:00
source_queue_names = zookeeper - > getChildren ( fs : : path ( source_path ) / " queue " ) ;
2020-03-10 13:15:12 +00:00
2021-04-28 17:49:27 +00:00
/// Check that log pointer of source replica didn't changed while we read queue entries
2021-05-09 10:45:50 +00:00
ops . push_back ( zkutil : : makeCheckRequest ( fs : : path ( source_path ) / " log_pointer " , log_pointer_stat . version ) ) ;
2020-03-10 13:15:12 +00:00
auto rc = zookeeper - > tryMulti ( ops , responses ) ;
2020-06-12 15:09:12 +00:00
if ( rc = = Coordination : : Error : : ZOK )
2020-03-10 13:15:12 +00:00
{
break ;
}
else if ( rc = = Coordination : : Error : : ZNODEEXISTS )
{
throw Exception (
" Can not clone replica, because the " + source_replica + " updated to new ClickHouse version " ,
ErrorCodes : : REPLICA_STATUS_CHANGED ) ;
}
else if ( responses [ 1 ] - > error = = Coordination : : Error : : ZBADVERSION )
{
/// If is_lost node version changed than source replica also lost,
/// so we cannot clone from it.
throw Exception (
" Can not clone replica, because the " + source_replica + " became lost " , ErrorCodes : : REPLICA_STATUS_CHANGED ) ;
}
else if ( responses . back ( ) - > error = = Coordination : : Error : : ZBADVERSION )
{
/// If source replica's log_pointer changed than we probably read
/// stale state of /queue and have to try one more time.
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Log pointer of source replica {} changed while we loading queue nodes. Will retry. " , source_replica ) ;
2020-03-10 13:15:12 +00:00
continue ;
}
else
{
zkutil : : KeeperMultiException : : check ( rc , ops , responses ) ;
}
}
2018-08-27 19:06:32 +00:00
2018-08-07 15:21:42 +00:00
std : : sort ( source_queue_names . begin ( ) , source_queue_names . end ( ) ) ;
2020-03-10 13:15:12 +00:00
2021-10-18 20:16:02 +00:00
struct QueueEntryInfo
2018-08-07 15:21:42 +00:00
{
2021-10-19 11:56:04 +00:00
String data = { } ;
Coordination : : Stat stat = { } ;
LogEntryPtr parsed_entry = { } ;
2021-10-18 20:16:02 +00:00
} ;
2021-10-25 14:01:23 +00:00
/// We got log pointer and list of queue entries of source replica.
/// At first we will get queue entries and then we will get list of active parts of source replica
/// to enqueue fetches for missing parts. If source replica executes and removes some entry concurrently
/// we will see produced part (or covering part) in replicas/source/parts and will enqueue fetch.
/// We will try to parse queue entries before copying them
/// to avoid creation of excessive and duplicating entries in our queue.
/// See also removePartAndEnqueueFetch(...)
2021-10-18 20:16:02 +00:00
std : : vector < QueueEntryInfo > source_queue ;
ActiveDataPartSet get_part_set { format_version } ;
ActiveDataPartSet drop_range_set { format_version } ;
{
std : : vector < zkutil : : ZooKeeper : : FutureGet > queue_get_futures ;
queue_get_futures . reserve ( source_queue_names . size ( ) ) ;
for ( const String & entry_name : source_queue_names )
queue_get_futures . push_back ( zookeeper - > asyncTryGet ( fs : : path ( source_path ) / " queue " / entry_name ) ) ;
2021-10-19 11:56:04 +00:00
source_queue . reserve ( source_queue_names . size ( ) ) ;
2021-10-18 20:16:02 +00:00
for ( size_t i = 0 ; i < source_queue_names . size ( ) ; + + i )
{
auto res = queue_get_futures [ i ] . get ( ) ;
/// It's ok if entry is already executed and removed: we also will get source parts set.
if ( res . error = = Coordination : : Error : : ZNONODE )
continue ;
assert ( res . error = = Coordination : : Error : : ZOK ) ;
2021-10-19 11:56:04 +00:00
source_queue . emplace_back ( ) ;
auto & info = source_queue . back ( ) ;
2021-10-18 20:16:02 +00:00
info . data = std : : move ( res . data ) ;
info . stat = std : : move ( res . stat ) ;
try
{
info . parsed_entry = LogEntry : : parse ( info . data , info . stat ) ;
}
catch ( . . . )
{
tryLogCurrentException ( log , " Cannot parse source queue entry " + source_queue_names [ i ] ) ;
}
/// It may be ok if source replica has newer version. We will copy entry as is.
if ( ! info . parsed_entry )
continue ;
info . parsed_entry - > znode_name = source_queue_names [ i ] ;
if ( info . parsed_entry - > type = = LogEntry : : DROP_RANGE )
drop_range_set . add ( info . parsed_entry - > new_part_name ) ;
2021-10-25 14:01:23 +00:00
if ( info . parsed_entry - > type = = LogEntry : : GET_PART )
{
String maybe_covering_drop_range = drop_range_set . getContainingPart ( info . parsed_entry - > new_part_name ) ;
if ( maybe_covering_drop_range . empty ( ) )
get_part_set . add ( info . parsed_entry - > new_part_name ) ;
}
2021-10-18 20:16:02 +00:00
}
2018-08-07 15:21:42 +00:00
}
2021-04-28 17:49:27 +00:00
/// We should do it after copying queue, because some ALTER_METADATA entries can be lost otherwise.
2021-05-07 17:09:39 +00:00
cloneMetadataIfNeeded ( source_replica , source_path , zookeeper ) ;
2021-04-28 17:49:27 +00:00
2018-08-07 15:21:42 +00:00
/// Add to the queue jobs to receive all the active parts that the reference/master replica has.
2021-05-08 10:59:55 +00:00
Strings source_replica_parts = zookeeper - > getChildren ( fs : : path ( source_path ) / " parts " ) ;
2021-10-18 20:16:02 +00:00
for ( const auto & active_part : source_replica_parts )
get_part_set . add ( active_part ) ;
2018-08-07 15:21:42 +00:00
2021-10-18 20:16:02 +00:00
Strings active_parts = get_part_set . getParts ( ) ;
2019-08-19 12:06:44 +00:00
/// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas.
2021-05-08 10:59:55 +00:00
Strings local_parts_in_zk = zookeeper - > getChildren ( fs : : path ( replica_path ) / " parts " ) ;
2019-08-19 12:06:44 +00:00
Strings parts_to_remove_from_zk ;
2021-06-20 08:24:43 +00:00
2019-08-19 12:06:44 +00:00
for ( const auto & part : local_parts_in_zk )
{
2021-10-18 20:16:02 +00:00
if ( get_part_set . getContainingPart ( part ) . empty ( ) )
2019-08-19 12:06:44 +00:00
{
parts_to_remove_from_zk . emplace_back ( part ) ;
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Source replica does not have part {}. Removing it from ZooKeeper. " , part ) ;
2019-08-19 12:06:44 +00:00
}
}
2021-06-20 08:24:43 +00:00
2021-07-14 17:05:50 +00:00
{
/// Check "is_lost" version after retrieving queue and parts.
/// If version has changed, then replica most likely has been dropped and parts set is inconsistent,
/// so throw exception and retry cloning.
Coordination : : Stat is_lost_stat_new ;
zookeeper - > get ( fs : : path ( source_path ) / " is_lost " , & is_lost_stat_new ) ;
if ( is_lost_stat_new . version ! = source_is_lost_stat . version )
2021-10-25 14:01:23 +00:00
throw Exception ( ErrorCodes : : REPLICA_STATUS_CHANGED , " Cannot clone {}, because it suddenly become lost "
" or removed broken part from ZooKeeper " , source_replica ) ;
2021-07-14 17:05:50 +00:00
}
2019-08-19 12:06:44 +00:00
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove_from_zk ) ;
auto local_active_parts = getDataParts ( ) ;
2021-06-20 08:24:43 +00:00
2019-08-19 12:06:44 +00:00
DataPartsVector parts_to_remove_from_working_set ;
2021-06-20 08:24:43 +00:00
2019-08-19 12:06:44 +00:00
for ( const auto & part : local_active_parts )
{
2021-10-18 20:16:02 +00:00
if ( get_part_set . getContainingPart ( part - > name ) . empty ( ) )
2019-08-19 12:06:44 +00:00
{
parts_to_remove_from_working_set . emplace_back ( part ) ;
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Source replica does not have part {}. Removing it from working set. " , part - > name ) ;
2019-08-19 12:06:44 +00:00
}
}
2020-10-06 20:05:28 +00:00
if ( getSettings ( ) - > detach_old_local_parts_when_cloning_replica )
{
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2021-06-20 08:24:43 +00:00
2020-10-06 20:05:28 +00:00
for ( const auto & part : parts_to_remove_from_working_set )
{
LOG_INFO ( log , " Detaching {} " , part - > relative_path ) ;
part - > makeCloneInDetached ( " clone " , metadata_snapshot ) ;
}
}
2019-08-19 12:06:44 +00:00
removePartsFromWorkingSet ( parts_to_remove_from_working_set , true ) ;
2021-10-18 20:16:02 +00:00
std : : unordered_set < String > created_get_parts ;
/// Avoid creation of GET_PART entries which covered by another GET_PART or DROP_RANGE
/// and creation of multiple entries with the same new_part_name.
2021-10-25 14:01:23 +00:00
auto should_ignore_log_entry = [ & drop_range_set , & get_part_set , this ] ( std : : unordered_set < String > & created_gets ,
const String & part_name , const String & log_msg_context ) - > bool
2021-10-18 20:16:02 +00:00
{
/// We should not create entries covered by DROP_RANGE, because we will remove them anyway (kind of optimization).
String covering_drop_range = drop_range_set . getContainingPart ( part_name ) ;
if ( ! covering_drop_range . empty ( ) )
{
LOG_TRACE ( log , " {} {}: it's covered by DROP_RANGE {} " , log_msg_context , part_name , covering_drop_range ) ;
return true ;
}
/// We should not create entries covered by GET_PART,
/// because GET_PART entry has no source parts and we can execute it only by fetching.
/// Parts covered by GET_PART are useless and may cause replication to stuck if covered part is lost.
String covering_get_part_entry = get_part_set . getContainingPart ( part_name ) ;
if ( covering_get_part_entry . empty ( ) )
return false ;
if ( covering_get_part_entry ! = part_name )
{
LOG_TRACE ( log , " {} {}: it's covered by GET_PART {} " , log_msg_context , part_name , covering_get_part_entry ) ;
return true ;
}
2021-10-19 11:56:04 +00:00
/// NOTE: It does not completely avoids duplication of GET_PART entries,
/// because it's possible that source replica has executed some GET_PART after we copied it's queue,
/// but before we copied its active parts set. In this case we will GET_PART entry in our queue
/// and later will pull the original GET_PART from replication log.
/// It should not cause any issues, but it does not allow to get rid of duplicated entries and add an assertion.
2021-10-25 14:01:23 +00:00
if ( created_gets . count ( part_name ) )
2021-10-18 20:16:02 +00:00
{
/// NOTE It would be better to copy log entry instead of creating GET_PART
/// if there are GET_PART and log entry of other type with the same new_part_name.
/// But it's a bit harder to implement, because it requires full-fledged virtual_parts set.
LOG_TRACE ( log , " {} {}: GET_PART for it is already created " , log_msg_context , part_name ) ;
return true ;
}
return false ;
} ;
2018-08-07 15:21:42 +00:00
for ( const String & name : active_parts )
{
2021-10-25 14:01:23 +00:00
if ( should_ignore_log_entry ( created_get_parts , name , " Not fetching " ) )
2021-10-18 20:16:02 +00:00
continue ;
2018-08-07 15:21:42 +00:00
LogEntry log_entry ;
2021-06-20 08:24:43 +00:00
2021-10-18 20:16:02 +00:00
if ( are_restoring_replica )
2021-06-20 08:24:43 +00:00
{
LOG_DEBUG ( log , " Obtaining checksum for path {} " , name ) ;
// The part we want to fetch is probably present in detached/ folder.
// However, we need to get part's checksum to check if it's not corrupt.
log_entry . type = LogEntry : : ATTACH_PART ;
MinimalisticDataPartChecksums desired_checksums ;
const fs : : path part_path = fs : : path ( source_path ) / " parts " / name ;
const String part_znode = zookeeper - > get ( part_path ) ;
if ( ! part_znode . empty ( ) )
desired_checksums = ReplicatedMergeTreePartHeader : : fromString ( part_znode ) . getChecksums ( ) ;
else
{
String desired_checksums_str = zookeeper - > get ( part_path / " checksums " ) ;
desired_checksums = MinimalisticDataPartChecksums : : deserializeFrom ( desired_checksums_str ) ;
}
const auto [ lo , hi ] = desired_checksums . hash_of_all_files ;
log_entry . part_checksum = getHexUIntUppercase ( hi ) + getHexUIntUppercase ( lo ) ;
}
2021-10-18 20:16:02 +00:00
else
{
log_entry . type = LogEntry : : GET_PART ;
}
2021-06-20 08:24:43 +00:00
2018-11-26 00:56:50 +00:00
log_entry . source_replica = " " ;
2018-08-07 15:21:42 +00:00
log_entry . new_part_name = name ;
log_entry . create_time = tryGetPartCreateTime ( zookeeper , source_path , name ) ;
2021-10-18 20:16:02 +00:00
LOG_TEST ( log , " Enqueueing {} for fetch " , name ) ;
2021-05-08 10:59:55 +00:00
zookeeper - > create ( fs : : path ( replica_path ) / " queue/queue- " , log_entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ;
2021-10-18 20:16:02 +00:00
created_get_parts . insert ( name ) ;
2018-08-07 15:21:42 +00:00
}
2018-08-27 19:06:32 +00:00
2021-10-18 20:16:02 +00:00
size_t total_parts_to_fetch = created_get_parts . size ( ) ;
LOG_DEBUG ( log , " Queued {} parts to be fetched, {} parts ignored " , total_parts_to_fetch , active_parts . size ( ) - total_parts_to_fetch ) ;
2018-08-07 15:21:42 +00:00
/// Add content of the reference/master replica queue to the queue.
2021-10-18 20:16:02 +00:00
size_t total_entries_to_copy = 0 ;
for ( const auto & entry_info : source_queue )
2018-08-07 15:21:42 +00:00
{
2021-10-19 11:56:04 +00:00
assert ( ! entry_info . data . empty ( ) ) ;
2021-10-18 20:16:02 +00:00
if ( entry_info . parsed_entry & & ! entry_info . parsed_entry - > new_part_name . empty ( ) )
{
const String & part_name = entry_info . parsed_entry - > new_part_name ;
const String & entry_name = entry_info . parsed_entry - > znode_name ;
const auto & entry_type = entry_info . parsed_entry - > type ;
2021-10-25 14:01:23 +00:00
if ( should_ignore_log_entry ( created_get_parts , part_name , fmt : : format ( " Not copying {} {} " , entry_name , entry_type ) ) )
2021-10-18 20:16:02 +00:00
continue ;
if ( entry_info . parsed_entry - > type = = LogEntry : : GET_PART )
created_get_parts . insert ( part_name ) ;
}
LOG_TEST ( log , " Copying entry {} " , entry_info . data ) ;
zookeeper - > create ( fs : : path ( replica_path ) / " queue/queue- " , entry_info . data , zkutil : : CreateMode : : PersistentSequential ) ;
+ + total_entries_to_copy ;
2018-08-07 15:21:42 +00:00
}
2021-10-18 20:16:02 +00:00
LOG_DEBUG ( log , " Copied {} queue entries, {} entries ignored " , total_entries_to_copy , source_queue . size ( ) - total_entries_to_copy ) ;
2018-08-20 23:08:45 +00:00
}
2021-05-07 17:09:39 +00:00
void StorageReplicatedMergeTree : : cloneMetadataIfNeeded ( const String & source_replica , const String & source_path , zkutil : : ZooKeeperPtr & zookeeper )
{
String source_metadata_version_str ;
bool metadata_version_exists = zookeeper - > tryGet ( source_path + " /metadata_version " , source_metadata_version_str ) ;
if ( ! metadata_version_exists )
{
/// For compatibility with version older than 20.3
/// TODO fix tests and delete it
LOG_WARNING ( log , " Node {} does not exist. "
" Most likely it's because too old version of ClickHouse is running on replica {}. "
" Will not check metadata consistency " ,
source_path + " /metadata_version " , source_replica ) ;
return ;
}
Int32 source_metadata_version = parse < Int32 > ( source_metadata_version_str ) ;
if ( metadata_version = = source_metadata_version )
return ;
/// Our metadata it not up to date with source replica metadata.
/// Metadata is updated by ALTER_METADATA entries, but some entries are probably cleaned up from the log.
/// It's also possible that some newer ALTER_METADATA entries are present in source_queue list,
/// and source replica are executing such entry right now (or had executed recently).
/// More than that, /metadata_version update is not atomic with /columns and /metadata update...
/// Fortunately, ALTER_METADATA seems to be idempotent,
/// and older entries of such type can be replaced with newer entries.
/// Let's try to get consistent values of source replica's /columns and /metadata
/// and prepend dummy ALTER_METADATA to our replication queue.
/// It should not break anything if source_queue already contains ALTER_METADATA entry
/// with greater or equal metadata_version, but it will update our metadata
/// if all such entries were cleaned up from the log and source_queue.
LOG_WARNING ( log , " Metadata version ({}) on replica is not up to date with metadata ({}) on source replica {} " ,
metadata_version , source_metadata_version , source_replica ) ;
String source_metadata ;
String source_columns ;
while ( true )
{
Coordination : : Stat metadata_stat ;
Coordination : : Stat columns_stat ;
source_metadata = zookeeper - > get ( source_path + " /metadata " , & metadata_stat ) ;
source_columns = zookeeper - > get ( source_path + " /columns " , & columns_stat ) ;
Coordination : : Requests ops ;
Coordination : : Responses responses ;
ops . emplace_back ( zkutil : : makeCheckRequest ( source_path + " /metadata " , metadata_stat . version ) ) ;
ops . emplace_back ( zkutil : : makeCheckRequest ( source_path + " /columns " , columns_stat . version ) ) ;
Coordination : : Error code = zookeeper - > tryMulti ( ops , responses ) ;
if ( code = = Coordination : : Error : : ZOK )
break ;
else if ( code = = Coordination : : Error : : ZBADVERSION )
LOG_WARNING ( log , " Metadata of replica {} was changed " , source_path ) ;
else
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
ReplicatedMergeTreeLogEntryData dummy_alter ;
dummy_alter . type = LogEntry : : ALTER_METADATA ;
dummy_alter . source_replica = source_replica ;
dummy_alter . metadata_str = source_metadata ;
dummy_alter . columns_str = source_columns ;
dummy_alter . alter_version = source_metadata_version ;
dummy_alter . create_time = time ( nullptr ) ;
zookeeper - > create ( replica_path + " /queue/queue- " , dummy_alter . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ;
/// We don't need to do anything with mutation_pointer, because mutation log cleanup process is different from
/// replication log cleanup. A mutation is removed from ZooKeeper only if all replicas had executed the mutation,
/// so all mutations which are greater or equal to our mutation pointer are still present in ZooKeeper.
}
2018-08-22 14:01:54 +00:00
void StorageReplicatedMergeTree : : cloneReplicaIfNeeded ( zkutil : : ZooKeeperPtr zookeeper )
2014-04-03 11:48:28 +00:00
{
2020-10-06 20:05:28 +00:00
Coordination : : Stat is_lost_stat ;
bool is_new_replica = true ;
2018-08-20 23:08:45 +00:00
String res ;
2021-06-20 08:24:43 +00:00
2021-05-08 10:59:55 +00:00
if ( zookeeper - > tryGet ( fs : : path ( replica_path ) / " is_lost " , res , & is_lost_stat ) )
2018-08-20 23:08:45 +00:00
{
if ( res = = " 0 " )
return ;
2020-10-06 20:05:28 +00:00
if ( is_lost_stat . version )
is_new_replica = false ;
2018-08-20 23:08:45 +00:00
}
else
{
2018-08-27 15:44:51 +00:00
/// Replica was created by old version of CH, so me must create "/is_lost".
2018-08-27 19:16:38 +00:00
/// Note that in old version of CH there was no "lost" replicas possible.
2020-10-06 20:05:28 +00:00
/// TODO is_lost node should always exist since v18.12, maybe we can replace `tryGet` with `get` and remove old code?
2021-05-08 10:59:55 +00:00
zookeeper - > create ( fs : : path ( replica_path ) / " is_lost " , " 0 " , zkutil : : CreateMode : : Persistent ) ;
2017-12-29 22:32:04 +00:00
return ;
2018-08-20 23:08:45 +00:00
}
2018-08-07 15:21:42 +00:00
2018-08-28 00:44:42 +00:00
/// is_lost is "1": it means that we are in repair mode.
2020-10-06 20:05:28 +00:00
/// Try choose source replica to clone.
/// Source replica must not be lost and should have minimal queue size and maximal log pointer.
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2020-10-06 20:05:28 +00:00
std : : vector < zkutil : : ZooKeeper : : FutureGet > futures ;
for ( const String & source_replica_name : replicas )
{
/// Do not clone from myself.
if ( source_replica_name = = replica_name )
continue ;
2021-05-08 10:59:55 +00:00
String source_replica_path = fs : : path ( zookeeper_path ) / " replicas " / source_replica_name ;
2018-08-28 00:44:42 +00:00
2020-10-06 20:05:28 +00:00
/// Obviously the following get operations are not atomic, but it's ok to choose good enough replica, not the best one.
/// NOTE: We may count some entries twice if log_pointer is moved.
2021-05-08 10:59:55 +00:00
futures . emplace_back ( zookeeper - > asyncTryGet ( fs : : path ( source_replica_path ) / " is_lost " ) ) ;
futures . emplace_back ( zookeeper - > asyncTryGet ( fs : : path ( source_replica_path ) / " log_pointer " ) ) ;
futures . emplace_back ( zookeeper - > asyncTryGet ( fs : : path ( source_replica_path ) / " queue " ) ) ;
2020-10-06 20:05:28 +00:00
}
2020-10-07 00:05:48 +00:00
/// Wait for results before getting log entries
for ( auto & future : futures )
future . wait ( ) ;
2018-08-28 00:44:42 +00:00
2021-05-08 10:59:55 +00:00
Strings log_entries = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " log " ) ;
2020-10-06 20:05:28 +00:00
size_t max_log_entry = 0 ;
if ( ! log_entries . empty ( ) )
{
2020-10-07 19:47:31 +00:00
String last_entry = * std : : max_element ( log_entries . begin ( ) , log_entries . end ( ) ) ;
max_log_entry = parse < UInt64 > ( last_entry . substr ( strlen ( " log- " ) ) ) ;
2020-10-06 20:05:28 +00:00
}
2020-10-07 00:05:48 +00:00
/// log_pointer can point to future entry, which was not created yet
2020-10-06 20:05:28 +00:00
+ + max_log_entry ;
size_t min_replication_lag = std : : numeric_limits < size_t > : : max ( ) ;
2018-08-07 15:21:42 +00:00
String source_replica ;
2018-08-27 13:51:22 +00:00
Coordination : : Stat source_is_lost_stat ;
2020-10-06 20:05:28 +00:00
size_t future_num = 0 ;
2018-08-07 15:21:42 +00:00
2020-10-06 20:05:28 +00:00
for ( const String & source_replica_name : replicas )
2018-08-07 15:21:42 +00:00
{
2020-10-06 20:05:28 +00:00
if ( source_replica_name = = replica_name )
continue ;
2017-04-01 07:20:54 +00:00
2020-10-06 20:05:28 +00:00
auto get_is_lost = futures [ future_num + + ] . get ( ) ;
auto get_log_pointer = futures [ future_num + + ] . get ( ) ;
auto get_queue = futures [ future_num + + ] . get ( ) ;
2020-10-07 19:47:31 +00:00
if ( get_is_lost . error ! = Coordination : : Error : : ZOK )
2018-08-20 23:08:45 +00:00
{
2021-01-04 08:15:13 +00:00
LOG_INFO ( log , " Not cloning {}, cannot get '/is_lost': {} " , source_replica_name , Coordination : : errorMessage ( get_is_lost . error ) ) ;
2020-10-07 19:47:31 +00:00
continue ;
2020-10-06 20:05:28 +00:00
}
else if ( get_is_lost . data ! = " 0 " )
2020-10-07 19:47:31 +00:00
{
2021-01-04 08:21:04 +00:00
LOG_INFO ( log , " Not cloning {}, it's lost " , source_replica_name ) ;
2020-10-06 20:05:28 +00:00
continue ;
2020-10-07 19:47:31 +00:00
}
2020-10-06 20:05:28 +00:00
if ( get_log_pointer . error ! = Coordination : : Error : : ZOK )
2020-10-07 19:47:31 +00:00
{
2021-01-04 08:15:13 +00:00
LOG_INFO ( log , " Not cloning {}, cannot get '/log_pointer': {} " , source_replica_name , Coordination : : errorMessage ( get_log_pointer . error ) ) ;
2020-10-06 20:05:28 +00:00
continue ;
2020-10-07 19:47:31 +00:00
}
2020-10-06 20:05:28 +00:00
if ( get_queue . error ! = Coordination : : Error : : ZOK )
2020-10-07 19:47:31 +00:00
{
2021-01-04 08:15:13 +00:00
LOG_INFO ( log , " Not cloning {}, cannot get '/queue': {} " , source_replica_name , Coordination : : errorMessage ( get_queue . error ) ) ;
2020-10-06 20:05:28 +00:00
continue ;
2020-10-07 19:47:31 +00:00
}
2020-10-06 20:05:28 +00:00
/// Replica is not lost and we can clone it. Let's calculate approx replication lag.
size_t source_log_pointer = get_log_pointer . data . empty ( ) ? 0 : parse < UInt64 > ( get_log_pointer . data ) ;
assert ( source_log_pointer < = max_log_entry ) ;
size_t replica_queue_lag = max_log_entry - source_log_pointer ;
size_t replica_queue_size = get_queue . stat . numChildren ;
size_t replication_lag = replica_queue_lag + replica_queue_size ;
2020-10-07 19:47:31 +00:00
LOG_INFO ( log , " Replica {} has log pointer '{}', approximate {} queue lag and {} queue size " ,
source_replica_name , get_log_pointer . data , replica_queue_lag , replica_queue_size ) ;
2020-10-06 20:05:28 +00:00
if ( replication_lag < min_replication_lag )
{
source_replica = source_replica_name ;
source_is_lost_stat = get_is_lost . stat ;
min_replication_lag = replication_lag ;
2018-08-20 23:08:45 +00:00
}
2018-08-20 13:31:24 +00:00
}
2018-08-27 19:06:32 +00:00
2018-08-27 19:16:38 +00:00
if ( source_replica . empty ( ) )
2018-08-20 13:31:24 +00:00
throw Exception ( " All replicas are lost " , ErrorCodes : : ALL_REPLICAS_LOST ) ;
2018-08-27 19:06:32 +00:00
2020-10-06 20:05:28 +00:00
if ( is_new_replica )
LOG_INFO ( log , " Will mimic {} " , source_replica ) ;
else
LOG_WARNING ( log , " Will mimic {} " , source_replica ) ;
2019-02-14 14:04:28 +00:00
/// Clear obsolete queue that we no longer need.
2021-05-08 10:59:55 +00:00
zookeeper - > removeChildren ( fs : : path ( replica_path ) / " queue " ) ;
2021-05-30 21:29:37 +00:00
queue . clear ( ) ;
2019-02-14 14:04:28 +00:00
2018-08-28 00:44:42 +00:00
/// Will do repair from the selected replica.
2018-08-23 13:55:59 +00:00
cloneReplica ( source_replica , source_is_lost_stat , zookeeper ) ;
2018-08-28 00:44:42 +00:00
/// If repair fails to whatever reason, the exception is thrown, is_lost will remain "1" and the replica will be repaired later.
2018-08-07 15:21:42 +00:00
2018-08-28 00:44:42 +00:00
/// If replica is repaired successfully, we remove is_lost flag.
2021-05-08 10:59:55 +00:00
zookeeper - > set ( fs : : path ( replica_path ) / " is_lost " , " 0 " ) ;
2018-08-07 15:21:42 +00:00
}
2021-08-09 12:58:23 +00:00
String StorageReplicatedMergeTree : : getLastQueueUpdateException ( ) const
{
std : : unique_lock lock ( last_queue_update_exception_lock ) ;
return last_queue_update_exception ;
}
2018-08-07 15:21:42 +00:00
2018-05-31 13:05:05 +00:00
void StorageReplicatedMergeTree : : queueUpdatingTask ( )
2014-04-03 11:48:28 +00:00
{
2017-12-29 22:32:04 +00:00
if ( ! queue_update_in_progress )
2017-11-17 08:58:35 +00:00
{
2017-12-29 22:32:04 +00:00
last_queue_update_start_time . store ( time ( nullptr ) ) ;
queue_update_in_progress = true ;
}
try
{
2021-08-18 09:49:22 +00:00
queue . pullLogsToQueue ( getZooKeeper ( ) , queue_updating_task - > getWatchCallback ( ) , ReplicatedMergeTreeQueue : : UPDATE ) ;
2017-12-29 22:32:04 +00:00
last_queue_update_finish_time . store ( time ( nullptr ) ) ;
queue_update_in_progress = false ;
2018-04-19 14:20:18 +00:00
}
2021-08-09 12:58:23 +00:00
catch ( const Coordination : : Exception & e )
2018-04-24 17:11:59 +00:00
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2018-04-19 14:20:18 +00:00
2021-08-09 12:58:23 +00:00
std : : unique_lock lock ( last_queue_update_exception_lock ) ;
last_queue_update_exception = getCurrentExceptionMessage ( false ) ;
if ( e . code = = Coordination : : Error : : ZSESSIONEXPIRED )
2021-07-27 14:35:20 +00:00
{
2021-08-09 12:58:23 +00:00
restarting_thread . wakeup ( ) ;
return ;
2018-07-05 16:30:52 +00:00
}
2017-12-21 18:17:06 +00:00
2021-08-09 12:58:23 +00:00
queue_updating_task - > scheduleAfter ( QUEUE_UPDATE_ERROR_SLEEP_MS ) ;
}
catch ( . . . )
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
std : : unique_lock lock ( last_queue_update_exception_lock ) ;
last_queue_update_exception = getCurrentExceptionMessage ( false ) ;
2018-05-31 13:05:05 +00:00
queue_updating_task - > scheduleAfter ( QUEUE_UPDATE_ERROR_SLEEP_MS ) ;
2018-04-24 17:11:59 +00:00
}
2014-04-03 08:47:59 +00:00
}
2014-03-21 19:17:59 +00:00
2014-10-18 17:37:55 +00:00
2018-05-31 13:05:05 +00:00
void StorageReplicatedMergeTree : : mutationsUpdatingTask ( )
2018-05-28 15:37:30 +00:00
{
try
{
2018-05-31 13:05:05 +00:00
queue . updateMutations ( getZooKeeper ( ) , mutations_updating_task - > getWatchCallback ( ) ) ;
2018-05-28 15:37:30 +00:00
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2018-05-28 15:37:30 +00:00
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2020-06-12 15:09:12 +00:00
if ( e . code = = Coordination : : Error : : ZSESSIONEXPIRED )
2018-05-28 15:37:30 +00:00
return ;
2018-05-31 13:05:05 +00:00
mutations_updating_task - > scheduleAfter ( QUEUE_UPDATE_ERROR_SLEEP_MS ) ;
2018-05-28 15:37:30 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2018-05-31 13:05:05 +00:00
mutations_updating_task - > scheduleAfter ( QUEUE_UPDATE_ERROR_SLEEP_MS ) ;
2018-05-28 15:37:30 +00:00
}
2018-04-19 14:20:18 +00:00
}
2020-10-23 08:54:00 +00:00
ReplicatedMergeTreeQueue : : SelectedEntryPtr StorageReplicatedMergeTree : : selectQueueEntry ( )
2014-04-03 11:48:28 +00:00
{
2017-04-01 07:20:54 +00:00
/// This object will mark the element of the queue as running.
2020-10-23 08:54:00 +00:00
ReplicatedMergeTreeQueue : : SelectedEntryPtr selected ;
2017-04-01 07:20:54 +00:00
try
{
2019-05-03 02:00:57 +00:00
selected = queue . selectEntryToProcess ( merger_mutator , * this ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
2018-04-06 21:46:57 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-04-01 07:20:54 +00:00
}
2020-09-30 12:40:46 +00:00
return selected ;
}
2017-04-01 07:20:54 +00:00
2021-09-16 21:19:58 +00:00
2020-10-23 08:54:00 +00:00
bool StorageReplicatedMergeTree : : processQueueEntry ( ReplicatedMergeTreeQueue : : SelectedEntryPtr selected_entry )
2020-09-30 12:40:46 +00:00
{
2020-10-23 08:54:00 +00:00
LogEntryPtr & entry = selected_entry - > log_entry ;
2020-09-30 12:40:46 +00:00
return queue . processEntry ( [ this ] { return getZooKeeper ( ) ; } , entry , [ & ] ( LogEntryPtr & entry_to_process )
2017-04-01 07:20:54 +00:00
{
try
{
2019-01-04 12:10:00 +00:00
return executeLogEntry ( * entry_to_process ) ;
2017-04-01 07:20:54 +00:00
}
catch ( const Exception & e )
{
if ( e . code ( ) = = ErrorCodes : : NO_REPLICA_HAS_PART )
{
/// If no one has the right part, probably not all replicas work; We will not write to log with Error level.
LOG_INFO ( log , e . displayText ( ) ) ;
}
else if ( e . code ( ) = = ErrorCodes : : ABORTED )
{
/// Interrupted merge or downloading a part is not an error.
LOG_INFO ( log , e . message ( ) ) ;
}
2017-11-20 19:33:12 +00:00
else if ( e . code ( ) = = ErrorCodes : : PART_IS_TEMPORARILY_LOCKED )
{
/// Part cannot be added temporarily
LOG_INFO ( log , e . displayText ( ) ) ;
2018-09-20 14:30:52 +00:00
cleanup_thread . wakeup ( ) ;
2017-11-20 19:33:12 +00:00
}
2017-04-01 07:20:54 +00:00
else
2018-04-06 21:46:57 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-04-01 07:20:54 +00:00
/** This exception will be written to the queue element, and it can be looked up using `system.replication_queue` table.
* The thread that performs this action will sleep a few seconds after the exception .
* See ` queue . processEntry ` function .
*/
throw ;
2020-12-03 13:54:05 +00:00
}
2017-04-01 07:20:54 +00:00
catch ( . . . )
{
2018-04-06 21:46:57 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-04-01 07:20:54 +00:00
throw ;
}
} ) ;
2014-04-03 11:48:28 +00:00
}
2021-09-08 00:21:21 +00:00
bool StorageReplicatedMergeTree : : scheduleDataProcessingJob ( BackgroundJobsAssignee & assignee )
2019-09-05 13:12:29 +00:00
{
2020-10-13 14:25:42 +00:00
/// If replication queue is stopped exit immediately as we successfully executed the task
if ( queue . actions_blocker . isCancelled ( ) )
2021-06-21 13:36:21 +00:00
return false ;
2019-08-19 17:59:16 +00:00
2020-10-13 14:25:42 +00:00
/// This object will mark the element of the queue as running.
2020-10-23 08:54:00 +00:00
ReplicatedMergeTreeQueue : : SelectedEntryPtr selected_entry = selectQueueEntry ( ) ;
2019-09-02 11:35:53 +00:00
2020-10-23 08:54:00 +00:00
if ( ! selected_entry )
2021-06-21 13:36:21 +00:00
return false ;
2014-10-18 17:37:55 +00:00
2021-09-16 21:19:58 +00:00
auto job_type = selected_entry - > log_entry - > type ;
2020-11-09 09:14:20 +00:00
/// Depending on entry type execute in fetches (small) pool or big merge_mutate pool
2021-09-16 21:19:58 +00:00
if ( job_type = = LogEntry : : GET_PART )
2021-06-21 13:36:21 +00:00
{
2021-09-08 00:21:21 +00:00
assignee . scheduleFetchTask ( ExecutableLambdaAdapter : : create (
2021-08-30 19:37:03 +00:00
[ this , selected_entry ] ( ) mutable
{
return processQueueEntry ( selected_entry ) ;
2021-09-02 21:31:32 +00:00
} , common_assignee_trigger , getStorageID ( ) ) ) ;
2021-06-21 13:36:21 +00:00
return true ;
}
2021-09-16 21:19:58 +00:00
else if ( job_type = = LogEntry : : MERGE_PARTS )
{
auto task = MergeFromLogEntryTask : : create ( selected_entry , * this , common_assignee_trigger ) ;
assignee . scheduleMergeMutateTask ( task ) ;
return true ;
}
else if ( job_type = = LogEntry : : MUTATE_PART )
{
auto task = MutateFromLogEntryTask : : create ( selected_entry , * this , common_assignee_trigger ) ;
assignee . scheduleMergeMutateTask ( task ) ;
return true ;
}
2020-10-26 11:02:47 +00:00
else
2019-08-19 17:59:16 +00:00
{
2021-09-30 21:26:24 +00:00
assignee . scheduleCommonTask ( ExecutableLambdaAdapter : : create (
2021-08-30 19:37:03 +00:00
[ this , selected_entry ] ( ) mutable
{
return processQueueEntry ( selected_entry ) ;
2021-11-09 12:26:51 +00:00
} , common_assignee_trigger , getStorageID ( ) ) , /* need_trigger */ true ) ;
2021-06-21 13:36:21 +00:00
return true ;
}
2019-09-05 13:12:29 +00:00
}
2019-08-19 17:59:16 +00:00
2020-10-26 11:02:47 +00:00
bool StorageReplicatedMergeTree : : canExecuteFetch ( const ReplicatedMergeTreeLogEntry & entry , String & disable_reason ) const
2019-09-05 13:12:29 +00:00
{
2020-10-26 11:02:47 +00:00
if ( fetcher . blocker . isCancelled ( ) )
2019-08-21 16:02:13 +00:00
{
2020-10-26 11:02:47 +00:00
disable_reason = fmt : : format ( " Not executing fetch of part {} because replicated fetches are cancelled now. " , entry . new_part_name ) ;
return false ;
2019-09-02 11:35:53 +00:00
}
2020-10-26 11:02:47 +00:00
size_t busy_threads_in_pool = CurrentMetrics : : values [ CurrentMetrics : : BackgroundFetchesPoolTask ] . load ( std : : memory_order_relaxed ) ;
if ( busy_threads_in_pool > = replicated_fetches_pool_size )
2019-08-19 17:59:16 +00:00
{
2020-10-26 11:02:47 +00:00
disable_reason = fmt : : format ( " Not executing fetch of part {} because {} fetches already executing, max {}. " , entry . new_part_name , busy_threads_in_pool , replicated_fetches_pool_size ) ;
return false ;
2019-08-19 17:59:16 +00:00
}
2020-10-26 11:02:47 +00:00
2021-05-27 12:54:47 +00:00
if ( replicated_fetches_throttler - > isThrottling ( ) )
{
disable_reason = fmt : : format ( " Not executing fetch of part {} because fetches have already throttled by network settings "
" <max_replicated_fetches_network_bandwidth> or <max_replicated_fetches_network_bandwidth_for_server>. " , entry . new_part_name ) ;
return false ;
}
2020-10-26 11:02:47 +00:00
return true ;
2019-08-19 17:59:16 +00:00
}
2014-10-18 17:37:55 +00:00
2019-09-05 13:12:29 +00:00
bool StorageReplicatedMergeTree : : partIsAssignedToBackgroundOperation ( const DataPartPtr & part ) const
{
2019-09-10 11:21:59 +00:00
return queue . isVirtualPart ( part ) ;
2019-09-05 13:12:29 +00:00
}
2019-08-20 09:59:19 +00:00
2018-05-31 13:05:05 +00:00
void StorageReplicatedMergeTree : : mergeSelectingTask ( )
2016-05-16 18:43:38 +00:00
{
2018-04-10 13:20:14 +00:00
if ( ! is_leader )
2017-12-29 22:32:04 +00:00
return ;
2017-12-21 18:17:06 +00:00
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2018-05-28 15:37:30 +00:00
const bool deduplicate = false ; /// TODO: read deduplicate option from table config
2020-12-01 09:10:12 +00:00
const Names deduplicate_by_columns = { } ;
2020-06-12 18:24:32 +00:00
CreateMergeEntryResult create_result = CreateMergeEntryResult : : Other ;
2017-04-01 07:20:54 +00:00
2017-12-29 22:32:04 +00:00
try
2017-11-17 08:58:35 +00:00
{
2018-05-28 15:37:30 +00:00
/// We must select parts for merge under merge_selecting_mutex because other threads
/// (OPTIMIZE queries) can assign new merges.
2019-01-02 06:44:36 +00:00
std : : lock_guard merge_selecting_lock ( merge_selecting_mutex ) ;
2017-12-21 18:17:06 +00:00
2018-05-28 15:37:30 +00:00
auto zookeeper = getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
2018-05-28 15:37:30 +00:00
ReplicatedMergeTreeMergePredicate merge_pred = queue . getMergePredicate ( zookeeper ) ;
2018-05-10 15:01:10 +00:00
2017-12-29 22:32:04 +00:00
/// If many merges is already queued, then will queue only small enough merges.
/// Otherwise merge queue could be filled with only large merges,
/// and in the same time, many small parts could be created and won't be merged.
2019-08-26 10:07:32 +00:00
2019-08-21 13:10:33 +00:00
auto merges_and_mutations_queued = queue . countMergesAndPartMutations ( ) ;
2020-09-03 13:00:13 +00:00
size_t merges_and_mutations_sum = merges_and_mutations_queued . merges + merges_and_mutations_queued . mutations ;
2019-08-26 18:08:58 +00:00
if ( merges_and_mutations_sum > = storage_settings_ptr - > max_replicated_merges_in_queue )
2017-12-29 22:32:04 +00:00
{
2020-06-12 18:24:32 +00:00
LOG_TRACE ( log , " Number of queued merges ({}) and part mutations ({}) "
" is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate. " ,
2020-09-03 13:00:13 +00:00
merges_and_mutations_queued . merges ,
merges_and_mutations_queued . mutations ,
2020-06-12 18:24:32 +00:00
storage_settings_ptr - > max_replicated_merges_in_queue ) ;
2017-12-29 22:32:04 +00:00
}
else
2017-04-01 07:20:54 +00:00
{
2019-06-17 19:41:48 +00:00
UInt64 max_source_parts_size_for_merge = merger_mutator . getMaxSourcePartsSizeForMerge (
2019-08-26 18:08:58 +00:00
storage_settings_ptr - > max_replicated_merges_in_queue , merges_and_mutations_sum ) ;
2020-09-02 08:18:50 +00:00
2019-06-17 19:41:48 +00:00
UInt64 max_source_part_size_for_mutation = merger_mutator . getMaxSourcePartSizeForMutation ( ) ;
2017-04-01 07:20:54 +00:00
2020-09-04 06:55:19 +00:00
bool merge_with_ttl_allowed = merges_and_mutations_queued . merges_with_ttl < storage_settings_ptr - > max_replicated_merges_with_ttl_in_queue & &
2020-09-04 10:08:09 +00:00
getTotalMergesWithTTLInMergeList ( ) < storage_settings_ptr - > max_number_of_merges_with_ttl_in_pool ;
2020-09-03 13:00:13 +00:00
2021-09-16 21:19:58 +00:00
auto future_merged_part = std : : make_shared < FutureMergedMutatedPart > ( ) ;
2020-11-02 14:38:18 +00:00
if ( storage_settings . get ( ) - > assign_part_uuids )
2021-09-16 21:19:58 +00:00
future_merged_part - > uuid = UUIDHelpers : : generateV4 ( ) ;
2020-11-02 14:38:18 +00:00
2019-06-17 19:41:48 +00:00
if ( max_source_parts_size_for_merge > 0 & &
2020-11-10 14:42:56 +00:00
merger_mutator . selectPartsToMerge ( future_merged_part , false , max_source_parts_size_for_merge , merge_pred , merge_with_ttl_allowed , nullptr ) = = SelectPartsDecision : : SELECTED )
2017-12-21 18:17:06 +00:00
{
2020-11-02 14:38:18 +00:00
create_result = createLogEntryToMergeParts (
zookeeper ,
2021-09-16 21:19:58 +00:00
future_merged_part - > parts ,
future_merged_part - > name ,
future_merged_part - > uuid ,
future_merged_part - > type ,
2020-11-02 14:38:18 +00:00
deduplicate ,
2020-12-01 09:10:12 +00:00
deduplicate_by_columns ,
2020-11-02 14:38:18 +00:00
nullptr ,
merge_pred . getVersion ( ) ,
2021-09-16 21:19:58 +00:00
future_merged_part - > merge_type ) ;
2019-06-17 19:41:48 +00:00
}
2020-06-12 18:24:32 +00:00
/// If there are many mutations in queue, it may happen, that we cannot enqueue enough merges to merge all new parts
2019-08-21 13:10:33 +00:00
else if ( max_source_part_size_for_mutation > 0 & & queue . countMutations ( ) > 0
2020-09-03 13:00:13 +00:00
& & merges_and_mutations_queued . mutations < storage_settings_ptr - > max_replicated_mutations_in_queue )
2019-06-17 19:41:48 +00:00
{
/// Choose a part to mutate.
DataPartsVector data_parts = getDataPartsVector ( ) ;
for ( const auto & part : data_parts )
2018-05-28 15:37:30 +00:00
{
2020-03-23 13:32:02 +00:00
if ( part - > getBytesOnDisk ( ) > max_source_part_size_for_mutation )
2019-06-17 19:41:48 +00:00
continue ;
2017-04-01 07:20:54 +00:00
2020-01-31 12:25:31 +00:00
std : : optional < std : : pair < Int64 , int > > desired_mutation_version = merge_pred . getDesiredMutationVersion ( part ) ;
2019-06-17 19:41:48 +00:00
if ( ! desired_mutation_version )
continue ;
2017-04-01 07:20:54 +00:00
2020-11-02 14:38:18 +00:00
create_result = createLogEntryToMutatePart (
* part ,
2021-09-16 21:19:58 +00:00
future_merged_part - > uuid ,
2020-11-02 14:38:18 +00:00
desired_mutation_version - > first ,
desired_mutation_version - > second ,
merge_pred . getVersion ( ) ) ;
2020-06-12 18:24:32 +00:00
if ( create_result = = CreateMergeEntryResult : : Ok )
2019-06-17 19:41:48 +00:00
break ;
2017-12-21 18:17:06 +00:00
}
2017-04-01 07:20:54 +00:00
}
}
2017-12-29 22:32:04 +00:00
}
catch ( . . . )
{
2018-04-10 13:20:14 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
2017-12-21 18:17:06 +00:00
}
2017-04-01 07:20:54 +00:00
2018-04-10 13:20:14 +00:00
if ( ! is_leader )
2017-12-29 22:32:04 +00:00
return ;
2017-11-17 08:58:35 +00:00
2020-06-12 18:24:32 +00:00
if ( create_result ! = CreateMergeEntryResult : : Ok
& & create_result ! = CreateMergeEntryResult : : LogUpdated )
{
2021-07-09 09:29:17 +00:00
merge_selecting_task - > scheduleAfter ( storage_settings_ptr - > merge_selecting_sleep_ms ) ;
2020-06-12 18:24:32 +00:00
}
2017-12-29 22:32:04 +00:00
else
2020-06-12 18:24:32 +00:00
{
2018-05-31 13:05:05 +00:00
merge_selecting_task - > schedule ( ) ;
2020-06-12 18:24:32 +00:00
}
2014-04-14 10:56:06 +00:00
}
2014-04-04 10:37:33 +00:00
2018-06-21 13:27:36 +00:00
void StorageReplicatedMergeTree : : mutationsFinalizingTask ( )
{
bool needs_reschedule = false ;
try
{
needs_reschedule = queue . tryFinalizeMutations ( getZooKeeper ( ) ) ;
}
catch ( . . . )
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
needs_reschedule = true ;
}
if ( needs_reschedule )
2020-04-27 16:19:04 +00:00
{
2018-06-21 13:27:36 +00:00
mutations_finalizing_task - > scheduleAfter ( MUTATIONS_FINALIZING_SLEEP_MS ) ;
2020-04-27 16:19:04 +00:00
}
else
{
/// Even if no mutations seems to be done or appeared we are trying to
/// finalize them in background because manual control the launch of
/// this function is error prone. This can lead to mutations that
/// processed all the parts but have is_done=0 state for a long time. Or
/// killed mutations, which are also considered as undone.
mutations_finalizing_task - > scheduleAfter ( MUTATIONS_FINALIZING_IDLE_SLEEP_MS ) ;
}
2018-06-21 13:27:36 +00:00
}
2020-06-12 18:24:32 +00:00
StorageReplicatedMergeTree : : CreateMergeEntryResult StorageReplicatedMergeTree : : createLogEntryToMergeParts (
2018-05-10 15:01:10 +00:00
zkutil : : ZooKeeperPtr & zookeeper ,
2019-05-03 02:00:57 +00:00
const DataPartsVector & parts ,
2018-05-10 15:01:10 +00:00
const String & merged_name ,
2020-11-02 14:38:18 +00:00
const UUID & merged_part_uuid ,
2020-02-11 13:41:26 +00:00
const MergeTreeDataPartType & merged_part_type ,
2018-05-10 15:01:10 +00:00
bool deduplicate ,
2020-12-01 09:10:12 +00:00
const Names & deduplicate_by_columns ,
2020-06-12 18:24:32 +00:00
ReplicatedMergeTreeLogEntryData * out_log_entry ,
2020-09-03 13:00:13 +00:00
int32_t log_version ,
MergeType merge_type )
2016-05-16 18:43:38 +00:00
{
2018-08-25 01:58:14 +00:00
std : : vector < std : : future < Coordination : : ExistsResponse > > exists_futures ;
2018-07-04 15:44:12 +00:00
exists_futures . reserve ( parts . size ( ) ) ;
2017-04-01 07:20:54 +00:00
for ( const auto & part : parts )
2021-05-08 10:59:55 +00:00
exists_futures . emplace_back ( zookeeper - > asyncExists ( fs : : path ( replica_path ) / " parts " / part - > name ) ) ;
2018-07-04 15:44:12 +00:00
bool all_in_zk = true ;
for ( size_t i = 0 ; i < parts . size ( ) ; + + i )
2017-04-01 07:20:54 +00:00
{
/// If there is no information about part in ZK, we will not merge it.
2020-06-12 15:09:12 +00:00
if ( exists_futures [ i ] . get ( ) . error = = Coordination : : Error : : ZNONODE )
2017-04-01 07:20:54 +00:00
{
all_in_zk = false ;
2018-07-04 15:44:12 +00:00
const auto & part = parts [ i ] ;
2017-10-02 16:34:01 +00:00
if ( part - > modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time ( nullptr ) )
2017-04-01 07:20:54 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Part {} (that was selected for merge) with age {} seconds exists locally but not in ZooKeeper. Won't do merge with that part and will check it. " , part - > name , ( time ( nullptr ) - part - > modification_time ) ) ;
2017-04-01 07:20:54 +00:00
enqueuePartForCheck ( part - > name ) ;
}
}
}
2018-07-04 15:44:12 +00:00
2017-04-01 07:20:54 +00:00
if ( ! all_in_zk )
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : MissingPart ;
2017-04-01 07:20:54 +00:00
2017-10-02 16:34:01 +00:00
ReplicatedMergeTreeLogEntryData entry ;
2017-04-01 07:20:54 +00:00
entry . type = LogEntry : : MERGE_PARTS ;
entry . source_replica = replica_name ;
entry . new_part_name = merged_name ;
2020-11-02 14:38:18 +00:00
entry . new_part_uuid = merged_part_uuid ;
2020-02-11 13:41:26 +00:00
entry . new_part_type = merged_part_type ;
2020-09-02 08:18:50 +00:00
entry . merge_type = merge_type ;
2017-04-17 15:14:56 +00:00
entry . deduplicate = deduplicate ;
2020-12-01 09:10:12 +00:00
entry . deduplicate_by_columns = deduplicate_by_columns ;
2017-08-04 14:00:26 +00:00
entry . create_time = time ( nullptr ) ;
2017-04-01 07:20:54 +00:00
for ( const auto & part : parts )
2018-04-20 16:18:16 +00:00
entry . source_parts . push_back ( part - > name ) ;
2017-04-01 07:20:54 +00:00
2020-06-12 18:24:32 +00:00
Coordination : : Requests ops ;
Coordination : : Responses responses ;
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " log/log- " , entry . toString ( ) ,
2020-06-12 18:24:32 +00:00
zkutil : : CreateMode : : PersistentSequential ) ) ;
2020-06-12 20:22:55 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " log " , " " , log_version ) ) ; /// Check and update version.
2020-06-12 18:24:32 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , responses ) ;
if ( code = = Coordination : : Error : : ZOK )
{
String path_created = dynamic_cast < const Coordination : : CreateResponse & > ( * responses . front ( ) ) . path_created ;
entry . znode_name = path_created . substr ( path_created . find_last_of ( ' / ' ) + 1 ) ;
2020-06-12 20:22:55 +00:00
2020-06-12 20:38:43 +00:00
ProfileEvents : : increment ( ProfileEvents : : CreatedLogEntryForMerge ) ;
2020-06-12 20:22:55 +00:00
LOG_TRACE ( log , " Created log entry {} for merge {} " , path_created , merged_name ) ;
2020-06-12 18:24:32 +00:00
}
else if ( code = = Coordination : : Error : : ZBADVERSION )
{
2020-06-12 20:38:43 +00:00
ProfileEvents : : increment ( ProfileEvents : : NotCreatedLogEntryForMerge ) ;
2020-06-12 20:22:55 +00:00
LOG_TRACE ( log , " Log entry is not created for merge {} because log was updated " , merged_name ) ;
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : LogUpdated ;
}
else
{
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
2017-04-01 07:20:54 +00:00
if ( out_log_entry )
* out_log_entry = entry ;
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : Ok ;
2016-05-16 18:43:38 +00:00
}
2020-06-12 18:24:32 +00:00
StorageReplicatedMergeTree : : CreateMergeEntryResult StorageReplicatedMergeTree : : createLogEntryToMutatePart (
2020-11-02 14:38:18 +00:00
const IMergeTreeDataPart & part , const UUID & new_part_uuid , Int64 mutation_version , int32_t alter_version , int32_t log_version )
2018-04-20 16:18:16 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
/// If there is no information about part in ZK, we will not mutate it.
2021-05-08 10:59:55 +00:00
if ( ! zookeeper - > exists ( fs : : path ( replica_path ) / " parts " / part . name ) )
2018-04-20 16:18:16 +00:00
{
if ( part . modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time ( nullptr ) )
{
2020-06-12 19:01:44 +00:00
LOG_WARNING ( log , " Part {} (that was selected for mutation) with age {} seconds exists locally but not in ZooKeeper. "
" Won't mutate that part and will check it. " , part . name , ( time ( nullptr ) - part . modification_time ) ) ;
2018-04-20 16:18:16 +00:00
enqueuePartForCheck ( part . name ) ;
}
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : MissingPart ;
2018-04-20 16:18:16 +00:00
}
MergeTreePartInfo new_part_info = part . info ;
new_part_info . mutation = mutation_version ;
2018-05-23 19:34:37 +00:00
String new_part_name = part . getNewName ( new_part_info ) ;
2018-04-20 16:18:16 +00:00
ReplicatedMergeTreeLogEntryData entry ;
entry . type = LogEntry : : MUTATE_PART ;
entry . source_replica = replica_name ;
entry . source_parts . push_back ( part . name ) ;
entry . new_part_name = new_part_name ;
2020-11-02 14:38:18 +00:00
entry . new_part_uuid = new_part_uuid ;
2018-04-20 16:18:16 +00:00
entry . create_time = time ( nullptr ) ;
2020-01-31 12:25:31 +00:00
entry . alter_version = alter_version ;
2018-04-20 16:18:16 +00:00
2020-06-12 18:24:32 +00:00
Coordination : : Requests ops ;
Coordination : : Responses responses ;
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " log/log- " , entry . toString ( ) ,
2020-06-12 18:24:32 +00:00
zkutil : : CreateMode : : PersistentSequential ) ) ;
2020-06-12 20:22:55 +00:00
ops . emplace_back ( zkutil : : makeSetRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " log " , " " , log_version ) ) ; /// Check and update version.
2020-06-12 18:24:32 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , responses ) ;
if ( code = = Coordination : : Error : : ZBADVERSION )
2020-06-12 20:22:55 +00:00
{
2020-06-12 20:38:43 +00:00
ProfileEvents : : increment ( ProfileEvents : : NotCreatedLogEntryForMutation ) ;
2020-06-12 20:22:55 +00:00
LOG_TRACE ( log , " Log entry is not created for mutation {} because log was updated " , new_part_name ) ;
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : LogUpdated ;
2020-06-12 20:22:55 +00:00
}
2020-06-12 18:24:32 +00:00
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2020-06-12 20:22:55 +00:00
2020-06-12 20:38:43 +00:00
ProfileEvents : : increment ( ProfileEvents : : CreatedLogEntryForMutation ) ;
2020-06-12 20:22:55 +00:00
LOG_TRACE ( log , " Created log entry for mutation {} " , new_part_name ) ;
2020-06-12 18:24:32 +00:00
return CreateMergeEntryResult : : Ok ;
2018-04-20 16:18:16 +00:00
}
2018-12-11 13:30:20 +00:00
void StorageReplicatedMergeTree : : removePartFromZooKeeper ( const String & part_name , Coordination : : Requests & ops , bool has_children )
2015-09-20 11:02:59 +00:00
{
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2015-09-20 11:02:59 +00:00
2018-12-11 13:30:20 +00:00
if ( has_children )
{
2021-05-08 10:59:55 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( fs : : path ( part_path ) / " checksums " , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeRemoveRequest ( fs : : path ( part_path ) / " columns " , - 1 ) ) ;
2018-12-11 13:30:20 +00:00
}
2018-03-24 00:45:04 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( part_path , - 1 ) ) ;
2015-09-20 11:02:59 +00:00
}
2021-01-27 10:07:18 +00:00
void StorageReplicatedMergeTree : : removePartFromZooKeeper ( const String & part_name )
{
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2021-01-27 10:07:18 +00:00
Coordination : : Stat stat ;
/// Part doesn't exist, nothing to remove
if ( ! zookeeper - > exists ( part_path , & stat ) )
return ;
Coordination : : Requests ops ;
removePartFromZooKeeper ( part_name , ops , stat . numChildren > 0 ) ;
zookeeper - > multi ( ops ) ;
}
2015-09-20 11:02:59 +00:00
2014-07-22 13:49:52 +00:00
void StorageReplicatedMergeTree : : removePartAndEnqueueFetch ( const String & part_name )
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2014-12-12 20:50:32 +00:00
2021-10-18 20:16:02 +00:00
/// We don't know exactly what happened to broken part
/// and we are going to remove all covered log entries.
/// It's quite dangerous, so clone covered parts to detached.
auto broken_part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
auto partition_range = getDataPartsPartitionRange ( broken_part_info . partition_id ) ;
for ( const auto & part : partition_range )
{
if ( ! broken_part_info . contains ( part - > info ) )
continue ;
/// Broken part itself ether already moved to detached or does not exist.
assert ( broken_part_info ! = part - > info ) ;
part - > makeCloneInDetached ( " covered-by-broken " , getInMemoryMetadataPtr ( ) ) ;
}
/// It's possible that queue contains entries covered by part_name.
/// For example, we had GET_PART all_1_42_5 and MUTATE_PART all_1_42_5_63,
/// then all_1_42_5_63 was executed by fetching, but part was written to disk incorrectly.
/// In this case we have to remove it as broken and create GET_PART all_1_42_5_63 to fetch it again,
/// but GET_PART all_1_42_5 may be still in the queue.
/// We should remove all covered entries before creating GET_PART entry, because:
/// 1. In the situation described above, we do not know how to merge/mutate all_1_42_5_63 from all_1_42_5,
/// so GET_PART all_1_42_5 (and all source parts) is useless. The only thing we can do is to fetch all_1_42_5_63.
/// 2. If all_1_42_5_63 is lost, then replication may stuck waiting for all_1_42_5_63 to appear,
/// because we may have some covered parts (more precisely, parts with the same min and max blocks)
queue . removePartProducingOpsInRange ( zookeeper , broken_part_info , { } ) ;
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2014-07-22 13:49:52 +00:00
2018-12-11 13:30:20 +00:00
Coordination : : Requests ops ;
time_t part_create_time = 0 ;
Coordination : : Stat stat ;
if ( zookeeper - > exists ( part_path , & stat ) )
{
2021-10-25 14:01:23 +00:00
/// Update version of /is_lost node to avoid race condition with cloneReplica(...).
/// cloneReplica(...) expects that if some entry was executed, then its new_part_name is added to /parts,
/// but we are going to remove it from /parts and add to queue again.
Coordination : : Stat is_lost_stat ;
String is_lost_value = zookeeper - > get ( replica_path + " /is_lost " , & is_lost_stat ) ;
assert ( is_lost_value = = " 0 " ) ;
ops . emplace_back ( zkutil : : makeSetRequest ( replica_path + " /is_lost " , is_lost_value , is_lost_stat . version ) ) ;
2018-12-11 13:30:20 +00:00
part_create_time = stat . ctime / 1000 ;
removePartFromZooKeeper ( part_name , ops , stat . numChildren > 0 ) ;
}
2017-04-01 07:20:54 +00:00
LogEntryPtr log_entry = std : : make_shared < LogEntry > ( ) ;
log_entry - > type = LogEntry : : GET_PART ;
2018-12-11 13:30:20 +00:00
log_entry - > create_time = part_create_time ;
2017-04-01 07:20:54 +00:00
log_entry - > source_replica = " " ;
log_entry - > new_part_name = part_name ;
2014-07-22 13:49:52 +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- " , log_entry - > toString ( ) ,
2017-04-01 07:20:54 +00:00
zkutil : : CreateMode : : PersistentSequential ) ) ;
2015-09-20 11:02:59 +00:00
2017-04-01 07:20:54 +00:00
auto results = zookeeper - > multi ( ops ) ;
2014-07-22 13:49:52 +00:00
2019-07-03 08:49:52 +00:00
String path_created = dynamic_cast < const Coordination : : CreateResponse & > ( * results . back ( ) ) . path_created ;
2017-04-01 07:20:54 +00:00
log_entry - > znode_name = path_created . substr ( path_created . find_last_of ( ' / ' ) + 1 ) ;
queue . insert ( zookeeper , log_entry ) ;
2014-07-22 13:49:52 +00:00
}
2014-10-18 17:37:55 +00:00
2018-04-06 16:06:07 +00:00
void StorageReplicatedMergeTree : : enterLeaderElection ( )
2014-04-04 10:37:33 +00:00
{
2018-04-06 16:06:07 +00:00
auto callback = [ this ] ( )
{
LOG_INFO ( log , " Became leader " ) ;
is_leader = true ;
2018-08-22 13:43:27 +00:00
merge_selecting_task - > activateAndSchedule ( ) ;
2018-04-06 16:06:07 +00:00
} ;
2016-10-14 11:47:11 +00:00
2018-04-06 16:06:07 +00:00
try
{
leader_election = std : : make_shared < zkutil : : LeaderElection > (
2021-04-10 23:33:54 +00:00
getContext ( ) - > getSchedulePool ( ) ,
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " leader_election " ,
2018-04-06 16:06:07 +00:00
* current_zookeeper , /// current_zookeeper lives for the lifetime of leader_election,
/// since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method.
callback ,
replica_name ) ;
}
catch ( . . . )
{
leader_election = nullptr ;
throw ;
}
}
void StorageReplicatedMergeTree : : exitLeaderElection ( )
{
if ( ! leader_election )
2017-04-01 07:20:54 +00:00
return ;
2016-10-14 09:59:10 +00:00
2018-04-06 16:06:07 +00:00
/// Shut down the leader election thread to avoid suddenly becoming the leader again after
/// we have stopped the merge_selecting_thread, but before we have deleted the leader_election object.
leader_election - > shutdown ( ) ;
if ( is_leader )
2017-12-21 18:17:06 +00:00
{
2018-04-06 16:06:07 +00:00
LOG_INFO ( log , " Stopped being leader " ) ;
is_leader = false ;
2018-05-31 13:05:05 +00:00
merge_selecting_task - > deactivate ( ) ;
2017-12-21 18:17:06 +00:00
}
2018-04-06 16:06:07 +00:00
/// Delete the node in ZK only after we have stopped the merge_selecting_thread - so that only one
/// replica assigns merges at any given time.
leader_election = nullptr ;
2014-04-04 10:37:33 +00:00
}
2021-04-10 23:33:54 +00:00
ConnectionTimeouts StorageReplicatedMergeTree : : getFetchPartHTTPTimeouts ( ContextPtr local_context )
2021-02-04 17:25:10 +00:00
{
2021-04-10 23:33:54 +00:00
auto timeouts = ConnectionTimeouts : : getHTTPTimeouts ( local_context ) ;
2021-02-04 17:25:10 +00:00
auto settings = getSettings ( ) ;
if ( settings - > replicated_fetches_http_connection_timeout . changed )
timeouts . connection_timeout = settings - > replicated_fetches_http_connection_timeout ;
if ( settings - > replicated_fetches_http_send_timeout . changed )
timeouts . send_timeout = settings - > replicated_fetches_http_send_timeout ;
if ( settings - > replicated_fetches_http_receive_timeout . changed )
timeouts . receive_timeout = settings - > replicated_fetches_http_receive_timeout ;
return timeouts ;
}
2020-09-18 10:57:33 +00:00
bool StorageReplicatedMergeTree : : checkReplicaHavePart ( const String & replica , const String & part_name )
{
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
return zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " parts " / part_name ) ;
2020-09-18 10:57:33 +00:00
}
2014-10-18 17:37:55 +00:00
2014-04-08 17:45:21 +00:00
String StorageReplicatedMergeTree : : findReplicaHavingPart ( const String & part_name , bool active )
2014-04-03 11:48:28 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2014-04-03 11:48:28 +00:00
2017-04-01 07:20:54 +00:00
/// Select replicas in uniformly random order.
2019-07-28 15:30:38 +00:00
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
2014-04-03 11:48:28 +00:00
2021-03-22 15:44:44 +00:00
LOG_TRACE ( log , " Candidate replicas: {} " , replicas . size ( ) ) ;
2017-04-01 07:20:54 +00:00
for ( const String & replica : replicas )
{
2021-02-14 22:59:13 +00:00
/// We aren't interested in ourself.
2017-04-01 07:20:54 +00:00
if ( replica = = replica_name )
continue ;
2016-08-10 04:20:50 +00:00
2021-03-22 15:44:44 +00:00
LOG_TRACE ( log , " Candidate replica: {} " , replica ) ;
2020-09-18 10:57:33 +00:00
if ( checkReplicaHavePart ( replica , part_name ) & &
2021-05-08 10:59:55 +00:00
( ! active | | zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) ) )
2017-04-01 07:20:54 +00:00
return replica ;
2015-09-15 01:20:40 +00:00
2017-04-01 07:20:54 +00:00
/// Obviously, replica could become inactive or even vanish after return from this method.
}
2014-04-03 11:48:28 +00:00
2017-04-01 07:20:54 +00:00
return { } ;
2016-08-09 20:39:28 +00:00
}
2018-05-23 14:33:55 +00:00
String StorageReplicatedMergeTree : : findReplicaHavingCoveringPart ( LogEntry & entry , bool active )
2016-08-09 20:39:28 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2017-04-01 07:20:54 +00:00
/// Select replicas in uniformly random order.
2019-07-28 15:30:38 +00:00
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
2017-04-01 07:20:54 +00:00
for ( const String & replica : replicas )
{
if ( replica = = replica_name )
continue ;
2021-05-08 10:59:55 +00:00
if ( active & & ! zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) )
2017-04-01 07:20:54 +00:00
continue ;
String largest_part_found ;
2021-05-08 10:59:55 +00:00
Strings parts = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " / replica / " parts " ) ;
2017-04-01 07:20:54 +00:00
for ( const String & part_on_replica : parts )
{
2017-08-25 20:41:45 +00:00
if ( part_on_replica = = entry . new_part_name
2019-05-03 02:00:57 +00:00
| | MergeTreePartInfo : : contains ( part_on_replica , entry . new_part_name , format_version ) )
2017-04-01 07:20:54 +00:00
{
2017-08-25 20:41:45 +00:00
if ( largest_part_found . empty ( )
2019-05-03 02:00:57 +00:00
| | MergeTreePartInfo : : contains ( part_on_replica , largest_part_found , format_version ) )
2017-04-01 07:20:54 +00:00
{
largest_part_found = part_on_replica ;
}
}
}
if ( ! largest_part_found . empty ( ) )
{
2017-05-12 13:47:42 +00:00
bool the_same_part = largest_part_found = = entry . new_part_name ;
/// Make a check in case if selected part differs from source part
if ( ! the_same_part )
{
String reject_reason ;
if ( ! queue . addFuturePartIfNotCoveredByThem ( largest_part_found , entry , reject_reason ) )
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Will not fetch part {} covering {}. {} " , largest_part_found , entry . new_part_name , reject_reason ) ;
2017-05-12 13:47:42 +00:00
return { } ;
}
}
2017-04-01 07:20:54 +00:00
return replica ;
}
}
return { } ;
2014-04-03 11:48:28 +00:00
}
2014-10-18 17:37:55 +00:00
2018-05-21 13:49:54 +00:00
String StorageReplicatedMergeTree : : findReplicaHavingCoveringPart (
const String & part_name , bool active , String & found_part_name )
{
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2018-05-21 13:49:54 +00:00
/// Select replicas in uniformly random order.
2019-07-28 15:30:38 +00:00
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
2018-05-21 13:49:54 +00:00
String largest_part_found ;
String largest_replica_found ;
for ( const String & replica : replicas )
{
if ( replica = = replica_name )
continue ;
2021-05-08 10:59:55 +00:00
if ( active & & ! zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) )
2018-05-21 13:49:54 +00:00
continue ;
2021-05-08 10:59:55 +00:00
Strings parts = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " / replica / " parts " ) ;
2018-05-21 13:49:54 +00:00
for ( const String & part_on_replica : parts )
{
if ( part_on_replica = = part_name
2019-05-03 02:00:57 +00:00
| | MergeTreePartInfo : : contains ( part_on_replica , part_name , format_version ) )
2018-05-21 13:49:54 +00:00
{
if ( largest_part_found . empty ( )
2019-05-03 02:00:57 +00:00
| | MergeTreePartInfo : : contains ( part_on_replica , largest_part_found , format_version ) )
2018-05-21 13:49:54 +00:00
{
largest_part_found = part_on_replica ;
largest_replica_found = replica ;
}
}
}
}
found_part_name = largest_part_found ;
return largest_replica_found ;
}
2017-03-12 19:18:07 +00:00
/** If a quorum is tracked for a part, update information about it in ZK.
2015-09-11 02:13:59 +00:00
*/
2020-10-07 11:28:48 +00:00
void StorageReplicatedMergeTree : : updateQuorum ( const String & part_name , bool is_parallel )
2015-09-11 02:13:59 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
/// Information on which replicas a part has been added, if the quorum has not yet been reached.
2021-05-08 10:59:55 +00:00
String quorum_status_path = fs : : path ( zookeeper_path ) / " quorum " / " status " ;
2020-10-09 11:20:20 +00:00
if ( is_parallel )
2021-05-08 10:59:55 +00:00
quorum_status_path = fs : : path ( zookeeper_path ) / " quorum " / " parallel " / part_name ;
2017-04-01 07:20:54 +00:00
/// The name of the previous part for which the quorum was reached.
2021-05-08 10:59:55 +00:00
const String quorum_last_part_path = fs : : path ( zookeeper_path ) / " quorum " / " last_part " ;
2017-04-01 07:20:54 +00:00
String value ;
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2017-04-01 07:20:54 +00:00
/// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed.
while ( zookeeper - > tryGet ( quorum_status_path , value , & stat ) )
{
2020-10-07 11:28:48 +00:00
ReplicatedMergeTreeQuorumEntry quorum_entry ( value ) ;
2017-04-01 07:20:54 +00:00
if ( quorum_entry . part_name ! = part_name )
{
2020-12-17 16:13:01 +00:00
LOG_TRACE ( log , " Quorum {}, already achieved for part {} current part {} " ,
quorum_status_path , part_name , quorum_entry . part_name ) ;
2017-04-01 07:20:54 +00:00
/// The quorum has already been achieved. Moreover, another INSERT with a quorum has already started.
break ;
}
quorum_entry . replicas . insert ( replica_name ) ;
if ( quorum_entry . replicas . size ( ) > = quorum_entry . required_number_of_replicas )
{
/// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum.
2020-12-17 16:13:01 +00:00
LOG_TRACE ( log , " Got {} replicas confirmed quorum {}, going to remove node " ,
quorum_entry . replicas . size ( ) , quorum_status_path ) ;
2017-04-01 07:20:54 +00:00
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
Coordination : : Responses responses ;
2018-10-23 08:19:47 +00:00
2020-10-07 11:28:48 +00:00
if ( ! is_parallel )
2020-10-06 21:49:48 +00:00
{
Coordination : : Stat added_parts_stat ;
String old_added_parts = zookeeper - > get ( quorum_last_part_path , & added_parts_stat ) ;
2018-09-17 12:28:29 +00:00
2020-10-06 21:49:48 +00:00
ReplicatedMergeTreeQuorumAddedParts parts_with_quorum ( format_version ) ;
2018-10-18 11:51:40 +00:00
2020-10-06 21:49:48 +00:00
if ( ! old_added_parts . empty ( ) )
parts_with_quorum . fromString ( old_added_parts ) ;
2018-09-19 11:08:04 +00:00
2020-10-06 21:49:48 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
/// We store one last part which reached quorum for each partition.
parts_with_quorum . added_parts [ part_info . partition_id ] = part_name ;
2018-10-23 08:19:47 +00:00
2020-10-06 21:49:48 +00:00
String new_added_parts = parts_with_quorum . toString ( ) ;
2018-10-23 08:19:47 +00:00
2020-10-06 21:49:48 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( quorum_status_path , stat . version ) ) ;
ops . emplace_back ( zkutil : : makeSetRequest ( quorum_last_part_path , new_added_parts , added_parts_stat . version ) ) ;
}
else
ops . emplace_back ( zkutil : : makeRemoveRequest ( quorum_status_path , stat . version ) ) ;
2018-08-14 09:57:18 +00:00
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 )
2017-04-01 07:20:54 +00:00
{
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZNONODE )
2017-04-01 07:20:54 +00:00
{
/// The quorum has already been achieved.
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZBADVERSION )
2017-04-01 07:20:54 +00:00
{
/// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue ;
}
else
2018-08-25 01:58:14 +00:00
throw Coordination : : Exception ( code , quorum_status_path ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2020-12-17 16:13:01 +00:00
LOG_TRACE ( log , " Quorum {} still not satisfied (have only {} replicas), updating node " ,
quorum_status_path , quorum_entry . replicas . size ( ) ) ;
2017-04-01 07:20:54 +00:00
/// We update the node, registering there one more replica.
auto code = zookeeper - > trySet ( quorum_status_path , quorum_entry . toString ( ) , stat . version ) ;
2020-06-12 15:09:12 +00:00
if ( code = = Coordination : : Error : : ZOK )
2017-04-01 07:20:54 +00:00
{
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZNONODE )
2017-04-01 07:20:54 +00:00
{
/// The quorum has already been achieved.
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZBADVERSION )
2017-04-01 07:20:54 +00:00
{
/// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue ;
}
else
2018-08-25 01:58:14 +00:00
throw Coordination : : Exception ( code , quorum_status_path ) ;
2017-04-01 07:20:54 +00:00
}
}
2015-09-11 02:13:59 +00:00
}
2020-04-20 10:56:59 +00:00
void StorageReplicatedMergeTree : : cleanLastPartNode ( const String & partition_id )
2020-04-10 21:29:54 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
/// The name of the previous part for which the quorum was reached.
2021-05-08 10:59:55 +00:00
const String quorum_last_part_path = fs : : path ( zookeeper_path ) / " quorum " / " last_part " ;
2020-04-10 21:29:54 +00:00
/// Delete information from "last_part" node.
while ( true )
{
Coordination : : Stat added_parts_stat ;
String old_added_parts = zookeeper - > get ( quorum_last_part_path , & added_parts_stat ) ;
ReplicatedMergeTreeQuorumAddedParts parts_with_quorum ( format_version ) ;
if ( ! old_added_parts . empty ( ) )
parts_with_quorum . fromString ( old_added_parts ) ;
/// Delete information about particular partition.
2020-04-23 09:00:43 +00:00
if ( ! parts_with_quorum . added_parts . count ( partition_id ) )
2020-04-10 21:29:54 +00:00
{
/// There is no information about interested part.
break ;
}
2020-04-20 10:56:59 +00:00
parts_with_quorum . added_parts . erase ( partition_id ) ;
2020-04-10 21:29:54 +00:00
String new_added_parts = parts_with_quorum . toString ( ) ;
auto code = zookeeper - > trySet ( quorum_last_part_path , new_added_parts , added_parts_stat . version ) ;
2020-06-12 15:09:12 +00:00
if ( code = = Coordination : : Error : : ZOK )
2020-04-10 21:29:54 +00:00
{
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZNONODE )
2020-04-10 21:29:54 +00:00
{
/// Node is deleted. It is impossible, but it is Ok.
break ;
}
2020-06-12 15:09:12 +00:00
else if ( code = = Coordination : : Error : : ZBADVERSION )
2020-04-10 21:29:54 +00:00
{
/// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue ;
}
else
throw Coordination : : Exception ( code , quorum_last_part_path ) ;
}
}
2020-11-03 09:24:10 +00:00
bool StorageReplicatedMergeTree : : partIsInsertingWithParallelQuorum ( const MergeTreePartInfo & part_info ) const
2020-10-01 10:38:50 +00:00
{
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
return zookeeper - > exists ( fs : : path ( zookeeper_path ) / " quorum " / " parallel " / part_info . getPartName ( ) ) ;
2020-11-03 09:24:10 +00:00
}
2020-10-01 10:38:50 +00:00
2021-03-10 13:27:08 +00:00
2020-11-03 09:24:10 +00:00
bool StorageReplicatedMergeTree : : partIsLastQuorumPart ( const MergeTreePartInfo & part_info ) const
{
auto zookeeper = getZooKeeper ( ) ;
2020-10-01 10:38:50 +00:00
2021-05-08 10:59:55 +00:00
const String parts_with_quorum_path = fs : : path ( zookeeper_path ) / " quorum " / " last_part " ;
2020-10-01 10:38:50 +00:00
2020-11-03 09:24:10 +00:00
String parts_with_quorum_str = zookeeper - > get ( parts_with_quorum_path ) ;
2020-10-01 10:38:50 +00:00
2020-11-03 09:24:10 +00:00
if ( parts_with_quorum_str . empty ( ) )
return false ;
2020-10-01 10:38:50 +00:00
2020-11-03 09:24:10 +00:00
ReplicatedMergeTreeQuorumAddedParts parts_with_quorum ( format_version ) ;
parts_with_quorum . fromString ( parts_with_quorum_str ) ;
2020-10-01 10:38:50 +00:00
2020-11-03 09:24:10 +00:00
auto partition_it = parts_with_quorum . added_parts . find ( part_info . partition_id ) ;
if ( partition_it = = parts_with_quorum . added_parts . end ( ) )
return false ;
2020-10-01 10:38:50 +00:00
2020-11-03 09:24:10 +00:00
return partition_it - > second = = part_info . getPartName ( ) ;
2020-10-01 10:38:50 +00:00
}
2021-03-10 13:27:08 +00:00
2020-06-26 11:30:23 +00:00
bool StorageReplicatedMergeTree : : fetchPart ( const String & part_name , const StorageMetadataPtr & metadata_snapshot ,
2020-08-27 14:19:18 +00:00
const String & source_replica_path , bool to_detached , size_t quorum , zkutil : : ZooKeeper : : Ptr zookeeper_ )
2014-04-03 11:48:28 +00:00
{
2020-08-27 14:19:18 +00:00
auto zookeeper = zookeeper_ ? zookeeper_ : getZooKeeper ( ) ;
2019-05-03 02:00:57 +00:00
const auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
2018-09-11 14:41:04 +00:00
2020-10-30 02:55:50 +00:00
if ( ! to_detached )
2017-10-03 19:04:56 +00:00
{
2020-10-30 02:55:50 +00:00
if ( auto part = getPartIfExists ( part_info , { IMergeTreeDataPart : : State : : Outdated , IMergeTreeDataPart : : State : : Deleting } ) )
{
LOG_DEBUG ( log , " Part {} should be deleted after previous attempt before fetch " , part - > name ) ;
/// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
cleanup_thread . wakeup ( ) ;
return false ;
}
2017-10-03 19:04:56 +00:00
}
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( currently_fetching_parts_mutex ) ;
2017-04-01 07:20:54 +00:00
if ( ! currently_fetching_parts . insert ( part_name ) . second )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Part {} is already fetching right now " , part_name ) ;
2017-04-01 07:20:54 +00:00
return false ;
}
}
2021-04-04 09:23:40 +00:00
SCOPE_EXIT_MEMORY
2017-04-01 07:20:54 +00:00
( {
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( currently_fetching_parts_mutex ) ;
2017-04-01 07:20:54 +00:00
currently_fetching_parts . erase ( part_name ) ;
} ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Fetching part {} from {} " , part_name , source_replica_path ) ;
2017-04-01 07:20:54 +00:00
2020-06-18 16:10:47 +00:00
TableLockHolder table_lock_holder ;
2017-04-01 07:20:54 +00:00
if ( ! to_detached )
2020-06-18 16:10:47 +00:00
table_lock_holder = lockForShare ( RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
/// Logging
2017-04-01 07:20:54 +00:00
Stopwatch stopwatch ;
2019-05-03 02:00:57 +00:00
MutableDataPartPtr part ;
DataPartsVector replaced_parts ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
auto write_part_log = [ & ] ( const ExecutionStatus & execution_status )
2017-04-01 07:20:54 +00:00
{
2018-05-25 19:44:14 +00:00
writePartLog (
PartLogElement : : DOWNLOAD_PART , execution_status , stopwatch . elapsed ( ) ,
part_name , part , replaced_parts , nullptr ) ;
2018-01-23 22:56:46 +00:00
} ;
2019-05-03 02:00:57 +00:00
DataPartPtr part_to_clone ;
2018-09-11 14:41:04 +00:00
{
/// If the desired part is a result of a part mutation, try to find the source part and compare
/// its checksums to the checksums of the desired part. If they match, we can just clone the local part.
/// If we have the source part, its part_info will contain covered_part_info.
auto covered_part_info = part_info ;
covered_part_info . mutation = 0 ;
2019-05-03 02:00:57 +00:00
auto source_part = getActiveContainingPart ( covered_part_info ) ;
2018-05-21 13:49:54 +00:00
2018-09-11 14:41:04 +00:00
if ( source_part )
{
MinimalisticDataPartChecksums source_part_checksums ;
source_part_checksums . computeTotalChecksums ( source_part - > checksums ) ;
2018-12-11 13:30:20 +00:00
MinimalisticDataPartChecksums desired_checksums ;
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( source_replica_path ) / " parts " / part_name ;
2018-12-11 13:30:20 +00:00
String part_znode = zookeeper - > get ( part_path ) ;
2021-06-20 08:24:43 +00:00
2018-12-11 13:30:20 +00:00
if ( ! part_znode . empty ( ) )
desired_checksums = ReplicatedMergeTreePartHeader : : fromString ( part_znode ) . getChecksums ( ) ;
else
{
2021-05-08 10:59:55 +00:00
String desired_checksums_str = zookeeper - > get ( fs : : path ( part_path ) / " checksums " ) ;
2018-12-11 13:30:20 +00:00
desired_checksums = MinimalisticDataPartChecksums : : deserializeFrom ( desired_checksums_str ) ;
}
2018-09-11 14:41:04 +00:00
if ( source_part_checksums = = desired_checksums )
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Found local part {} with the same checksums as {} " , source_part - > name , part_name ) ;
2018-09-11 14:41:04 +00:00
part_to_clone = source_part ;
}
}
}
2021-03-10 13:27:08 +00:00
ReplicatedMergeTreeAddress address ;
ConnectionTimeouts timeouts ;
String interserver_scheme ;
2021-04-07 16:06:20 +00:00
InterserverCredentialsPtr credentials ;
2021-03-08 09:38:07 +00:00
std : : optional < CurrentlySubmergingEmergingTagger > tagger_ptr ;
2019-05-03 02:00:57 +00:00
std : : function < MutableDataPartPtr ( ) > get_part ;
2021-03-17 15:49:04 +00:00
2018-09-11 14:41:04 +00:00
if ( part_to_clone )
2018-01-23 22:56:46 +00:00
{
2018-09-11 14:41:04 +00:00
get_part = [ & , part_to_clone ] ( )
{
2020-06-26 11:30:23 +00:00
return cloneAndLoadDataPartOnSameDisk ( part_to_clone , " tmp_clone_ " , part_info , metadata_snapshot ) ;
2018-09-11 14:41:04 +00:00
} ;
}
else
{
2021-05-08 10:59:55 +00:00
address . fromString ( zookeeper - > get ( fs : : path ( source_replica_path ) / " host " ) ) ;
2021-04-10 23:33:54 +00:00
timeouts = getFetchPartHTTPTimeouts ( getContext ( ) ) ;
2021-02-04 17:25:10 +00:00
2021-04-10 23:33:54 +00:00
credentials = getContext ( ) - > getInterserverCredentials ( ) ;
interserver_scheme = getContext ( ) - > getInterserverScheme ( ) ;
2018-09-11 14:41:04 +00:00
2021-04-07 13:52:11 +00:00
get_part = [ & , address , timeouts , credentials , interserver_scheme ] ( )
2018-09-11 14:41:04 +00:00
{
if ( interserver_scheme ! = address . scheme )
throw Exception ( " Interserver schemes are different: ' " + interserver_scheme
+ " ' != ' " + address . scheme + " ', can't fetch part from " + address . host ,
2020-12-25 13:38:04 +00:00
ErrorCodes : : INTERSERVER_SCHEME_DOESNT_MATCH ) ;
2018-07-31 10:34:35 +00:00
2018-09-11 14:41:04 +00:00
return fetcher . fetchPart (
2021-03-08 09:38:07 +00:00
metadata_snapshot ,
2021-05-21 16:14:01 +00:00
getContext ( ) ,
2021-03-08 09:38:07 +00:00
part_name ,
source_replica_path ,
address . host ,
address . replication_port ,
timeouts ,
2021-04-07 13:52:11 +00:00
credentials - > getUser ( ) ,
credentials - > getPassword ( ) ,
2021-03-08 09:38:07 +00:00
interserver_scheme ,
2021-05-26 20:37:44 +00:00
replicated_fetches_throttler ,
2021-03-08 09:38:07 +00:00
to_detached ,
" " ,
2021-03-12 09:58:32 +00:00
& tagger_ptr ,
true ) ;
2018-09-11 14:41:04 +00:00
} ;
}
2018-05-21 13:49:54 +00:00
2018-01-23 22:56:46 +00:00
try
{
2018-09-11 14:41:04 +00:00
part = get_part ( ) ;
2017-11-20 19:33:12 +00:00
2018-01-23 22:56:46 +00:00
if ( ! to_detached )
2017-04-01 07:20:54 +00:00
{
2019-05-03 02:00:57 +00:00
Transaction transaction ( * this ) ;
renameTempPartAndReplace ( part , nullptr , & transaction ) ;
2017-04-01 07:20:54 +00:00
2018-03-21 23:30:20 +00:00
replaced_parts = checkPartChecksumsAndCommit ( transaction , part ) ;
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
/** If a quorum is tracked for this part, you must update it.
* If you do not have time , in case of losing the session , when you restart the server - see the ` ReplicatedMergeTreeRestartingThread : : updateQuorumIfWeHavePart ` method .
*/
if ( quorum )
2020-10-07 11:28:48 +00:00
{
/// Check if this quorum insert is parallel or not
2021-05-08 10:59:55 +00:00
if ( zookeeper - > exists ( fs : : path ( zookeeper_path ) / " quorum " / " parallel " / part_name ) )
2020-10-07 11:28:48 +00:00
updateQuorum ( part_name , true ) ;
2021-05-08 10:59:55 +00:00
else if ( zookeeper - > exists ( fs : : path ( zookeeper_path ) / " quorum " / " status " ) )
2020-10-08 15:35:41 +00:00
updateQuorum ( part_name , false ) ;
2020-10-07 11:28:48 +00:00
}
2020-10-06 21:49:48 +00:00
/// merged parts that are still inserted with quorum. if it only contains one block, it hasn't been merged before
2020-10-07 11:28:48 +00:00
if ( part_info . level ! = 0 | | part_info . mutation ! = 0 )
2020-10-06 21:49:48 +00:00
{
2021-05-08 10:59:55 +00:00
Strings quorum_parts = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " quorum " / " parallel " ) ;
2020-10-06 21:49:48 +00:00
for ( const String & quorum_part : quorum_parts )
{
auto quorum_part_info = MergeTreePartInfo : : fromPartName ( quorum_part , format_version ) ;
if ( part_info . contains ( quorum_part_info ) )
2020-10-07 11:28:48 +00:00
updateQuorum ( quorum_part , true ) ;
2020-10-06 21:49:48 +00:00
}
}
2017-04-01 07:20:54 +00:00
2018-05-31 13:05:05 +00:00
merge_selecting_task - > schedule ( ) ;
2018-01-23 22:56:46 +00:00
for ( const auto & replaced_part : replaced_parts )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Part {} is rendered obsolete by fetching part {} " , replaced_part - > name , part_name ) ;
2018-01-23 22:56:46 +00:00
ProfileEvents : : increment ( ProfileEvents : : ObsoleteReplicatedParts ) ;
}
2017-04-01 07:20:54 +00:00
2018-01-23 22:56:46 +00:00
write_part_log ( { } ) ;
}
else
2017-04-01 07:20:54 +00:00
{
2020-08-27 14:19:18 +00:00
// The fetched part is valuable and should not be cleaned like a temp part.
part - > is_temp = false ;
2021-05-08 10:59:55 +00:00
part - > renameTo ( fs : : path ( " detached " ) / part_name , true ) ;
2017-04-01 07:20:54 +00:00
}
}
2020-08-28 00:53:22 +00:00
catch ( const Exception & e )
{
/// The same part is being written right now (but probably it's not committed yet).
/// We will check the need for fetch later.
if ( e . code ( ) = = ErrorCodes : : DIRECTORY_ALREADY_EXISTS )
return false ;
throw ;
}
2018-01-23 22:56:46 +00:00
catch ( . . . )
2017-04-01 07:20:54 +00:00
{
2018-01-23 22:56:46 +00:00
if ( ! to_detached )
write_part_log ( ExecutionStatus : : fromCurrentException ( ) ) ;
throw ;
2017-04-01 07:20:54 +00:00
}
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartFetches ) ;
2018-09-11 14:41:04 +00:00
if ( part_to_clone )
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Cloned part {} from {}{} " , part_name , part_to_clone - > name , to_detached ? " (to 'detached' directory) " : " " ) ;
2018-09-11 14:41:04 +00:00
else
2020-05-24 19:17:15 +00:00
LOG_DEBUG ( log , " Fetched part {} from {}{} " , part_name , source_replica_path , to_detached ? " (to 'detached' directory) " : " " ) ;
2018-09-11 14:41:04 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2014-04-03 11:48:28 +00:00
}
2014-03-21 19:17:59 +00:00
2014-10-18 17:37:55 +00:00
2021-03-09 17:49:50 +00:00
bool StorageReplicatedMergeTree : : fetchExistsPart ( const String & part_name , const StorageMetadataPtr & metadata_snapshot ,
const String & source_replica_path , DiskPtr replaced_disk , String replaced_part_path )
{
auto zookeeper = getZooKeeper ( ) ;
const auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
if ( auto part = getPartIfExists ( part_info , { IMergeTreeDataPart : : State : : Outdated , IMergeTreeDataPart : : State : : Deleting } ) )
{
LOG_DEBUG ( log , " Part {} should be deleted after previous attempt before fetch " , part - > name ) ;
/// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
cleanup_thread . wakeup ( ) ;
return false ;
}
{
std : : lock_guard lock ( currently_fetching_parts_mutex ) ;
if ( ! currently_fetching_parts . insert ( part_name ) . second )
{
LOG_DEBUG ( log , " Part {} is already fetching right now " , part_name ) ;
return false ;
}
}
2021-04-04 09:23:40 +00:00
SCOPE_EXIT_MEMORY
2021-03-09 17:49:50 +00:00
( {
std : : lock_guard lock ( currently_fetching_parts_mutex ) ;
currently_fetching_parts . erase ( part_name ) ;
} ) ;
LOG_DEBUG ( log , " Fetching part {} from {} " , part_name , source_replica_path ) ;
TableLockHolder table_lock_holder = lockForShare ( RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
/// Logging
Stopwatch stopwatch ;
MutableDataPartPtr part ;
DataPartsVector replaced_parts ;
auto write_part_log = [ & ] ( const ExecutionStatus & execution_status )
{
writePartLog (
PartLogElement : : DOWNLOAD_PART , execution_status , stopwatch . elapsed ( ) ,
part_name , part , replaced_parts , nullptr ) ;
} ;
std : : function < MutableDataPartPtr ( ) > get_part ;
2021-05-08 10:59:55 +00:00
ReplicatedMergeTreeAddress address ( zookeeper - > get ( fs : : path ( source_replica_path ) / " host " ) ) ;
2021-04-10 23:33:54 +00:00
auto timeouts = ConnectionTimeouts : : getHTTPTimeouts ( getContext ( ) ) ;
auto credentials = getContext ( ) - > getInterserverCredentials ( ) ;
String interserver_scheme = getContext ( ) - > getInterserverScheme ( ) ;
2021-03-09 17:49:50 +00:00
2021-04-07 13:52:11 +00:00
get_part = [ & , address , timeouts , interserver_scheme , credentials ] ( )
2021-03-10 13:27:08 +00:00
{
if ( interserver_scheme ! = address . scheme )
throw Exception ( " Interserver schemes are different: ' " + interserver_scheme
+ " ' != ' " + address . scheme + " ', can't fetch part from " + address . host ,
ErrorCodes : : INTERSERVER_SCHEME_DOESNT_MATCH ) ;
2021-03-09 17:49:50 +00:00
2021-03-10 13:27:08 +00:00
return fetcher . fetchPart (
2021-05-21 16:14:01 +00:00
metadata_snapshot , getContext ( ) , part_name , source_replica_path ,
2021-03-10 13:27:08 +00:00
address . host , address . replication_port ,
2021-05-26 20:37:44 +00:00
timeouts , credentials - > getUser ( ) , credentials - > getPassword ( ) ,
interserver_scheme , replicated_fetches_throttler , false , " " , nullptr , true ,
2021-03-10 13:27:08 +00:00
replaced_disk ) ;
} ;
2021-03-09 17:49:50 +00:00
try
{
part = get_part ( ) ;
if ( part - > volume - > getDisk ( ) - > getName ( ) ! = replaced_disk - > getName ( ) )
throw Exception ( " Part " + part - > name + " fetched on wrong disk " + part - > volume - > getDisk ( ) - > getName ( ) , ErrorCodes : : LOGICAL_ERROR ) ;
replaced_disk - > removeFileIfExists ( replaced_part_path ) ;
replaced_disk - > moveDirectory ( part - > getFullRelativePath ( ) , replaced_part_path ) ;
}
catch ( const Exception & e )
{
/// The same part is being written right now (but probably it's not committed yet).
/// We will check the need for fetch later.
if ( e . code ( ) = = ErrorCodes : : DIRECTORY_ALREADY_EXISTS )
return false ;
throw ;
}
catch ( . . . )
{
write_part_log ( ExecutionStatus : : fromCurrentException ( ) ) ;
throw ;
}
ProfileEvents : : increment ( ProfileEvents : : ReplicatedPartFetches ) ;
LOG_DEBUG ( log , " Fetched part {} from {} " , part_name , source_replica_path ) ;
return true ;
}
2017-06-06 17:06:14 +00:00
void StorageReplicatedMergeTree : : startup ( )
{
if ( is_readonly )
return ;
2020-06-03 22:11:06 +00:00
try
{
2020-10-21 19:24:16 +00:00
InterserverIOEndpointPtr data_parts_exchange_ptr = std : : make_shared < DataPartsExchange : : Service > ( * this ) ;
[[maybe_unused]] auto prev_ptr = std : : atomic_exchange ( & data_parts_exchange_endpoint , data_parts_exchange_ptr ) ;
assert ( prev_ptr = = nullptr ) ;
2021-04-10 23:33:54 +00:00
getContext ( ) - > getInterserverIOHandler ( ) . addEndpoint ( data_parts_exchange_ptr - > getId ( replica_path ) , data_parts_exchange_ptr ) ;
2017-12-25 14:56:32 +00:00
2020-06-03 22:11:06 +00:00
/// In this thread replica will be activated.
restarting_thread . start ( ) ;
2018-03-22 19:46:01 +00:00
2020-08-08 00:47:03 +00:00
/// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attempt to do it
2020-06-03 22:11:06 +00:00
startup_event . wait ( ) ;
2020-02-18 17:31:49 +00:00
2020-06-23 16:40:58 +00:00
startBackgroundMovesIfNeeded ( ) ;
2020-11-24 14:24:48 +00:00
part_moves_between_shards_orchestrator . start ( ) ;
2020-06-03 22:11:06 +00:00
}
catch ( . . . )
2020-03-07 03:52:50 +00:00
{
2020-06-03 22:11:06 +00:00
/// Exception safety: failed "startup" does not require a call to "shutdown" from the caller.
/// And it should be able to safely destroy table after exception in "startup" method.
2020-06-03 22:15:13 +00:00
/// It means that failed "startup" must not create any background tasks that we will have to wait.
2020-06-03 22:11:06 +00:00
try
{
shutdown ( ) ;
}
catch ( . . . )
{
std : : terminate ( ) ;
}
/// Note: after failed "startup", the table will be in a state that only allows to destroy the object.
throw ;
2020-03-07 03:52:50 +00:00
}
2017-06-06 17:06:14 +00:00
}
2014-03-22 14:44:44 +00:00
void StorageReplicatedMergeTree : : shutdown ( )
{
2018-08-21 14:03:06 +00:00
/// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
fetcher . blocker . cancelForever ( ) ;
2019-08-01 15:36:12 +00:00
merger_mutator . merges_blocker . cancelForever ( ) ;
2019-09-02 11:35:53 +00:00
parts_mover . moves_blocker . cancelForever ( ) ;
2018-08-21 14:03:06 +00:00
restarting_thread . shutdown ( ) ;
2021-09-08 00:21:21 +00:00
background_operations_assignee . finish ( ) ;
2020-11-24 14:24:48 +00:00
part_moves_between_shards_orchestrator . shutdown ( ) ;
2020-03-27 10:53:04 +00:00
{
auto lock = queue . lockQueue ( ) ;
2020-05-13 03:35:43 +00:00
/// Cancel logs pulling after background task were cancelled. It's still
/// required because we can trigger pullLogsToQueue during manual OPTIMIZE,
/// MUTATE, etc. query.
queue . pull_log_blocker . cancelForever ( ) ;
2020-03-27 10:53:04 +00:00
}
2021-09-08 00:21:21 +00:00
background_moves_assignee . finish ( ) ;
2017-04-01 07:20:54 +00:00
2020-10-21 19:24:16 +00:00
auto data_parts_exchange_ptr = std : : atomic_exchange ( & data_parts_exchange_endpoint , InterserverIOEndpointPtr { } ) ;
if ( data_parts_exchange_ptr )
2017-04-01 07:20:54 +00:00
{
2021-04-10 23:33:54 +00:00
getContext ( ) - > getInterserverIOHandler ( ) . removeEndpointIfExists ( data_parts_exchange_ptr - > getId ( replica_path ) ) ;
2020-01-14 14:27:48 +00:00
/// Ask all parts exchange handlers to finish asap. New ones will fail to start
2020-10-21 19:24:16 +00:00
data_parts_exchange_ptr - > blocker . cancelForever ( ) ;
2020-01-14 14:27:48 +00:00
/// Wait for all of them
2020-10-21 19:24:16 +00:00
std : : unique_lock lock ( data_parts_exchange_ptr - > rwlock ) ;
2017-04-01 07:20:54 +00:00
}
2014-04-25 13:55:15 +00:00
}
2014-03-22 14:44:44 +00:00
StorageReplicatedMergeTree : : ~ StorageReplicatedMergeTree ( )
{
2017-04-01 07:20:54 +00:00
try
{
shutdown ( ) ;
}
2018-11-23 18:52:00 +00:00
catch ( . . . )
2017-04-01 07:20:54 +00:00
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
2014-03-22 14:44:44 +00:00
}
2014-10-18 17:37:55 +00:00
2019-10-28 17:27:43 +00:00
ReplicatedMergeTreeQuorumAddedParts : : PartitionIdToMaxBlock StorageReplicatedMergeTree : : getMaxAddedBlocks ( ) const
{
ReplicatedMergeTreeQuorumAddedParts : : PartitionIdToMaxBlock max_added_blocks ;
for ( const auto & data_part : getDataParts ( ) )
{
max_added_blocks [ data_part - > info . partition_id ]
= std : : max ( max_added_blocks [ data_part - > info . partition_id ] , data_part - > info . max_block ) ;
}
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
const String quorum_status_path = fs : : path ( zookeeper_path ) / " quorum " / " status " ;
2019-10-28 17:27:43 +00:00
String value ;
Coordination : : Stat stat ;
if ( zookeeper - > tryGet ( quorum_status_path , value , & stat ) )
{
ReplicatedMergeTreeQuorumEntry quorum_entry ;
quorum_entry . fromString ( value ) ;
auto part_info = MergeTreePartInfo : : fromPartName ( quorum_entry . part_name , format_version ) ;
max_added_blocks [ part_info . partition_id ] = part_info . max_block - 1 ;
}
String added_parts_str ;
2021-05-08 10:59:55 +00:00
if ( zookeeper - > tryGet ( fs : : path ( zookeeper_path ) / " quorum " / " last_part " , added_parts_str ) )
2019-10-28 17:27:43 +00:00
{
if ( ! added_parts_str . empty ( ) )
{
ReplicatedMergeTreeQuorumAddedParts part_with_quorum ( format_version ) ;
part_with_quorum . fromString ( added_parts_str ) ;
auto added_parts = part_with_quorum . added_parts ;
for ( const auto & added_part : added_parts )
if ( ! getActiveContainingPart ( added_part . second ) )
throw Exception (
" Replica doesn't have part " + added_part . second
+ " which was successfully written to quorum of other replicas. "
" Send query to another replica or disable 'select_sequential_consistency' setting. " ,
ErrorCodes : : REPLICA_IS_NOT_IN_QUORUM ) ;
for ( const auto & max_block : part_with_quorum . getMaxInsertedBlocks ( ) )
max_added_blocks [ max_block . first ] = max_block . second ;
}
}
return max_added_blocks ;
}
2020-10-01 17:34:22 +00:00
void StorageReplicatedMergeTree : : read (
QueryPlan & query_plan ,
2017-04-01 07:20:54 +00:00
const Names & column_names ,
2020-06-16 14:25:08 +00:00
const StorageMetadataPtr & metadata_snapshot ,
2020-09-20 17:52:17 +00:00
SelectQueryInfo & query_info ,
2021-04-10 23:33:54 +00:00
ContextPtr local_context ,
2021-02-10 14:12:49 +00:00
QueryProcessingStage : : Enum processed_stage ,
2019-02-18 23:38:44 +00:00
const size_t max_block_size ,
2017-06-02 15:54:39 +00:00
const unsigned num_streams )
2014-03-22 14:44:44 +00:00
{
2017-05-24 21:38:56 +00:00
/** The `select_sequential_consistency` setting has two meanings:
* 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas .
* 2. Do not read parts that have not yet been written to the quorum of the replicas .
* For this you have to synchronously go to ZooKeeper .
*/
2021-04-10 23:33:54 +00:00
if ( local_context - > getSettingsRef ( ) . select_sequential_consistency )
2017-04-01 07:20:54 +00:00
{
2021-05-27 16:53:58 +00:00
auto max_added_blocks = std : : make_shared < ReplicatedMergeTreeQuorumAddedParts : : PartitionIdToMaxBlock > ( getMaxAddedBlocks ( ) ) ;
2021-02-10 14:12:49 +00:00
if ( auto plan = reader . read (
2021-05-27 16:53:58 +00:00
column_names , metadata_snapshot , query_info , local_context , max_block_size , num_streams , processed_stage , std : : move ( max_added_blocks ) ) )
2020-11-10 10:26:26 +00:00
query_plan = std : : move ( * plan ) ;
2020-10-01 17:34:22 +00:00
return ;
2019-10-28 17:27:43 +00:00
}
2018-09-19 14:34:41 +00:00
2021-02-10 14:12:49 +00:00
if ( auto plan = reader . read ( column_names , metadata_snapshot , query_info , local_context , max_block_size , num_streams , processed_stage ) )
2020-11-10 10:26:26 +00:00
query_plan = std : : move ( * plan ) ;
2020-10-01 17:34:22 +00:00
}
Pipe StorageReplicatedMergeTree : : read (
const Names & column_names ,
const StorageMetadataPtr & metadata_snapshot ,
2020-11-10 12:02:22 +00:00
SelectQueryInfo & query_info ,
2021-04-10 23:33:54 +00:00
ContextPtr local_context ,
2020-10-01 17:34:22 +00:00
QueryProcessingStage : : Enum processed_stage ,
const size_t max_block_size ,
const unsigned num_streams )
{
QueryPlan plan ;
2021-04-10 23:33:54 +00:00
read ( plan , column_names , metadata_snapshot , query_info , local_context , processed_stage , max_block_size , num_streams ) ;
2021-03-04 17:38:12 +00:00
return plan . convertToPipe (
2021-04-10 23:33:54 +00:00
QueryPlanOptimizationSettings : : fromContext ( local_context ) ,
BuildQueryPipelineSettings : : fromContext ( local_context ) ) ;
2019-10-28 17:27:43 +00:00
}
2018-10-18 12:16:53 +00:00
2018-10-18 11:51:40 +00:00
2020-03-29 08:50:27 +00:00
template < class Func >
2020-11-25 13:47:32 +00:00
void StorageReplicatedMergeTree : : foreachCommittedParts ( Func & & func , bool select_sequential_consistency ) const
2019-10-28 17:27:43 +00:00
{
2020-09-30 23:50:58 +00:00
std : : optional < ReplicatedMergeTreeQuorumAddedParts : : PartitionIdToMaxBlock > max_added_blocks = { } ;
2020-10-04 09:42:03 +00:00
/**
* Synchronously go to ZooKeeper when select_sequential_consistency enabled
*/
2020-11-25 13:47:32 +00:00
if ( select_sequential_consistency )
2020-09-30 23:50:58 +00:00
max_added_blocks = getMaxAddedBlocks ( ) ;
2019-10-28 17:27:43 +00:00
auto lock = lockParts ( ) ;
2020-04-22 06:22:14 +00:00
for ( const auto & part : getDataPartsStateRange ( DataPartState : : Committed ) )
2019-10-28 17:27:43 +00:00
{
if ( part - > isEmpty ( ) )
continue ;
2018-10-18 11:51:40 +00:00
2020-09-30 23:50:58 +00:00
if ( max_added_blocks )
{
auto blocks_iterator = max_added_blocks - > find ( part - > info . partition_id ) ;
if ( blocks_iterator = = max_added_blocks - > end ( ) | | part - > info . max_block > blocks_iterator - > second )
continue ;
}
2018-08-21 14:26:20 +00:00
2020-03-29 08:50:27 +00:00
func ( part ) ;
2017-04-01 07:20:54 +00:00
}
2020-03-29 08:50:27 +00:00
}
2020-11-25 13:47:32 +00:00
std : : optional < UInt64 > StorageReplicatedMergeTree : : totalRows ( const Settings & settings ) const
2020-03-29 08:50:27 +00:00
{
UInt64 res = 0 ;
2020-11-25 13:47:32 +00:00
foreachCommittedParts ( [ & res ] ( auto & part ) { res + = part - > rows_count ; } , settings . select_sequential_consistency ) ;
2020-03-29 08:50:27 +00:00
return res ;
}
2021-04-10 23:33:54 +00:00
std : : optional < UInt64 > StorageReplicatedMergeTree : : totalRowsByPartitionPredicate ( const SelectQueryInfo & query_info , ContextPtr local_context ) const
2020-09-21 10:13:01 +00:00
{
2021-03-03 08:36:20 +00:00
DataPartsVector parts ;
2021-04-10 23:33:54 +00:00
foreachCommittedParts ( [ & ] ( auto & part ) { parts . push_back ( part ) ; } , local_context - > getSettingsRef ( ) . select_sequential_consistency ) ;
return totalRowsByPartitionPredicateImpl ( query_info , local_context , parts ) ;
2020-09-21 10:13:01 +00:00
}
2020-11-25 13:47:32 +00:00
std : : optional < UInt64 > StorageReplicatedMergeTree : : totalBytes ( const Settings & settings ) const
2020-03-29 08:50:27 +00:00
{
UInt64 res = 0 ;
2020-11-25 13:47:32 +00:00
foreachCommittedParts ( [ & res ] ( auto & part ) { res + = part - > getBytesOnDisk ( ) ; } , settings . select_sequential_consistency ) ;
2019-10-28 17:27:43 +00:00
return res ;
2014-03-22 14:44:44 +00:00
}
2014-10-18 17:37:55 +00:00
2016-01-17 08:12:48 +00:00
void StorageReplicatedMergeTree : : assertNotReadonly ( ) const
2014-04-02 10:10:37 +00:00
{
2017-04-01 07:20:54 +00:00
if ( is_readonly )
2020-08-25 17:09:35 +00:00
throw Exception ( ErrorCodes : : TABLE_IS_READ_ONLY , " Table is in readonly mode (zookeeper path: {}) " , zookeeper_path) ;
2016-01-17 08:12:48 +00:00
}
2021-07-23 19:33:59 +00:00
SinkToStoragePtr StorageReplicatedMergeTree : : write ( const ASTPtr & /*query*/ , const StorageMetadataPtr & metadata_snapshot , ContextPtr local_context )
2016-01-17 08:12:48 +00:00
{
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2017-04-01 07:20:54 +00:00
assertNotReadonly ( ) ;
2014-05-13 11:24:04 +00:00
2021-04-10 23:33:54 +00:00
const Settings & query_settings = local_context - > getSettingsRef ( ) ;
2019-08-26 18:08:58 +00:00
bool deduplicate = storage_settings_ptr - > replicated_deduplication_window ! = 0 & & query_settings . insert_deduplicate ;
2017-10-24 19:32:23 +00:00
2020-12-01 09:10:12 +00:00
// TODO: should we also somehow pass list of columns to deduplicate on to the ReplicatedMergeTreeBlockOutputStream ?
2021-07-23 19:33:59 +00:00
return std : : make_shared < ReplicatedMergeTreeSink > (
2020-06-26 11:27:19 +00:00
* this , metadata_snapshot , query_settings . insert_quorum ,
2020-06-15 17:41:44 +00:00
query_settings . insert_quorum_timeout . totalMilliseconds ( ) ,
query_settings . max_partitions_per_insert_block ,
2020-09-30 23:16:27 +00:00
query_settings . insert_quorum_parallel ,
2020-11-13 07:54:05 +00:00
deduplicate ,
2021-02-10 14:12:49 +00:00
local_context ) ;
2014-04-02 10:10:37 +00:00
}
2014-03-22 14:44:44 +00:00
2014-10-18 17:37:55 +00:00
2020-06-12 18:24:32 +00:00
bool StorageReplicatedMergeTree : : optimize (
2020-06-22 09:03:53 +00:00
const ASTPtr & ,
2020-08-28 13:45:42 +00:00
const StorageMetadataPtr & ,
2020-06-17 13:39:26 +00:00
const ASTPtr & partition ,
bool final ,
bool deduplicate ,
2020-12-01 09:10:12 +00:00
const Names & deduplicate_by_columns ,
2021-04-10 23:33:54 +00:00
ContextPtr query_context )
2014-05-08 08:03:03 +00:00
{
Fix concurrent OPTIMIZE and DROP for ReplicatedMergeTree
Found with fuzzer [1] for 00992_system_parts_race_condition_zookeeper:
2021.03.13 11:12:30.385188 [ 42042 ] {2d3a8e17-26be-47c1-974f-bd2c9fc7c3af} <Debug> executeQuery: (from [::1]:58192, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') CREATE TABLE alter_tabl
e (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_3.alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_p
eriod = 1, cleanup_delay_period_random_add = 0;
...
2021.03.13 11:12:30.678387 [ 42042 ] {528cafc5-a02b-4df8-a531-a9a98e37b478} <Debug> executeQuery: (from [::1]:58192, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_3.alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0;
...
2021.03.13 11:12:40.671994 [ 4193 ] {d96ee93c-69b0-4e89-b411-16c382ae27a8} <Debug> executeQuery: (from [::1]:59714, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') OPTIMIZE TABLE alter_table FINAL
...
2021.03.13 11:12:40.990174 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> executeQuery: (from [::1]:59768, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') DROP TABLE alter_table;
...
2021.03.13 11:12:41.333054 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> test_3.alter_table (d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705): Removing part from filesystem 7_0_0_0
...
2021.03.13 11:12:41.335380 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> DatabaseCatalog: Waiting for table d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705 to be finally dropped
...
2021.03.13 11:12:41.781032 [ 4193 ] {d96ee93c-69b0-4e89-b411-16c382ae27a8} <Debug> test_3.alter_table (d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705): Waiting for queue-0000000085 to disappear from r2 queue
...
2021.03.13 11:12:41.900039 [ 371 ] {} <Trace> test_3.alter_table2 (ReplicatedMergeTreeQueue): Not executing log entry queue-0000000085 of type MERGE_PARTS for part 7_0_0_1 because part 7_0_0_0 is not ready yet (log entry for that part is being processed).
2021.03.13 11:12:41.900213 [ 365 ] {} <Trace> test_3.alter_table2 (ReplicatedMergeTreeQueue): Cannot execute alter metadata queue-0000000056 with version 22 because another alter 21 must be executed before
2021.03.13 11:12:41.900231 [ 13762 ] {} <Trace> test_3.alter_table2 (ae877c49-0d30-416d-9afe-27fd457d8fc4): Executing log entry to merge parts -7_0_0_0 to -7_0_0_1
2021.03.13 11:12:41.900330 [ 13762 ] {} <Debug> test_3.alter_table2 (ae877c49-0d30-416d-9afe-27fd457d8fc4): Don't have all parts for merge -7_0_0_1; will try to fetch it instead
...
[1]: https://clickhouse-test-reports.s3.yandex.net/21691/eb3710c164b991b8d4f86b1435a65f9eceb8f1f5/stress_test_(address).html#fail1
2021-03-13 15:05:54 +00:00
/// NOTE: exclusive lock cannot be used here, since this may lead to deadlock (see comments below),
/// but it should be safe to use non-exclusive to avoid dropping parts that may be required for processing queue.
2021-04-10 23:33:54 +00:00
auto table_lock = lockForShare ( query_context - > getCurrentQueryId ( ) , query_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
Fix concurrent OPTIMIZE and DROP for ReplicatedMergeTree
Found with fuzzer [1] for 00992_system_parts_race_condition_zookeeper:
2021.03.13 11:12:30.385188 [ 42042 ] {2d3a8e17-26be-47c1-974f-bd2c9fc7c3af} <Debug> executeQuery: (from [::1]:58192, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') CREATE TABLE alter_tabl
e (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_3.alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_p
eriod = 1, cleanup_delay_period_random_add = 0;
...
2021.03.13 11:12:30.678387 [ 42042 ] {528cafc5-a02b-4df8-a531-a9a98e37b478} <Debug> executeQuery: (from [::1]:58192, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_3.alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0;
...
2021.03.13 11:12:40.671994 [ 4193 ] {d96ee93c-69b0-4e89-b411-16c382ae27a8} <Debug> executeQuery: (from [::1]:59714, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') OPTIMIZE TABLE alter_table FINAL
...
2021.03.13 11:12:40.990174 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> executeQuery: (from [::1]:59768, using production parser) (comment: '/usr/share/clickhouse-test/queries/1_stateful/00153_aggregate_arena_race.sql') DROP TABLE alter_table;
...
2021.03.13 11:12:41.333054 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> test_3.alter_table (d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705): Removing part from filesystem 7_0_0_0
...
2021.03.13 11:12:41.335380 [ 2298 ] {a80f9306-3a73-4778-a921-db53249247e3} <Debug> DatabaseCatalog: Waiting for table d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705 to be finally dropped
...
2021.03.13 11:12:41.781032 [ 4193 ] {d96ee93c-69b0-4e89-b411-16c382ae27a8} <Debug> test_3.alter_table (d4fedaca-e0f6-4c22-9a4f-9f4d11b6b705): Waiting for queue-0000000085 to disappear from r2 queue
...
2021.03.13 11:12:41.900039 [ 371 ] {} <Trace> test_3.alter_table2 (ReplicatedMergeTreeQueue): Not executing log entry queue-0000000085 of type MERGE_PARTS for part 7_0_0_1 because part 7_0_0_0 is not ready yet (log entry for that part is being processed).
2021.03.13 11:12:41.900213 [ 365 ] {} <Trace> test_3.alter_table2 (ReplicatedMergeTreeQueue): Cannot execute alter metadata queue-0000000056 with version 22 because another alter 21 must be executed before
2021.03.13 11:12:41.900231 [ 13762 ] {} <Trace> test_3.alter_table2 (ae877c49-0d30-416d-9afe-27fd457d8fc4): Executing log entry to merge parts -7_0_0_0 to -7_0_0_1
2021.03.13 11:12:41.900330 [ 13762 ] {} <Debug> test_3.alter_table2 (ae877c49-0d30-416d-9afe-27fd457d8fc4): Don't have all parts for merge -7_0_0_1; will try to fetch it instead
...
[1]: https://clickhouse-test-reports.s3.yandex.net/21691/eb3710c164b991b8d4f86b1435a65f9eceb8f1f5/stress_test_(address).html#fail1
2021-03-13 15:05:54 +00:00
2017-04-01 07:20:54 +00:00
assertNotReadonly ( ) ;
2016-05-16 18:43:38 +00:00
2018-04-06 16:06:07 +00:00
if ( ! is_leader )
2020-06-19 14:18:58 +00:00
throw Exception ( " OPTIMIZE cannot be done on this replica because it is not a leader " , ErrorCodes : : NOT_A_LEADER ) ;
2016-05-16 18:43:38 +00:00
2021-10-06 17:07:35 +00:00
auto handle_noop = [ & ] ( const String & message )
2017-04-01 07:20:54 +00:00
{
2021-10-06 17:07:35 +00:00
if ( query_context - > getSettingsRef ( ) . optimize_throw_if_noop )
throw Exception ( message , ErrorCodes : : CANNOT_ASSIGN_OPTIMIZE ) ;
return false ;
} ;
2018-01-12 17:30:21 +00:00
2021-10-06 17:07:35 +00:00
auto zookeeper = getZooKeeper ( ) ;
UInt64 disk_space = getStoragePolicy ( ) - > getMaxUnreservedFreeSpace ( ) ;
const auto storage_settings_ptr = getSettings ( ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
std : : vector < ReplicatedMergeTreeLogEntryData > merge_entries ;
2019-08-01 11:10:42 +00:00
2021-10-06 17:07:35 +00:00
auto try_assign_merge = [ & ] ( const String & partition_id ) - > bool
{
constexpr size_t max_retries = 10 ;
size_t try_no = 0 ;
for ( ; try_no < max_retries ; + + try_no )
2018-01-12 17:30:21 +00:00
{
2021-10-06 17:07:35 +00:00
/// We must select parts for merge under merge_selecting_mutex because other threads
/// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
std : : lock_guard merge_selecting_lock ( merge_selecting_mutex ) ;
ReplicatedMergeTreeMergePredicate can_merge = queue . getMergePredicate ( zookeeper ) ;
2018-07-06 15:25:22 +00:00
2021-10-06 17:07:35 +00:00
auto future_merged_part = std : : make_shared < FutureMergedMutatedPart > ( ) ;
if ( storage_settings . get ( ) - > assign_part_uuids )
future_merged_part - > uuid = UUIDHelpers : : generateV4 ( ) ;
2018-07-06 15:25:22 +00:00
2021-10-06 17:07:35 +00:00
constexpr const char * unknown_disable_reason = " unknown reason " ;
String disable_reason = unknown_disable_reason ;
SelectPartsDecision select_decision = SelectPartsDecision : : CANNOT_SELECT ;
2018-07-16 03:27:43 +00:00
2021-10-06 17:07:35 +00:00
if ( partition_id . empty ( ) )
2018-07-06 15:25:22 +00:00
{
2021-10-06 17:07:35 +00:00
select_decision = merger_mutator . selectPartsToMerge (
future_merged_part , /* aggressive */ true , storage_settings_ptr - > max_bytes_to_merge_at_max_space_in_pool ,
can_merge , /* merge_with_ttl_allowed */ false , & disable_reason ) ;
2018-07-06 15:25:22 +00:00
}
2021-10-06 17:07:35 +00:00
else
2018-07-06 15:25:22 +00:00
{
2021-10-06 17:07:35 +00:00
select_decision = merger_mutator . selectAllPartsToMergeWithinPartition (
future_merged_part , disk_space , can_merge , partition_id , final , metadata_snapshot ,
& disable_reason , query_context - > getSettingsRef ( ) . optimize_skip_merged_partitions ) ;
}
2019-08-19 17:59:16 +00:00
2021-10-06 17:07:35 +00:00
/// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization)
if ( select_decision = = SelectPartsDecision : : NOTHING_TO_MERGE )
return false ;
2020-11-02 14:38:18 +00:00
2021-10-06 17:07:35 +00:00
if ( select_decision ! = SelectPartsDecision : : SELECTED )
{
constexpr const char * message_fmt = " Cannot select parts for optimization: {} " ;
assert ( disable_reason ! = unknown_disable_reason ) ;
if ( ! partition_id . empty ( ) )
disable_reason + = fmt : : format ( " (in partition {}) " , partition_id ) ;
String message = fmt : : format ( message_fmt , disable_reason ) ;
LOG_INFO ( log , message ) ;
return handle_noop ( message ) ;
}
2020-10-13 18:25:45 +00:00
2021-10-06 17:07:35 +00:00
ReplicatedMergeTreeLogEntryData merge_entry ;
CreateMergeEntryResult create_result = createLogEntryToMergeParts (
zookeeper , future_merged_part - > parts ,
future_merged_part - > name , future_merged_part - > uuid , future_merged_part - > type ,
deduplicate , deduplicate_by_columns ,
& merge_entry , can_merge . getVersion ( ) , future_merged_part - > merge_type ) ;
2020-06-12 18:24:32 +00:00
2021-10-06 17:07:35 +00:00
if ( create_result = = CreateMergeEntryResult : : MissingPart )
{
String message = " Can't create merge queue node in ZooKeeper, because some parts are missing " ;
LOG_TRACE ( log , message ) ;
return handle_noop ( message ) ;
}
2020-11-10 14:42:56 +00:00
2021-10-06 17:07:35 +00:00
if ( create_result = = CreateMergeEntryResult : : LogUpdated )
continue ;
2020-06-12 18:24:32 +00:00
2021-10-06 17:07:35 +00:00
merge_entries . push_back ( std : : move ( merge_entry ) ) ;
return true ;
}
2020-06-12 18:24:32 +00:00
2021-10-06 17:07:35 +00:00
assert ( try_no = = max_retries ) ;
String message = fmt : : format ( " Can't create merge queue node in ZooKeeper, because log was updated in every of {} tries " , try_no ) ;
LOG_TRACE ( log , message ) ;
return handle_noop ( message ) ;
} ;
2020-06-12 18:24:32 +00:00
2021-10-06 17:07:35 +00:00
bool assigned = false ;
if ( ! partition & & final )
{
DataPartsVector data_parts = getDataPartsVector ( ) ;
std : : unordered_set < String > partition_ids ;
2020-06-12 18:24:32 +00:00
2021-10-06 17:07:35 +00:00
for ( const DataPartPtr & part : data_parts )
partition_ids . emplace ( part - > info . partition_id ) ;
2016-05-16 18:43:38 +00:00
2021-10-06 17:07:35 +00:00
for ( const String & partition_id : partition_ids )
{
assigned = try_assign_merge ( partition_id ) ;
if ( ! assigned )
2020-06-12 18:24:32 +00:00
break ;
2018-07-06 15:25:22 +00:00
}
2017-04-01 07:20:54 +00:00
}
2021-10-06 17:07:35 +00:00
else
{
String partition_id ;
if ( partition )
partition_id = getPartitionIDFromQuery ( partition , query_context ) ;
assigned = try_assign_merge ( partition_id ) ;
}
2015-04-14 13:44:38 +00:00
2021-08-20 12:59:57 +00:00
table_lock . reset ( ) ;
for ( auto & merge_entry : merge_entries )
2021-08-23 12:57:50 +00:00
waitForLogEntryToBeProcessedIfNecessary ( merge_entry , query_context ) ;
2018-05-21 13:49:54 +00:00
2021-10-06 17:07:35 +00:00
return assigned ;
2014-05-08 08:03:03 +00:00
}
2020-01-30 12:54:52 +00:00
bool StorageReplicatedMergeTree : : executeMetadataAlter ( const StorageReplicatedMergeTree : : LogEntry & entry )
{
2021-04-28 17:49:27 +00:00
if ( entry . alter_version < metadata_version )
{
/// TODO Can we replace it with LOGICAL_ERROR?
2021-05-07 13:39:21 +00:00
/// As for now, it may rerely happen due to reordering of ALTER_METADATA entries in the queue of
/// non-initial replica and also may happen after stale replica recovery.
2021-04-28 17:49:27 +00:00
LOG_WARNING ( log , " Attempt to update metadata of version {} "
" to older version {} when processing log entry {}: {} " ,
metadata_version , entry . alter_version , entry . znode_name , entry . toString ( ) ) ;
return true ;
}
2020-01-30 12:54:52 +00:00
auto zookeeper = getZooKeeper ( ) ;
auto columns_from_entry = ColumnsDescription : : parse ( entry . columns_str ) ;
auto metadata_from_entry = ReplicatedMergeTreeTableMetadata : : parse ( entry . metadata_str ) ;
MergeTreeData : : DataParts parts ;
/// If metadata nodes have changed, we will update table structure locally.
2020-02-13 13:13:23 +00:00
Coordination : : Requests requests ;
2021-05-08 10:59:55 +00:00
requests . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( replica_path ) / " columns " , entry . columns_str , - 1 ) ) ;
requests . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( replica_path ) / " metadata " , entry . metadata_str , - 1 ) ) ;
2020-02-01 12:46:22 +00:00
2020-02-13 13:13:23 +00:00
zookeeper - > multi ( requests ) ;
2020-01-30 12:54:52 +00:00
2020-02-13 13:13:23 +00:00
{
2021-10-25 17:49:49 +00:00
auto table_lock_holder = lockForShare ( RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
auto alter_lock_holder = lockForAlter ( getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2020-02-13 13:13:23 +00:00
LOG_INFO ( log , " Metadata changed in ZooKeeper. Applying changes locally. " ) ;
2020-01-30 12:54:52 +00:00
2020-06-18 17:19:11 +00:00
auto metadata_diff = ReplicatedMergeTreeTableMetadata ( * this , getInMemoryMetadataPtr ( ) ) . checkAndFindDiff ( metadata_from_entry ) ;
2020-02-13 13:13:23 +00:00
setTableStructure ( std : : move ( columns_from_entry ) , metadata_diff ) ;
metadata_version = entry . alter_version ;
2020-01-30 12:54:52 +00:00
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Applied changes to the metadata of the table. Current metadata version: {} " , metadata_version ) ;
2020-01-30 12:54:52 +00:00
}
2020-01-31 12:25:31 +00:00
2020-02-17 16:33:05 +00:00
/// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node
2021-05-08 10:59:55 +00:00
zookeeper - > createOrUpdate ( fs : : path ( replica_path ) / " metadata_version " , std : : to_string ( metadata_version ) , zkutil : : CreateMode : : Persistent ) ;
2020-02-14 10:17:04 +00:00
2020-01-30 12:54:52 +00:00
return true ;
}
2014-10-18 17:37:55 +00:00
2020-11-10 10:23:46 +00:00
std : : set < String > StorageReplicatedMergeTree : : getPartitionIdsAffectedByCommands (
2021-04-10 23:33:54 +00:00
const MutationCommands & commands , ContextPtr query_context ) const
2020-11-10 10:23:46 +00:00
{
std : : set < String > affected_partition_ids ;
for ( const auto & command : commands )
{
if ( ! command . partition )
{
affected_partition_ids . clear ( ) ;
break ;
}
affected_partition_ids . insert (
getPartitionIDFromQuery ( command . partition , query_context )
) ;
}
return affected_partition_ids ;
}
PartitionBlockNumbersHolder StorageReplicatedMergeTree : : allocateBlockNumbersInAffectedPartitions (
2021-04-10 23:33:54 +00:00
const MutationCommands & commands , ContextPtr query_context , const zkutil : : ZooKeeperPtr & zookeeper ) const
2020-11-10 10:23:46 +00:00
{
const std : : set < String > mutation_affected_partition_ids = getPartitionIdsAffectedByCommands ( commands , query_context ) ;
if ( mutation_affected_partition_ids . size ( ) = = 1 )
{
const auto & affected_partition_id = * mutation_affected_partition_ids . cbegin ( ) ;
auto block_number_holder = allocateBlockNumber ( affected_partition_id , zookeeper ) ;
if ( ! block_number_holder . has_value ( ) )
return { } ;
auto block_number = block_number_holder - > getNumber ( ) ; /// Avoid possible UB due to std::move
return { { { affected_partition_id , block_number } } , std : : move ( block_number_holder ) } ;
}
else
{
/// TODO: Implement optimal block number aqcuisition algorithm in multiple (but not all) partitions
EphemeralLocksInAllPartitions lock_holder (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " block_numbers " , " block- " , fs : : path ( zookeeper_path ) / " temp " , * zookeeper ) ;
2020-11-10 10:23:46 +00:00
PartitionBlockNumbersHolder : : BlockNumbersType block_numbers ;
for ( const auto & lock : lock_holder . getLocks ( ) )
{
if ( mutation_affected_partition_ids . empty ( ) | | mutation_affected_partition_ids . count ( lock . partition_id ) )
block_numbers [ lock . partition_id ] = lock . number ;
}
return { std : : move ( block_numbers ) , std : : move ( lock_holder ) } ;
}
}
2019-03-05 10:12:20 +00:00
void StorageReplicatedMergeTree : : alter (
2021-10-25 17:49:49 +00:00
const AlterCommands & commands , ContextPtr query_context , AlterLockHolder & table_lock_holder )
2014-07-16 08:58:59 +00:00
{
2017-04-01 07:20:54 +00:00
assertNotReadonly ( ) ;
2016-01-17 08:12:48 +00:00
2019-12-10 20:47:05 +00:00
auto table_id = getStorageID ( ) ;
2019-08-26 14:50:34 +00:00
2020-08-27 13:10:10 +00:00
if ( commands . isSettingsAlter ( ) )
2019-08-06 13:04:29 +00:00
{
2019-08-26 18:08:58 +00:00
/// We don't replicate storage_settings_ptr ALTER. It's local operation.
2019-08-06 16:29:31 +00:00
/// Also we don't upgrade alter lock to table structure lock.
2020-06-08 18:23:26 +00:00
StorageInMemoryMetadata future_metadata = getInMemoryMetadata ( ) ;
2020-08-27 13:10:10 +00:00
commands . apply ( future_metadata , query_context ) ;
2020-02-14 13:17:50 +00:00
2020-09-18 10:57:33 +00:00
merge_strategy_picker . refreshState ( ) ;
2020-06-08 18:23:26 +00:00
changeSettings ( future_metadata . settings_changes , table_lock_holder ) ;
2019-08-27 09:34:53 +00:00
2020-06-08 18:23:26 +00:00
DatabaseCatalog : : instance ( ) . getDatabase ( table_id . database_name ) - > alterTable ( query_context , table_id , future_metadata ) ;
2019-08-06 13:04:29 +00:00
return ;
}
2020-06-12 18:24:32 +00:00
auto ast_to_str = [ ] ( ASTPtr query ) - > String
{
2019-12-27 14:36:59 +00:00
if ( ! query )
return " " ;
return queryToString ( query ) ;
} ;
2020-11-10 10:23:46 +00:00
const auto zookeeper = getZooKeeper ( ) ;
2020-01-31 12:25:31 +00:00
2020-02-13 16:16:09 +00:00
std : : optional < ReplicatedMergeTreeLogEntryData > alter_entry ;
2020-01-31 19:30:33 +00:00
std : : optional < String > mutation_znode ;
2014-07-16 08:58:59 +00:00
2020-02-17 12:47:34 +00:00
while ( true )
2017-04-01 07:20:54 +00:00
{
2020-02-13 16:16:09 +00:00
/// Clear nodes from previous iteration
alter_entry . emplace ( ) ;
2020-02-14 10:17:04 +00:00
mutation_znode . reset ( ) ;
2020-02-13 16:16:09 +00:00
2020-06-16 16:55:04 +00:00
auto current_metadata = getInMemoryMetadataPtr ( ) ;
2017-04-01 07:20:54 +00:00
2020-06-16 16:55:04 +00:00
StorageInMemoryMetadata future_metadata = * current_metadata ;
2020-08-27 13:10:10 +00:00
commands . apply ( future_metadata , query_context ) ;
2017-04-01 07:20:54 +00:00
2020-06-16 16:55:04 +00:00
ReplicatedMergeTreeTableMetadata future_metadata_in_zk ( * this , current_metadata ) ;
if ( ast_to_str ( future_metadata . sorting_key . definition_ast ) ! = ast_to_str ( current_metadata - > sorting_key . definition_ast ) )
2020-10-15 13:02:39 +00:00
{
/// We serialize definition_ast as list, because code which apply ALTER (setTableStructure) expect serialized non empty expression
/// list here and we cannot change this representation for compatibility. Also we have preparsed AST `sorting_key.expression_list_ast`
/// in KeyDescription, but it contain version column for VersionedCollapsingMergeTree, which shouldn't be defined as a part of key definition AST.
/// So the best compatible way is just to convert definition_ast to list and serialize it. In all other places key.expression_list_ast should be used.
future_metadata_in_zk . sorting_key = serializeAST ( * extractKeyExpressionList ( future_metadata . sorting_key . definition_ast ) ) ;
}
2019-04-15 09:30:45 +00:00
2020-08-27 13:10:10 +00:00
if ( ast_to_str ( future_metadata . sampling_key . definition_ast ) ! = ast_to_str ( current_metadata - > sampling_key . definition_ast ) )
2020-10-15 13:02:39 +00:00
future_metadata_in_zk . sampling_expression = serializeAST ( * extractKeyExpressionList ( future_metadata . sampling_key . definition_ast ) ) ;
2020-08-27 13:10:10 +00:00
2020-06-27 19:31:11 +00:00
if ( ast_to_str ( future_metadata . partition_key . definition_ast ) ! = ast_to_str ( current_metadata - > partition_key . definition_ast ) )
2020-10-15 13:02:39 +00:00
future_metadata_in_zk . partition_key = serializeAST ( * extractKeyExpressionList ( future_metadata . partition_key . definition_ast ) ) ;
2020-06-27 19:31:11 +00:00
2020-06-16 16:55:04 +00:00
if ( ast_to_str ( future_metadata . table_ttl . definition_ast ) ! = ast_to_str ( current_metadata - > table_ttl . definition_ast ) )
2020-09-20 13:27:33 +00:00
{
if ( future_metadata . table_ttl . definition_ast )
future_metadata_in_zk . ttl_table = serializeAST ( * future_metadata . table_ttl . definition_ast ) ;
else /// TTL was removed
future_metadata_in_zk . ttl_table = " " ;
}
2019-02-05 14:50:25 +00:00
2020-06-01 12:11:23 +00:00
String new_indices_str = future_metadata . secondary_indices . toString ( ) ;
2020-06-16 16:55:04 +00:00
if ( new_indices_str ! = current_metadata - > secondary_indices . toString ( ) )
2020-02-13 16:16:09 +00:00
future_metadata_in_zk . skip_indices = new_indices_str ;
2019-08-14 19:51:03 +00:00
2021-05-16 08:49:38 +00:00
String new_projections_str = future_metadata . projections . toString ( ) ;
if ( new_projections_str ! = current_metadata - > projections . toString ( ) )
future_metadata_in_zk . projections = new_projections_str ;
2020-02-13 16:16:09 +00:00
String new_constraints_str = future_metadata . constraints . toString ( ) ;
2020-06-16 16:55:04 +00:00
if ( new_constraints_str ! = current_metadata - > constraints . toString ( ) )
2020-02-13 16:16:09 +00:00
future_metadata_in_zk . constraints = new_constraints_str ;
2017-04-01 07:20:54 +00:00
2018-11-01 13:30:38 +00:00
Coordination : : Requests ops ;
2020-11-24 10:24:39 +00:00
size_t alter_path_idx = std : : numeric_limits < size_t > : : max ( ) ;
size_t mutation_path_idx = std : : numeric_limits < size_t > : : max ( ) ;
2017-04-01 07:20:54 +00:00
2020-02-13 16:16:09 +00:00
String new_metadata_str = future_metadata_in_zk . toString ( ) ;
2021-05-08 10:59:55 +00:00
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " metadata " , new_metadata_str , metadata_version ) ) ;
2019-03-05 10:12:20 +00:00
2020-02-13 16:16:09 +00:00
String new_columns_str = future_metadata . columns . toString ( ) ;
2021-05-08 10:59:55 +00:00
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " columns " , new_columns_str , - 1 ) ) ;
2017-04-01 07:20:54 +00:00
2020-06-16 16:55:04 +00:00
if ( ast_to_str ( current_metadata - > settings_changes ) ! = ast_to_str ( future_metadata . settings_changes ) )
2018-11-01 13:30:38 +00:00
{
2020-02-13 16:16:09 +00:00
/// Just change settings
2020-06-16 16:55:04 +00:00
StorageInMemoryMetadata metadata_copy = * current_metadata ;
metadata_copy . settings_changes = future_metadata . settings_changes ;
changeSettings ( metadata_copy . settings_changes , table_lock_holder ) ;
DatabaseCatalog : : instance ( ) . getDatabase ( table_id . database_name ) - > alterTable ( query_context , table_id , metadata_copy ) ;
2018-11-01 13:30:38 +00:00
}
2017-04-01 07:20:54 +00:00
2020-06-27 19:05:00 +00:00
/// We can be sure, that in case of successful commit in zookeeper our
2020-02-13 16:16:09 +00:00
/// version will increments by 1. Because we update with version check.
int new_metadata_version = metadata_version + 1 ;
2017-04-01 07:20:54 +00:00
2020-02-13 16:16:09 +00:00
alter_entry - > type = LogEntry : : ALTER_METADATA ;
alter_entry - > source_replica = replica_name ;
alter_entry - > metadata_str = new_metadata_str ;
alter_entry - > columns_str = new_columns_str ;
alter_entry - > alter_version = new_metadata_version ;
alter_entry - > create_time = time ( nullptr ) ;
2017-04-01 07:20:54 +00:00
2020-08-27 13:10:10 +00:00
auto maybe_mutation_commands = commands . getMutationCommands (
2021-04-10 23:33:54 +00:00
* current_metadata , query_context - > getSettingsRef ( ) . materialize_ttl_after_modify , query_context ) ;
2021-04-28 17:49:27 +00:00
bool have_mutation = ! maybe_mutation_commands . empty ( ) ;
alter_entry - > have_mutation = have_mutation ;
2017-04-01 07:20:54 +00:00
2020-11-24 10:24:39 +00:00
alter_path_idx = ops . size ( ) ;
2020-06-12 18:24:32 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( zookeeper_path ) / " log/log- " , alter_entry - > toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2018-11-01 13:30:38 +00:00
2020-11-10 10:23:46 +00:00
PartitionBlockNumbersHolder partition_block_numbers_holder ;
2021-04-28 17:49:27 +00:00
if ( have_mutation )
2020-01-31 12:25:31 +00:00
{
2021-05-08 10:59:55 +00:00
const String mutations_path ( fs : : path ( zookeeper_path ) / " mutations " ) ;
2018-11-01 13:30:38 +00:00
2020-02-05 11:18:11 +00:00
ReplicatedMergeTreeMutationEntry mutation_entry ;
2020-02-13 16:16:09 +00:00
mutation_entry . alter_version = new_metadata_version ;
2020-11-10 10:23:46 +00:00
mutation_entry . source_replica = replica_name ;
mutation_entry . commands = std : : move ( maybe_mutation_commands ) ;
2020-02-05 11:18:11 +00:00
Coordination : : Stat mutations_stat ;
zookeeper - > get ( mutations_path , & mutations_stat ) ;
2018-11-01 13:30:38 +00:00
2020-11-10 10:23:46 +00:00
partition_block_numbers_holder =
2021-05-28 13:27:36 +00:00
allocateBlockNumbersInAffectedPartitions ( mutation_entry . commands , query_context , zookeeper ) ;
2018-11-01 13:30:38 +00:00
2020-11-10 10:23:46 +00:00
mutation_entry . block_numbers = partition_block_numbers_holder . getBlockNumbers ( ) ;
2020-02-05 11:18:11 +00:00
mutation_entry . create_time = time ( nullptr ) ;
2014-07-16 08:58:59 +00:00
2020-02-05 11:18:11 +00:00
ops . emplace_back ( zkutil : : makeSetRequest ( mutations_path , String ( ) , mutations_stat . version ) ) ;
2020-11-24 10:24:39 +00:00
mutation_path_idx = ops . size ( ) ;
2020-02-05 11:18:11 +00:00
ops . emplace_back (
2021-05-08 10:59:55 +00:00
zkutil : : makeCreateRequest ( fs : : path ( mutations_path ) / " " , mutation_entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2020-01-31 12:25:31 +00:00
}
2014-07-16 08:58:59 +00:00
2021-04-10 23:33:54 +00:00
if ( auto txn = query_context - > getZooKeeperMetadataTransaction ( ) )
2020-11-20 16:06:27 +00:00
{
2021-07-30 16:34:18 +00:00
/// It would be better to clone ops instead of moving, so we could retry on ZBADVERSION,
/// but clone() is not implemented for Coordination::Request.
2021-02-08 19:36:17 +00:00
txn - > moveOpsTo ( ops ) ;
2020-11-24 10:24:39 +00:00
/// NOTE: IDatabase::alterTable(...) is called when executing ALTER_METADATA queue entry without query context,
/// so we have to update metadata of DatabaseReplicated here.
2021-05-08 10:59:55 +00:00
String metadata_zk_path = fs : : path ( txn - > getDatabaseZooKeeperPath ( ) ) / " metadata " / escapeForFileName ( table_id . table_name ) ;
2020-11-24 10:24:39 +00:00
auto ast = DatabaseCatalog : : instance ( ) . getDatabase ( table_id . database_name ) - > getCreateTableQuery ( table_id . table_name , query_context ) ;
2021-02-08 19:36:17 +00:00
applyMetadataChangesToCreateQuery ( ast , future_metadata ) ;
2020-11-24 10:24:39 +00:00
ops . emplace_back ( zkutil : : makeSetRequest ( metadata_zk_path , getObjectDefinitionFromCreateQuery ( ast ) , - 1 ) ) ;
2020-11-20 16:06:27 +00:00
}
2020-01-31 19:30:33 +00:00
Coordination : : Responses results ;
2020-06-12 15:09:12 +00:00
Coordination : : Error rc = zookeeper - > tryMulti ( ops , results ) ;
2018-11-01 13:30:38 +00:00
2020-11-10 10:23:46 +00:00
/// For the sake of constitency with mechanics of concurrent background process of assigning parts merge tasks
/// this placeholder must be held up until the moment of committing into ZK of the mutation entry
/// See ReplicatedMergeTreeMergePredicate::canMergeTwoParts() method
partition_block_numbers_holder . reset ( ) ;
2020-06-12 15:09:12 +00:00
if ( rc = = Coordination : : Error : : ZOK )
2020-01-31 19:30:33 +00:00
{
2021-04-28 17:49:27 +00:00
if ( have_mutation )
2017-04-01 07:20:54 +00:00
{
2020-02-17 16:33:05 +00:00
/// ALTER_METADATA record in replication /log
2020-11-24 10:24:39 +00:00
String alter_path = dynamic_cast < const Coordination : : CreateResponse & > ( * results [ alter_path_idx ] ) . path_created ;
2020-02-13 16:16:09 +00:00
alter_entry - > znode_name = alter_path . substr ( alter_path . find_last_of ( ' / ' ) + 1 ) ;
2018-11-01 13:30:38 +00:00
2020-02-17 16:33:05 +00:00
/// ReplicatedMergeTreeMutationEntry record in /mutations
2020-11-24 10:24:39 +00:00
String mutation_path = dynamic_cast < const Coordination : : CreateResponse & > ( * results [ mutation_path_idx ] ) . path_created ;
2020-01-31 19:30:33 +00:00
mutation_znode = mutation_path . substr ( mutation_path . find_last_of ( ' / ' ) + 1 ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2020-02-17 16:33:05 +00:00
/// ALTER_METADATA record in replication /log
2020-11-24 10:24:39 +00:00
String alter_path = dynamic_cast < const Coordination : : CreateResponse & > ( * results [ alter_path_idx ] ) . path_created ;
2020-02-13 16:16:09 +00:00
alter_entry - > znode_name = alter_path . substr ( alter_path . find_last_of ( ' / ' ) + 1 ) ;
2017-04-01 07:20:54 +00:00
}
2020-02-13 14:48:38 +00:00
break ;
2017-04-01 07:20:54 +00:00
}
2020-06-12 15:09:12 +00:00
else if ( rc = = Coordination : : Error : : ZBADVERSION )
2017-04-01 07:20:54 +00:00
{
2020-06-12 15:09:12 +00:00
if ( results [ 0 ] - > error ! = Coordination : : Error : : ZOK )
2021-09-14 11:45:33 +00:00
throw Exception ( " Metadata on replica is not up to date with common metadata in Zookeeper. It means that this replica still not applied some of previous alters. "
2021-09-14 10:57:05 +00:00
" Probably too many alters executing concurrently (highly not recommended). You can retry this error " ,
2020-06-27 19:31:11 +00:00
ErrorCodes : : CANNOT_ASSIGN_ALTER ) ;
2017-04-01 07:20:54 +00:00
2021-07-30 16:34:18 +00:00
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if ( query_context - > getZooKeeperMetadataTransaction ( ) )
throw Exception ( " Cannot execute alter, because mutations version was suddenly changed due to concurrent alter " ,
ErrorCodes : : CANNOT_ASSIGN_ALTER ) ;
2020-02-13 14:48:38 +00:00
continue ;
2020-01-31 19:30:33 +00:00
}
else
{
2020-02-13 14:48:38 +00:00
throw Coordination : : Exception ( " Alter cannot be assigned because of Zookeeper error " , rc ) ;
2020-01-31 19:30:33 +00:00
}
}
2017-04-01 07:20:54 +00:00
2021-10-25 17:49:49 +00:00
table_lock_holder . unlock ( ) ;
2017-04-01 07:20:54 +00:00
2021-08-23 12:57:50 +00:00
LOG_DEBUG ( log , " Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes. " ) ;
waitForLogEntryToBeProcessedIfNecessary ( * alter_entry , query_context , " Some replicas doesn't finish metadata alter: " ) ;
2017-04-01 07:20:54 +00:00
2020-02-14 10:17:04 +00:00
if ( mutation_znode )
2020-02-13 20:09:48 +00:00
{
LOG_DEBUG ( log , " Metadata changes applied. Will wait for data changes. " ) ;
2021-04-10 23:33:54 +00:00
waitMutation ( * mutation_znode , query_context - > getSettingsRef ( ) . replication_alter_partitions_sync ) ;
2020-02-13 20:09:48 +00:00
LOG_DEBUG ( log , " Data changes applied. " ) ;
2017-04-01 07:20:54 +00:00
}
2014-07-16 08:58:59 +00:00
}
2018-05-21 13:49:54 +00:00
/// If new version returns ordinary name, else returns part name containing the first and last month of the month
2020-10-05 13:52:03 +00:00
/// NOTE: use it in pair with getFakePartCoveringAllPartsInPartition(...)
2021-09-16 16:03:31 +00:00
String getPartNamePossiblyFake ( MergeTreeDataFormatVersion format_version , const MergeTreePartInfo & part_info )
2014-08-07 09:23:55 +00:00
{
2017-09-07 16:21:06 +00:00
if ( format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
2017-08-25 20:41:45 +00:00
{
/// The date range is all month long.
const auto & lut = DateLUT : : instance ( ) ;
2018-05-21 13:49:54 +00:00
time_t start_time = lut . YYYYMMDDToDate ( parse < UInt32 > ( part_info . partition_id + " 01 " ) ) ;
2021-02-14 11:00:40 +00:00
DayNum left_date = DayNum ( lut . toDayNum ( start_time ) . toUnderType ( ) ) ;
2018-05-25 13:29:15 +00:00
DayNum right_date = DayNum ( static_cast < size_t > ( left_date ) + lut . daysInMonth ( start_time ) - 1 ) ;
2017-08-25 20:41:45 +00:00
return part_info . getPartNameV0 ( left_date , right_date ) ;
}
2014-08-07 09:23:55 +00:00
2018-05-21 13:49:54 +00:00
return part_info . getPartName ( ) ;
}
2014-10-18 17:37:55 +00:00
2021-05-13 14:04:36 +00:00
bool StorageReplicatedMergeTree : : getFakePartCoveringAllPartsInPartition ( const String & partition_id , MergeTreePartInfo & part_info ,
std : : optional < EphemeralLockInZooKeeper > & delimiting_block_lock , bool for_replace_range )
2015-04-21 13:10:08 +00:00
{
2017-04-01 07:20:54 +00:00
/// Even if there is no data in the partition, you still need to mark the range for deletion.
/// - Because before executing DETACH, tasks for downloading parts to this partition can be executed.
2017-06-26 08:54:58 +00:00
Int64 left = 0 ;
2017-04-01 07:20:54 +00:00
2017-08-14 18:16:11 +00:00
/** Let's skip one number in `block_numbers` for the partition being deleted, and we will only delete parts until this number.
2019-05-03 02:00:57 +00:00
* This prohibits merges of deleted parts with the new inserted
2017-04-01 07:20:54 +00:00
* Invariant : merges of deleted parts with other parts do not appear in the log .
* NOTE : If you need to similarly support a ` DROP PART ` request , you will have to think of some new mechanism for it ,
* to guarantee this invariant .
*/
Int64 right ;
2018-05-14 14:51:33 +00:00
Int64 mutation_version ;
2017-04-01 07:20:54 +00:00
{
2017-06-25 00:01:10 +00:00
auto zookeeper = getZooKeeper ( ) ;
2021-05-13 14:04:36 +00:00
delimiting_block_lock = allocateBlockNumber ( partition_id , zookeeper ) ;
right = delimiting_block_lock - > getNumber ( ) ;
2021-05-24 11:23:23 +00:00
/// Make sure we cover all parts in drop range.
/// There might be parts with mutation version greater than current block number
2021-05-24 12:13:42 +00:00
/// if some part mutation has been assigned after block number allocation, but before creation of DROP_RANGE entry.
2021-05-20 06:55:28 +00:00
mutation_version = MergeTreePartInfo : : MAX_BLOCK_NUMBER ;
2017-04-01 07:20:54 +00:00
}
2021-05-13 11:29:59 +00:00
if ( for_replace_range )
2020-10-05 13:52:03 +00:00
{
2021-05-14 12:55:30 +00:00
/// NOTE Do not decrement max block number for REPLACE_RANGE, because there are invariants:
2021-05-13 11:29:59 +00:00
/// - drop range for REPLACE PARTITION must contain at least 2 blocks (1 skipped block and at least 1 real block)
/// - drop range for MOVE PARTITION/ATTACH PARTITION FROM always contains 1 block
2017-04-01 07:20:54 +00:00
2021-05-14 12:55:30 +00:00
/// NOTE UINT_MAX was previously used as max level for REPLACE/MOVE PARTITION (it was incorrect)
part_info = MergeTreePartInfo ( partition_id , left , right , MergeTreePartInfo : : MAX_LEVEL , mutation_version ) ;
2021-05-13 14:04:36 +00:00
return right ! = 0 ;
2020-10-05 13:52:03 +00:00
}
2017-11-15 16:32:47 +00:00
2021-05-13 14:04:36 +00:00
/// Empty partition.
if ( right = = 0 )
return false ;
- - right ;
2019-01-22 19:56:53 +00:00
/// Artificial high level is chosen, to make this part "covering" all parts inside.
2021-05-13 14:04:36 +00:00
part_info = MergeTreePartInfo ( partition_id , left , right , MergeTreePartInfo : : MAX_LEVEL , mutation_version ) ;
2018-05-21 13:49:54 +00:00
return true ;
2017-06-16 16:47:09 +00:00
}
2021-06-20 08:24:43 +00:00
void StorageReplicatedMergeTree : : restoreMetadataInZooKeeper ( )
{
LOG_INFO ( log , " Restoring replica metadata " ) ;
if ( ! is_readonly | | has_metadata_in_zookeeper )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " It's a bug: replica is not readonly " ) ;
if ( are_restoring_replica . exchange ( true ) )
throw Exception ( ErrorCodes : : CONCURRENT_ACCESS_NOT_SUPPORTED , " Replica restoration in progress " ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2021-09-06 14:24:03 +00:00
const DataPartsVector all_parts = getAllDataPartsVector ( ) ;
2021-06-20 08:24:43 +00:00
Strings active_parts_names ;
/// Why all parts (not only Committed) are moved to detached/:
/// After ZK metadata restoration ZK resets sequential counters (including block number counters), so one may
/// potentially encounter a situation that a part we want to attach already exists.
for ( const auto & part : all_parts )
{
if ( part - > getState ( ) = = DataPartState : : Committed )
active_parts_names . push_back ( part - > name ) ;
forgetPartAndMoveToDetached ( part ) ;
}
LOG_INFO ( log , " Moved all parts to detached/ " ) ;
const bool is_first_replica = createTableIfNotExists ( metadata_snapshot ) ;
LOG_INFO ( log , " Created initial ZK nodes, replica is first: {} " , is_first_replica ) ;
if ( ! is_first_replica )
createReplica ( metadata_snapshot ) ;
createNewZooKeeperNodes ( ) ;
LOG_INFO ( log , " Created ZK nodes for table " ) ;
is_readonly = false ;
has_metadata_in_zookeeper = true ;
if ( is_first_replica )
for ( const String & part_name : active_parts_names )
attachPartition ( std : : make_shared < ASTLiteral > ( part_name ) , metadata_snapshot , true , getContext ( ) ) ;
LOG_INFO ( log , " Attached all partitions, starting table " ) ;
startup ( ) ;
are_restoring_replica . store ( false ) ;
}
2021-05-17 14:26:36 +00:00
void StorageReplicatedMergeTree : : dropPartNoWaitNoThrow ( const String & part_name )
2021-04-20 02:31:08 +00:00
{
assertNotReadonly ( ) ;
if ( ! is_leader )
throw Exception ( " DROP PART cannot be done on this replica because it is not a leader " , ErrorCodes : : NOT_A_LEADER ) ;
zkutil : : ZooKeeperPtr zookeeper = getZooKeeper ( ) ;
LogEntry entry ;
2021-05-25 17:25:00 +00:00
dropPartImpl ( zookeeper , part_name , entry , /*detach=*/ false , /*throw_if_noop=*/ false ) ;
2021-04-20 02:31:08 +00:00
}
2017-06-16 16:47:09 +00:00
2021-05-17 14:26:36 +00:00
void StorageReplicatedMergeTree : : dropPart ( const String & part_name , bool detach , ContextPtr query_context )
2017-06-16 16:47:09 +00:00
{
assertNotReadonly ( ) ;
2020-06-19 14:18:58 +00:00
if ( ! is_leader )
2021-05-17 14:26:36 +00:00
throw Exception ( " DROP PART cannot be done on this replica because it is not a leader " , ErrorCodes : : NOT_A_LEADER ) ;
2017-06-16 16:47:09 +00:00
2017-11-15 16:32:47 +00:00
zkutil : : ZooKeeperPtr zookeeper = getZooKeeper ( ) ;
2018-04-21 00:35:20 +00:00
LogEntry entry ;
2020-09-04 15:48:51 +00:00
2021-05-25 17:25:00 +00:00
dropPartImpl ( zookeeper , part_name , entry , detach , /*throw_if_noop=*/ true ) ;
2020-09-04 15:48:51 +00:00
2021-08-23 12:57:50 +00:00
waitForLogEntryToBeProcessedIfNecessary ( entry , query_context ) ;
2021-05-17 14:26:36 +00:00
}
2017-11-15 16:32:47 +00:00
2021-05-17 14:26:36 +00:00
void StorageReplicatedMergeTree : : dropPartition ( const ASTPtr & partition , bool detach , ContextPtr query_context )
{
assertNotReadonly ( ) ;
if ( ! is_leader )
throw Exception ( " DROP PARTITION cannot be done on this replica because it is not a leader " , ErrorCodes : : NOT_A_LEADER ) ;
2017-11-15 16:32:47 +00:00
2021-05-17 14:26:36 +00:00
zkutil : : ZooKeeperPtr zookeeper = getZooKeeper ( ) ;
2018-04-21 00:35:20 +00:00
LogEntry entry ;
2020-09-04 15:48:51 +00:00
2021-05-17 14:26:36 +00:00
String partition_id = getPartitionIDFromQuery ( partition , query_context ) ;
bool did_drop = dropAllPartsInPartition ( * zookeeper , partition_id , entry , query_context , detach ) ;
2020-09-04 15:48:51 +00:00
if ( did_drop )
2017-07-24 09:59:55 +00:00
{
2021-08-23 12:57:50 +00:00
waitForLogEntryToBeProcessedIfNecessary ( entry , query_context ) ;
2020-09-04 15:48:51 +00:00
cleanLastPartNode ( partition_id ) ;
}
2018-04-21 00:35:20 +00:00
}
2017-07-24 09:59:55 +00:00
2017-11-15 16:32:47 +00:00
2020-06-18 10:29:13 +00:00
void StorageReplicatedMergeTree : : truncate (
2021-04-10 23:33:54 +00:00
const ASTPtr & , const StorageMetadataPtr & , ContextPtr query_context , TableExclusiveLockHolder & table_lock )
2018-04-21 00:35:20 +00:00
{
2019-08-27 20:43:08 +00:00
table_lock . release ( ) ; /// Truncate is done asynchronously.
2018-04-21 00:35:20 +00:00
assertNotReadonly ( ) ;
2020-06-19 14:18:58 +00:00
if ( ! is_leader )
throw Exception ( " TRUNCATE cannot be done on this replica because it is not a leader " , ErrorCodes : : NOT_A_LEADER ) ;
2017-04-01 07:20:54 +00:00
2018-04-21 00:35:20 +00:00
zkutil : : ZooKeeperPtr zookeeper = getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
Strings partitions = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " block_numbers " ) ;
2017-04-01 07:20:54 +00:00
2021-08-20 12:59:57 +00:00
std : : vector < std : : unique_ptr < LogEntry > > entries_to_wait ;
entries_to_wait . reserve ( partitions . size ( ) ) ;
2018-04-21 00:35:20 +00:00
for ( String & partition_id : partitions )
2017-04-01 07:20:54 +00:00
{
2021-08-20 12:59:57 +00:00
auto entry = std : : make_unique < LogEntry > ( ) ;
if ( dropAllPartsInPartition ( * zookeeper , partition_id , * entry , query_context , false ) )
entries_to_wait . push_back ( std : : move ( entry ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-04-21 00:35:20 +00:00
2021-08-20 12:59:57 +00:00
for ( const auto & entry : entries_to_wait )
2021-08-23 12:57:50 +00:00
waitForLogEntryToBeProcessedIfNecessary ( * entry , query_context ) ;
2016-01-28 01:00:27 +00:00
}
2016-01-26 02:00:25 +00:00
2016-01-28 16:06:57 +00:00
2020-07-28 15:10:36 +00:00
PartitionCommandsResultInfo StorageReplicatedMergeTree : : attachPartition (
const ASTPtr & partition ,
const StorageMetadataPtr & metadata_snapshot ,
bool attach_part ,
2021-04-10 23:33:54 +00:00
ContextPtr query_context )
2014-08-07 11:46:01 +00:00
{
2017-04-01 07:20:54 +00:00
assertNotReadonly ( ) ;
2020-07-28 15:10:36 +00:00
PartitionCommandsResultInfo results ;
2019-08-29 16:17:47 +00:00
PartsTemporaryRename renamed_parts ( * this , " detached/ " ) ;
2019-07-30 19:11:15 +00:00
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach ( partition , attach_part , query_context , renamed_parts ) ;
2019-07-30 17:24:40 +00:00
2021-01-12 18:46:03 +00:00
/// TODO Allow to use quorum here.
2021-07-23 19:33:59 +00:00
ReplicatedMergeTreeSink output ( * this , metadata_snapshot , 0 , 0 , 0 , false , false , query_context ,
2021-02-15 17:31:58 +00:00
/*is_attach*/ true ) ;
2021-01-12 18:46:03 +00:00
2019-07-30 19:11:15 +00:00
for ( size_t i = 0 ; i < loaded_parts . size ( ) ; + + i )
2017-04-01 07:20:54 +00:00
{
2021-01-12 18:46:03 +00:00
const String old_name = loaded_parts [ i ] - > name ;
2019-07-30 19:11:15 +00:00
output . writeExistingPart ( loaded_parts [ i ] ) ;
2021-01-12 18:46:03 +00:00
2019-07-30 19:11:15 +00:00
renamed_parts . old_and_new_names [ i ] . first . clear ( ) ;
2021-01-12 18:46:03 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Attached part {} as {} " , old_name , loaded_parts [ i ] - > name ) ;
2021-01-12 18:46:03 +00:00
2020-07-28 15:10:36 +00:00
results . push_back ( PartitionCommandResultInfo {
. partition_id = loaded_parts [ i ] - > info . partition_id ,
. part_name = loaded_parts [ i ] - > name ,
. old_part_name = old_name ,
} ) ;
2017-04-01 07:20:54 +00:00
}
2020-07-28 15:10:36 +00:00
return results ;
2014-08-07 11:46:01 +00:00
}
2017-06-25 00:51:51 +00:00
2018-08-03 09:39:01 +00:00
void StorageReplicatedMergeTree : : checkTableCanBeDropped ( ) const
2017-01-23 19:18:25 +00:00
{
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID ( ) ;
2021-04-10 23:33:54 +00:00
getContext ( ) - > checkTableCanBeDropped ( table_id . database_name , table_id . table_name , getTotalActiveSizeInBytes ( ) ) ;
2017-01-23 19:18:25 +00:00
}
2014-10-18 17:37:55 +00:00
2020-09-26 19:18:28 +00:00
void StorageReplicatedMergeTree : : checkTableCanBeRenamed ( ) const
{
if ( ! allow_renaming )
throw Exception ( " Cannot rename Replicated table, because zookeeper_path contains implicit 'database' or 'table' macro. "
" We cannot rename path in ZooKeeper, so path may become inconsistent with table name. If you really want to rename table, "
" you should edit metadata file first and restart server or reattach the table. " , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2020-04-07 14:05:51 +00:00
void StorageReplicatedMergeTree : : rename ( const String & new_path_to_table_data , const StorageID & new_table_id )
2014-07-28 14:33:30 +00:00
{
2020-09-26 19:18:28 +00:00
checkTableCanBeRenamed ( ) ;
2020-04-07 14:05:51 +00:00
MergeTreeData : : rename ( new_path_to_table_data , new_table_id ) ;
2014-07-28 14:33:30 +00:00
2018-04-17 17:59:42 +00:00
/// Update table name in zookeeper
2020-06-14 01:23:53 +00:00
if ( ! is_readonly )
{
/// We don't do it for readonly tables, because it will be updated on next table startup.
/// It is also Ok to skip ZK error for the same reason.
try
{
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
zookeeper - > set ( fs : : path ( replica_path ) / " host " , getReplicatedMergeTreeAddress ( ) . toString ( ) ) ;
2020-06-14 01:23:53 +00:00
}
catch ( Coordination : : Exception & e )
{
LOG_WARNING ( log , " Cannot update the value of 'host' node (replica address) in ZooKeeper: {} " , e . displayText ( ) ) ;
}
}
2018-04-17 17:59:42 +00:00
2017-04-01 07:20:54 +00:00
/// TODO: You can update names of loggers.
2014-07-28 14:33:30 +00:00
}
2014-10-18 17:37:55 +00:00
2020-11-10 10:23:46 +00:00
bool StorageReplicatedMergeTree : : existsNodeCached ( const std : : string & path ) const
2016-01-24 05:00:24 +00:00
{
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( existing_nodes_cache_mutex ) ;
2017-04-01 07:20:54 +00:00
if ( existing_nodes_cache . count ( path ) )
return true ;
}
2016-01-24 05:00:24 +00:00
2017-04-01 07:20:54 +00:00
bool res = getZooKeeper ( ) - > exists ( path ) ;
2016-01-24 05:00:24 +00:00
2017-04-01 07:20:54 +00:00
if ( res )
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( existing_nodes_cache_mutex ) ;
2017-04-01 07:20:54 +00:00
existing_nodes_cache . insert ( path ) ;
}
2016-01-24 05:00:24 +00:00
2017-04-01 07:20:54 +00:00
return res ;
2016-01-24 05:00:24 +00:00
}
2018-07-04 16:31:21 +00:00
std : : optional < EphemeralLockInZooKeeper >
2018-05-28 15:37:30 +00:00
StorageReplicatedMergeTree : : allocateBlockNumber (
2021-05-21 09:30:49 +00:00
const String & partition_id , const zkutil : : ZooKeeperPtr & zookeeper , const String & zookeeper_block_id_path , const String & zookeeper_path_prefix ) const
2014-08-07 09:23:55 +00:00
{
2021-05-21 09:30:49 +00:00
String zookeeper_table_path ;
if ( zookeeper_path_prefix . empty ( ) )
zookeeper_table_path = zookeeper_path ;
else
zookeeper_table_path = zookeeper_path_prefix ;
2019-01-22 19:56:53 +00:00
/// Lets check for duplicates in advance, to avoid superfluous block numbers allocation
2018-08-25 01:58:14 +00:00
Coordination : : Requests deduplication_check_ops ;
2018-05-21 13:49:54 +00:00
if ( ! zookeeper_block_id_path . empty ( ) )
2017-06-26 14:21:25 +00:00
{
2018-05-21 13:49:54 +00:00
deduplication_check_ops . emplace_back ( zkutil : : makeCreateRequest ( zookeeper_block_id_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
deduplication_check_ops . emplace_back ( zkutil : : makeRemoveRequest ( zookeeper_block_id_path , - 1 ) ) ;
2017-06-26 14:21:25 +00:00
}
2017-04-01 07:20:54 +00:00
2021-05-23 07:54:48 +00:00
String block_numbers_path = fs : : path ( zookeeper_table_path ) / " block_numbers " ;
2021-05-08 10:59:55 +00:00
String partition_path = fs : : path ( block_numbers_path ) / partition_id ;
2018-05-21 13:49:54 +00:00
2017-08-14 18:16:11 +00:00
if ( ! existsNodeCached ( partition_path ) )
2017-06-26 14:21:25 +00:00
{
2018-08-25 01:58:14 +00:00
Coordination : : Requests ops ;
2018-04-04 12:39:48 +00:00
ops . push_back ( zkutil : : makeCreateRequest ( partition_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
/// We increment data version of the block_numbers node so that it becomes possible
/// to check in a ZK transaction that the set of partitions didn't change
/// (unfortunately there is no CheckChildren op).
ops . push_back ( zkutil : : makeSetRequest ( block_numbers_path , " " , - 1 ) ) ;
2018-08-25 01:58:14 +00:00
Coordination : : Responses responses ;
2020-06-12 15:09:12 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , responses ) ;
if ( code ! = Coordination : : Error : : ZOK & & code ! = Coordination : : Error : : ZNODEEXISTS )
2018-04-04 12:39:48 +00:00
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2017-06-26 14:21:25 +00:00
}
2017-04-01 07:20:54 +00:00
2018-07-04 16:31:21 +00:00
EphemeralLockInZooKeeper lock ;
2018-05-21 13:49:54 +00:00
/// 2 RTT
try
{
2018-07-04 16:31:21 +00:00
lock = EphemeralLockInZooKeeper (
2021-05-23 07:54:48 +00:00
fs : : path ( partition_path ) / " block- " , fs : : path ( zookeeper_table_path ) / " temp " , * zookeeper , & deduplication_check_ops ) ;
2018-05-21 13:49:54 +00:00
}
catch ( const zkutil : : KeeperMultiException & e )
{
2020-06-12 15:09:12 +00:00
if ( e . code = = Coordination : : Error : : ZNODEEXISTS & & e . getPathForFirstFailedOp ( ) = = zookeeper_block_id_path )
2018-05-21 13:49:54 +00:00
return { } ;
throw Exception ( " Cannot allocate block number in ZooKeeper: " + e . displayText ( ) , ErrorCodes : : KEEPER_EXCEPTION ) ;
}
2018-08-25 01:58:14 +00:00
catch ( const Coordination : : Exception & e )
2018-05-21 13:49:54 +00:00
{
throw Exception ( " Cannot allocate block number in ZooKeeper: " + e . displayText ( ) , ErrorCodes : : KEEPER_EXCEPTION ) ;
}
return { std : : move ( lock ) } ;
2014-08-07 09:23:55 +00:00
}
2014-10-18 17:37:55 +00:00
2021-08-23 12:57:50 +00:00
Strings StorageReplicatedMergeTree : : tryWaitForAllReplicasToProcessLogEntry (
2021-08-20 12:59:57 +00:00
const String & table_zookeeper_path , const ReplicatedMergeTreeLogEntryData & entry , Int64 wait_for_inactive_timeout )
2014-08-07 09:23:55 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Waiting for all replicas to process {} " , entry . znode_name ) ;
2014-08-07 11:46:01 +00:00
2020-01-07 18:16:37 +00:00
auto zookeeper = getZooKeeper ( ) ;
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( table_zookeeper_path ) / " replicas " ) ;
2020-01-17 13:54:22 +00:00
Strings unwaited ;
2021-08-23 12:57:50 +00:00
bool wait_for_inactive = wait_for_inactive_timeout ! = 0 ;
2017-04-01 07:20:54 +00:00
for ( const String & replica : replicas )
2019-12-20 10:00:21 +00:00
{
2021-08-20 12:59:57 +00:00
if ( wait_for_inactive | | zookeeper - > exists ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " is_active " ) )
2019-12-20 10:00:21 +00:00
{
2021-08-23 12:57:50 +00:00
if ( ! tryWaitForReplicaToProcessLogEntry ( table_zookeeper_path , replica , entry , wait_for_inactive_timeout ) )
2020-05-12 14:11:09 +00:00
unwaited . push_back ( replica ) ;
2019-12-20 10:00:21 +00:00
}
2020-01-17 13:54:22 +00:00
else
{
unwaited . push_back ( replica ) ;
}
2019-12-20 10:00:21 +00:00
}
2014-10-18 17:37:55 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Finished waiting for all replicas to process {} " , entry . znode_name ) ;
2020-01-17 13:54:22 +00:00
return unwaited ;
2014-10-18 17:37:55 +00:00
}
2021-08-23 12:57:50 +00:00
void StorageReplicatedMergeTree : : waitForAllReplicasToProcessLogEntry (
const String & table_zookeeper_path , const ReplicatedMergeTreeLogEntryData & entry , Int64 wait_for_inactive_timeout , const String & error_context )
2020-12-04 13:01:12 +00:00
{
2021-08-23 12:57:50 +00:00
Strings unfinished_replicas = tryWaitForAllReplicasToProcessLogEntry ( table_zookeeper_path , entry , wait_for_inactive_timeout ) ;
2021-08-20 12:59:57 +00:00
if ( unfinished_replicas . empty ( ) )
return ;
2014-10-18 17:37:55 +00:00
2021-08-23 12:57:50 +00:00
throw Exception ( ErrorCodes : : UNFINISHED , " {}Timeout exceeded while waiting for replicas {} to process entry {}. "
" Probably some replicas are inactive " , error_context , fmt : : join ( unfinished_replicas , " , " ) , entry . znode_name ) ;
2020-12-04 13:01:12 +00:00
}
2021-08-23 12:57:50 +00:00
void StorageReplicatedMergeTree : : waitForLogEntryToBeProcessedIfNecessary ( const ReplicatedMergeTreeLogEntryData & entry , ContextPtr query_context , const String & error_context )
2020-12-04 13:01:12 +00:00
{
2021-08-23 12:57:50 +00:00
/// If necessary, wait until the operation is performed on itself or on all replicas.
Int64 wait_for_inactive_timeout = query_context - > getSettingsRef ( ) . replication_wait_for_inactive_replica_timeout ;
if ( query_context - > getSettingsRef ( ) . replication_alter_partitions_sync = = 1 )
{
bool finished = tryWaitForReplicaToProcessLogEntry ( zookeeper_path , replica_name , entry , wait_for_inactive_timeout ) ;
if ( ! finished )
{
throw Exception ( ErrorCodes : : UNFINISHED , " {}Log entry {} is not precessed on local replica, "
" most likely because the replica was shut down. " , error_context , entry . znode_name ) ;
}
}
else if ( query_context - > getSettingsRef ( ) . replication_alter_partitions_sync = = 2 )
{
waitForAllReplicasToProcessLogEntry ( zookeeper_path , entry , wait_for_inactive_timeout , error_context ) ;
}
2021-08-20 12:59:57 +00:00
}
2020-12-04 13:01:12 +00:00
2021-08-23 12:57:50 +00:00
bool StorageReplicatedMergeTree : : tryWaitForReplicaToProcessLogEntry (
2021-08-20 12:59:57 +00:00
const String & table_zookeeper_path , const String & replica , const ReplicatedMergeTreeLogEntryData & entry , Int64 wait_for_inactive_timeout )
2014-10-18 17:37:55 +00:00
{
2017-04-01 07:20:54 +00:00
String entry_str = entry . toString ( ) ;
String log_node_name ;
2021-06-07 10:01:57 +00:00
/** Wait for entries from `log` directory (a common log, from where replicas copy entries to their queue) to be processed.
2017-04-01 07:20:54 +00:00
*
* The problem is that the numbers ( ` sequential ` node ) of the queue elements in ` log ` and in ` queue ` do not match .
* ( And the numbers of the same log element for different replicas do not match in the ` queue ` . )
*/
/** First, you need to wait until replica takes `queue` element from the `log` to its queue,
* if it has not been done already ( see the ` pullLogsToQueue ` function ) .
*
* To do this , check its node ` log_pointer ` - the maximum number of the element taken from ` log ` + 1.
*/
2020-09-09 13:32:50 +00:00
bool waiting_itself = replica = = replica_name ;
2021-08-23 12:57:50 +00:00
/// Do not wait if timeout is zero
bool wait_for_inactive = wait_for_inactive_timeout ! = 0 ;
/// Wait for unlimited time if timeout is negative
bool check_timeout = wait_for_inactive_timeout > 0 ;
2021-08-20 12:59:57 +00:00
Stopwatch time_waiting ;
2020-09-09 13:32:50 +00:00
const auto & stop_waiting = [ & ] ( )
2020-05-12 14:11:09 +00:00
{
2021-08-18 09:49:22 +00:00
bool stop_waiting_itself = waiting_itself & & partial_shutdown_called ;
2021-08-23 12:57:50 +00:00
bool timeout_exceeded = check_timeout & & wait_for_inactive_timeout < time_waiting . elapsedSeconds ( ) ;
2021-08-20 12:59:57 +00:00
bool stop_waiting_inactive = ( ! wait_for_inactive | | timeout_exceeded )
& & ! getZooKeeper ( ) - > exists ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " is_active " ) ;
return is_dropped | | stop_waiting_itself | | stop_waiting_inactive ;
2020-05-12 14:11:09 +00:00
} ;
2021-01-18 12:15:07 +00:00
/// Don't recheck ZooKeeper too often
constexpr auto event_wait_timeout_ms = 3000 ;
2020-05-12 14:11:09 +00:00
2021-09-16 15:38:27 +00:00
LOG_DEBUG ( log , " Waiting for {} to process log entry " , replica ) ;
2021-08-20 12:59:57 +00:00
2021-09-16 15:47:57 +00:00
if ( startsWith ( entry . znode_name , " log- " ) )
2017-04-01 07:20:54 +00:00
{
2021-06-07 10:01:57 +00:00
/// Take the number from the node name `log-xxxxxxxxxx`.
2017-04-01 07:20:54 +00:00
UInt64 log_index = parse < UInt64 > ( entry . znode_name . substr ( entry . znode_name . size ( ) - 10 ) ) ;
log_node_name = entry . znode_name ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Waiting for {} to pull {} to queue " , replica , log_node_name ) ;
2017-04-01 07:20:54 +00:00
/// Let's wait until entry gets into the replica queue.
2021-08-20 12:59:57 +00:00
bool pulled_to_queue = false ;
2021-09-16 15:38:27 +00:00
do
2017-04-01 07:20:54 +00:00
{
zkutil : : EventPtr event = std : : make_shared < Poco : : Event > ( ) ;
2021-05-08 10:59:55 +00:00
String log_pointer = getZooKeeper ( ) - > get ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " log_pointer " , nullptr , event ) ;
2017-04-01 07:20:54 +00:00
if ( ! log_pointer . empty ( ) & & parse < UInt64 > ( log_pointer ) > log_index )
2021-08-20 12:59:57 +00:00
{
pulled_to_queue = true ;
2017-04-01 07:20:54 +00:00
break ;
2021-08-20 12:59:57 +00:00
}
2017-04-01 07:20:54 +00:00
2021-01-18 12:15:07 +00:00
/// Wait with timeout because we can be already shut down, but not dropped.
/// So log_pointer node will exist, but we will never update it because all background threads already stopped.
/// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method,
/// but the query will never finish because the drop already shut down the table.
2021-09-16 15:38:27 +00:00
if ( ! stop_waiting ( ) )
event - > tryWait ( event_wait_timeout_ms ) ;
} while ( ! stop_waiting ( ) ) ;
2021-08-20 12:59:57 +00:00
if ( ! pulled_to_queue )
return false ;
2021-09-16 15:47:57 +00:00
LOG_DEBUG ( log , " Looking for node corresponding to {} in {} queue " , log_node_name , replica ) ;
2017-04-01 07:20:54 +00:00
}
2021-09-16 15:47:57 +00:00
else if ( ! entry . log_entry_id . empty ( ) )
{
/// First pass, check the table log.
/// If found in the log, wait for replica to fetch it to the queue.
/// If not found in the log, it is already in the queue.
LOG_DEBUG ( log , " Looking for log entry with id `{}` in the log " , entry . log_entry_id ) ;
String log_pointer = getZooKeeper ( ) - > get ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " log_pointer " ) ;
Strings log_entries = getZooKeeper ( ) - > getChildren ( fs : : path ( table_zookeeper_path ) / " log " ) ;
UInt64 log_index = 0 ;
bool found = false ;
for ( const String & log_entry_name : log_entries )
{
log_index = parse < UInt64 > ( log_entry_name . substr ( log_entry_name . size ( ) - 10 ) ) ;
if ( ! log_pointer . empty ( ) & & log_index < parse < UInt64 > ( log_pointer ) )
continue ;
String log_entry_str ;
Coordination : : Stat log_entry_stat ;
bool exists = getZooKeeper ( ) - > tryGet ( fs : : path ( table_zookeeper_path ) / " log " / log_entry_name , log_entry_str , & log_entry_stat ) ;
ReplicatedMergeTreeLogEntryData log_entry = * ReplicatedMergeTreeLogEntry : : parse ( log_entry_str , log_entry_stat ) ;
if ( exists & & entry . log_entry_id = = log_entry . log_entry_id )
{
LOG_DEBUG ( log , " Found log entry with id `{}` in the log " , entry . log_entry_id ) ;
found = true ;
log_node_name = log_entry_name ;
break ;
}
}
if ( found )
{
LOG_DEBUG ( log , " Waiting for {} to pull {} to queue " , replica , log_node_name ) ;
/// Let's wait until entry gets into the replica queue.
bool pulled_to_queue = false ;
do
{
zkutil : : EventPtr event = std : : make_shared < Poco : : Event > ( ) ;
log_pointer = getZooKeeper ( ) - > get ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " log_pointer " , nullptr , event ) ;
if ( ! log_pointer . empty ( ) & & parse < UInt64 > ( log_pointer ) > log_index )
{
pulled_to_queue = true ;
break ;
}
2017-04-01 07:20:54 +00:00
2021-09-16 15:47:57 +00:00
/// Wait with timeout because we can be already shut down, but not dropped.
/// So log_pointer node will exist, but we will never update it because all background threads already stopped.
/// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method,
/// but the query will never finish because the drop already shut down the table.
if ( ! stop_waiting ( ) )
event - > tryWait ( event_wait_timeout_ms ) ;
} while ( ! stop_waiting ( ) ) ;
if ( ! pulled_to_queue )
return false ;
}
}
else
{
throw Exception ( " Logical error: unexpected name of log node: " + entry . znode_name , ErrorCodes : : LOGICAL_ERROR ) ;
}
2017-04-01 07:20:54 +00:00
/** Second - find the corresponding entry in the queue of the specified replica.
2021-06-07 10:01:57 +00:00
* Its number may not match the ` log ` node . Therefore , we search by comparing the content .
2017-04-01 07:20:54 +00:00
*/
2021-05-08 10:59:55 +00:00
Strings queue_entries = getZooKeeper ( ) - > getChildren ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " queue " ) ;
2017-04-01 07:20:54 +00:00
String queue_entry_to_wait_for ;
for ( const String & entry_name : queue_entries )
{
String queue_entry_str ;
2021-09-16 15:47:57 +00:00
Coordination : : Stat queue_entry_stat ;
bool exists = getZooKeeper ( ) - > tryGet ( fs : : path ( table_zookeeper_path ) / " replicas " / replica / " queue " / entry_name , queue_entry_str , & queue_entry_stat ) ;
2017-04-01 07:20:54 +00:00
if ( exists & & queue_entry_str = = entry_str )
{
queue_entry_to_wait_for = entry_name ;
break ;
}
2021-09-16 15:47:57 +00:00
else if ( ! entry . log_entry_id . empty ( ) )
{
/// Check if the id matches rather than just contents. This entry
/// might have been written by different ClickHouse versions and
/// it is hard to guarantee same text representation.
ReplicatedMergeTreeLogEntryData queue_entry = * ReplicatedMergeTreeLogEntry : : parse ( queue_entry_str , queue_entry_stat ) ;
if ( entry . log_entry_id = = queue_entry . log_entry_id )
{
queue_entry_to_wait_for = entry_name ;
break ;
}
}
2017-04-01 07:20:54 +00:00
}
/// While looking for the record, it has already been executed and deleted.
if ( queue_entry_to_wait_for . empty ( ) )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " No corresponding node found. Assuming it has been already processed. Found {} nodes " , queue_entries . size ( ) ) ;
2020-05-12 14:11:09 +00:00
return true ;
2017-04-01 07:20:54 +00:00
}
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Waiting for {} to disappear from {} queue " , queue_entry_to_wait_for , replica ) ;
2017-04-01 07:20:54 +00:00
2020-05-12 14:11:09 +00:00
/// Third - wait until the entry disappears from the replica queue or replica become inactive.
2021-05-08 10:59:55 +00:00
String path_to_wait_on = fs : : path ( table_zookeeper_path ) / " replicas " / replica / " queue " / queue_entry_to_wait_for ;
2020-05-12 14:11:09 +00:00
2020-09-09 13:32:50 +00:00
return getZooKeeper ( ) - > waitForDisappear ( path_to_wait_on , stop_waiting ) ;
2014-08-07 09:23:55 +00:00
}
2014-10-07 18:44:03 +00:00
void StorageReplicatedMergeTree : : getStatus ( Status & res , bool with_zk_fields )
2014-10-06 05:18:17 +00:00
{
2017-04-01 07:20:54 +00:00
auto zookeeper = tryGetZooKeeper ( ) ;
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2017-04-01 07:20:54 +00:00
2018-04-06 16:06:07 +00:00
res . is_leader = is_leader ;
2019-08-26 18:08:58 +00:00
res . can_become_leader = storage_settings_ptr - > replicated_can_become_leader ;
2017-04-01 07:20:54 +00:00
res . is_readonly = is_readonly ;
res . is_session_expired = ! zookeeper | | zookeeper - > expired ( ) ;
res . queue = queue . getStatus ( ) ;
2017-04-17 15:06:12 +00:00
res . absolute_delay = getAbsoluteDelay ( ) ; /// NOTE: may be slightly inconsistent with queue status.
2017-04-01 07:20:54 +00:00
res . parts_to_check = part_check_thread . size ( ) ;
res . zookeeper_path = zookeeper_path ;
res . replica_name = replica_name ;
res . replica_path = replica_path ;
2020-01-30 12:54:52 +00:00
res . columns_version = - 1 ;
2017-04-01 07:20:54 +00:00
2020-02-15 00:11:09 +00:00
res . log_max_index = 0 ;
res . log_pointer = 0 ;
res . total_replicas = 0 ;
res . active_replicas = 0 ;
2021-08-09 12:58:23 +00:00
res . last_queue_update_exception = getLastQueueUpdateException ( ) ;
2021-07-27 14:35:20 +00:00
2020-02-15 00:11:09 +00:00
if ( with_zk_fields & & ! res . is_session_expired )
{
try
2017-04-01 07:20:54 +00:00
{
2021-05-08 10:59:55 +00:00
auto log_entries = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " log " ) ;
2017-04-01 07:20:54 +00:00
2021-05-04 11:29:50 +00:00
if ( ! log_entries . empty ( ) )
2020-02-15 00:11:09 +00:00
{
const String & last_log_entry = * std : : max_element ( log_entries . begin ( ) , log_entries . end ( ) ) ;
res . log_max_index = parse < UInt64 > ( last_log_entry . substr ( strlen ( " log- " ) ) ) ;
}
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
String log_pointer_str = zookeeper - > get ( fs : : path ( replica_path ) / " log_pointer " ) ;
2020-02-15 00:11:09 +00:00
res . log_pointer = log_pointer_str . empty ( ) ? 0 : parse < UInt64 > ( log_pointer_str ) ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
auto all_replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2020-02-15 00:11:09 +00:00
res . total_replicas = all_replicas . size ( ) ;
for ( const String & replica : all_replicas )
2021-08-04 13:19:42 +00:00
{
bool is_replica_active = zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) ;
res . active_replicas + = static_cast < UInt8 > ( is_replica_active ) ;
res . replica_is_active . emplace ( replica , is_replica_active ) ;
}
2020-02-15 00:11:09 +00:00
}
2020-02-15 00:13:00 +00:00
catch ( const Coordination : : Exception & )
2020-02-15 00:11:09 +00:00
{
res . zookeeper_exception = getCurrentExceptionMessage ( false ) ;
}
2017-04-01 07:20:54 +00:00
}
2014-10-06 05:18:17 +00:00
}
2014-10-09 20:28:33 +00:00
2015-09-24 00:21:02 +00:00
void StorageReplicatedMergeTree : : getQueue ( LogEntriesData & res , String & replica_name_ )
{
2017-04-01 07:20:54 +00:00
replica_name_ = replica_name ;
queue . getEntries ( res ) ;
2015-09-24 00:21:02 +00:00
}
2020-11-24 14:24:48 +00:00
std : : vector < PartMovesBetweenShardsOrchestrator : : Entry > StorageReplicatedMergeTree : : getPartMovesBetweenShardsEntries ( )
{
return part_moves_between_shards_orchestrator . getEntries ( ) ;
}
2017-04-17 15:06:12 +00:00
time_t StorageReplicatedMergeTree : : getAbsoluteDelay ( ) const
{
time_t min_unprocessed_insert_time = 0 ;
time_t max_processed_insert_time = 0 ;
queue . getInsertTimes ( min_unprocessed_insert_time , max_processed_insert_time ) ;
2017-05-19 18:48:25 +00:00
/// Load start time, then finish time to avoid reporting false delay when start time is updated
/// between loading of two variables.
time_t queue_update_start_time = last_queue_update_start_time . load ( ) ;
time_t queue_update_finish_time = last_queue_update_finish_time . load ( ) ;
2017-04-17 15:06:12 +00:00
time_t current_time = time ( nullptr ) ;
2017-05-19 18:48:25 +00:00
if ( ! queue_update_finish_time )
2017-04-17 15:06:12 +00:00
{
2017-05-19 18:48:25 +00:00
/// We have not updated queue even once yet (perhaps replica is readonly).
2017-04-17 15:06:12 +00:00
/// As we have no info about the current state of replication log, return effectively infinite delay.
return current_time ;
}
else if ( min_unprocessed_insert_time )
{
/// There are some unprocessed insert entries in queue.
return ( current_time > min_unprocessed_insert_time ) ? ( current_time - min_unprocessed_insert_time ) : 0 ;
}
2017-05-19 18:48:25 +00:00
else if ( queue_update_start_time > queue_update_finish_time )
2017-04-17 15:06:12 +00:00
{
/// Queue is empty, but there are some in-flight or failed queue update attempts
/// (likely because of problems with connecting to ZooKeeper).
/// Return the time passed since last attempt.
2017-05-19 18:48:25 +00:00
return ( current_time > queue_update_start_time ) ? ( current_time - queue_update_start_time ) : 0 ;
2017-04-17 15:06:12 +00:00
}
else
{
/// Everything is up-to-date.
return 0 ;
}
}
2015-09-24 00:21:02 +00:00
2016-01-17 13:00:42 +00:00
void StorageReplicatedMergeTree : : getReplicaDelays ( time_t & out_absolute_delay , time_t & out_relative_delay )
2015-11-05 20:08:18 +00:00
{
2017-04-01 07:20:54 +00:00
assertNotReadonly ( ) ;
2016-01-17 13:00:42 +00:00
2017-04-17 15:06:12 +00:00
time_t current_time = time ( nullptr ) ;
2016-01-17 13:00:42 +00:00
2017-04-17 15:06:12 +00:00
out_absolute_delay = getAbsoluteDelay ( ) ;
2017-04-01 07:20:54 +00:00
out_relative_delay = 0 ;
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
/** Relative delay is the maximum difference of absolute delay from any other replica,
* ( if this replica lags behind any other live replica , or zero , otherwise ) .
* Calculated only if the absolute delay is large enough .
*/
2016-01-17 13:00:42 +00:00
2020-06-19 14:18:58 +00:00
if ( out_absolute_delay < static_cast < time_t > ( storage_settings_ptr - > min_relative_delay_to_measure ) )
2017-04-01 07:20:54 +00:00
return ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
time_t max_replicas_unprocessed_insert_time = 0 ;
bool have_replica_with_nothing_unprocessed = false ;
2016-01-19 18:49:37 +00:00
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & replica : replicas )
{
if ( replica = = replica_name )
continue ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
/// Skip dead replicas.
2021-05-08 10:59:55 +00:00
if ( ! zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) )
2017-04-01 07:20:54 +00:00
continue ;
2016-04-02 21:22:39 +00:00
2017-04-01 07:20:54 +00:00
String value ;
2021-05-08 10:59:55 +00:00
if ( ! zookeeper - > tryGet ( fs : : path ( zookeeper_path ) / " replicas " / replica / " min_unprocessed_insert_time " , value ) )
2017-04-01 07:20:54 +00:00
continue ;
2016-01-17 13:00:42 +00:00
2017-04-01 07:20:54 +00:00
time_t replica_time = value . empty ( ) ? 0 : parse < time_t > ( value ) ;
2016-01-19 18:49:37 +00:00
2017-04-01 07:20:54 +00:00
if ( replica_time = = 0 )
{
/** Note
* The conclusion that the replica does not lag may be incorrect ,
* because the information about ` min_unprocessed_insert_time ` is taken
* only from that part of the log that has been moved to the queue .
2018-05-31 13:05:05 +00:00
* If the replica for some reason has stalled ` queueUpdatingTask ` ,
2017-04-01 07:20:54 +00:00
* then ` min_unprocessed_insert_time ` will be incorrect .
*/
2016-04-02 21:22:39 +00:00
2017-04-01 07:20:54 +00:00
have_replica_with_nothing_unprocessed = true ;
break ;
}
2016-01-19 18:49:37 +00:00
2017-04-01 07:20:54 +00:00
if ( replica_time > max_replicas_unprocessed_insert_time )
max_replicas_unprocessed_insert_time = replica_time ;
}
2015-11-05 20:08:18 +00:00
2017-04-01 07:20:54 +00:00
if ( have_replica_with_nothing_unprocessed )
out_relative_delay = out_absolute_delay ;
2017-04-17 15:06:12 +00:00
else
{
max_replicas_unprocessed_insert_time = std : : min ( current_time , max_replicas_unprocessed_insert_time ) ;
time_t min_replicas_delay = current_time - max_replicas_unprocessed_insert_time ;
if ( out_absolute_delay > min_replicas_delay )
out_relative_delay = out_absolute_delay - min_replicas_delay ;
}
2015-11-05 20:08:18 +00:00
}
2020-07-15 16:37:52 +00:00
void StorageReplicatedMergeTree : : fetchPartition (
const ASTPtr & partition ,
const StorageMetadataPtr & metadata_snapshot ,
const String & from_ ,
2021-04-13 04:40:33 +00:00
bool fetch_part ,
2021-04-10 23:33:54 +00:00
ContextPtr query_context )
2014-10-09 20:28:33 +00:00
{
2021-01-08 05:10:00 +00:00
Macros : : MacroExpansionInfo info ;
2021-05-04 11:29:50 +00:00
info . expand_special_macros_only = false ; //-V1048
2021-01-07 14:13:17 +00:00
info . table_id = getStorageID ( ) ;
info . table_id . uuid = UUIDHelpers : : Nil ;
2021-04-10 23:33:54 +00:00
auto expand_from = query_context - > getMacros ( ) - > expand ( from_ , info ) ;
2021-01-07 14:13:17 +00:00
String auxiliary_zookeeper_name = extractZooKeeperName ( expand_from ) ;
2021-10-28 16:19:41 +00:00
String from = extractZooKeeperPath ( expand_from , /* check_starts_with_slash */ true ) ;
2020-08-27 14:19:18 +00:00
if ( from . empty ( ) )
throw Exception ( " ZooKeeper path should not be empty " , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2020-08-28 11:12:51 +00:00
zkutil : : ZooKeeperPtr zookeeper ;
2020-11-16 12:30:54 +00:00
if ( auxiliary_zookeeper_name ! = default_zookeeper_name )
2021-04-10 23:33:54 +00:00
zookeeper = getContext ( ) - > getAuxiliaryZooKeeper ( auxiliary_zookeeper_name ) ;
2020-08-28 11:12:51 +00:00
else
zookeeper = getZooKeeper ( ) ;
2017-04-01 07:20:54 +00:00
if ( from . back ( ) = = ' / ' )
from . resize ( from . size ( ) - 1 ) ;
2021-04-13 04:40:33 +00:00
if ( fetch_part )
2017-08-14 18:16:11 +00:00
{
2021-04-13 04:40:33 +00:00
String part_name = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
2021-04-13 09:34:04 +00:00
auto part_path = findReplicaHavingPart ( part_name , from , zookeeper ) ;
2021-04-13 04:40:33 +00:00
2021-04-13 09:34:04 +00:00
if ( part_path . empty ( ) )
2021-04-14 10:55:42 +00:00
throw Exception ( ErrorCodes : : NO_REPLICA_HAS_PART , " Part {} does not exist on any replica " , part_name ) ;
2021-04-13 04:40:33 +00:00
/** Let's check that there is no such part in the `detached` directory (where we will write the downloaded parts).
* Unreliable ( there is a race condition ) - such a part may appear a little later .
*/
2021-04-14 02:05:41 +00:00
if ( checkIfDetachedPartExists ( part_name ) )
2021-04-14 09:54:56 +00:00
throw Exception ( ErrorCodes : : DUPLICATE_DATA_PART , " Detached part " + part_name + " already exists. " ) ;
2021-04-13 04:40:33 +00:00
LOG_INFO ( log , " Will fetch part {} from shard {} (zookeeper '{}') " , part_name , from_ , auxiliary_zookeeper_name ) ;
try
2019-05-12 14:57:23 +00:00
{
2021-04-13 09:34:04 +00:00
/// part name , metadata, part_path , true, 0, zookeeper
if ( ! fetchPart ( part_name , metadata_snapshot , part_path , true , 0 , zookeeper ) )
2021-04-14 09:54:56 +00:00
throw Exception ( ErrorCodes : : UNFINISHED , " Failed to fetch part {} from {} " , part_name , from_ ) ;
2019-05-12 14:57:23 +00:00
}
2021-04-13 04:40:33 +00:00
catch ( const DB : : Exception & e )
{
if ( e . code ( ) ! = ErrorCodes : : RECEIVED_ERROR_FROM_REMOTE_IO_SERVER & & e . code ( ) ! = ErrorCodes : : RECEIVED_ERROR_TOO_MANY_REQUESTS
& & e . code ( ) ! = ErrorCodes : : CANNOT_READ_ALL_DATA )
throw ;
2019-05-12 14:57:23 +00:00
2021-04-13 04:40:33 +00:00
LOG_INFO ( log , e . displayText ( ) ) ;
}
return ;
2017-08-14 18:16:11 +00:00
}
2017-04-01 07:20:54 +00:00
2021-04-13 04:40:33 +00:00
String partition_id = getPartitionIDFromQuery ( partition , query_context ) ;
LOG_INFO ( log , " Will fetch partition {} from shard {} (zookeeper '{}') " , partition_id , from_ , auxiliary_zookeeper_name ) ;
/** Let's check that there is no such partition in the `detached` directory (where we will write the downloaded parts).
* Unreliable ( there is a race condition ) - such a partition may appear a little later .
*/
2021-04-14 02:05:41 +00:00
if ( checkIfDetachedPartitionExists ( partition_id ) )
2021-04-13 04:40:33 +00:00
throw Exception ( " Detached partition " + partition_id + " already exists. " , ErrorCodes : : PARTITION_ALREADY_EXISTS ) ;
2017-04-01 07:20:54 +00:00
zkutil : : Strings replicas ;
zkutil : : Strings active_replicas ;
String best_replica ;
{
/// List of replicas of source shard.
2021-05-08 10:59:55 +00:00
replicas = zookeeper - > getChildren ( fs : : path ( from ) / " replicas " ) ;
2017-04-01 07:20:54 +00:00
/// Leave only active replicas.
active_replicas . reserve ( replicas . size ( ) ) ;
for ( const String & replica : replicas )
2021-05-08 10:59:55 +00:00
if ( zookeeper - > exists ( fs : : path ( from ) / " replicas " / replica / " is_active " ) )
2017-04-01 07:20:54 +00:00
active_replicas . push_back ( replica ) ;
if ( active_replicas . empty ( ) )
throw Exception ( " No active replicas for shard " + from , ErrorCodes : : NO_ACTIVE_REPLICAS ) ;
/** You must select the best (most relevant) replica.
* This is a replica with the maximum ` log_pointer ` , then with the minimum ` queue ` size .
* NOTE This is not exactly the best criteria . It does not make sense to download old partitions ,
* and it would be nice to be able to choose the replica closest by network .
* NOTE Of course , there are data races here . You can solve it by retrying .
*/
Int64 max_log_pointer = - 1 ;
UInt64 min_queue_size = std : : numeric_limits < UInt64 > : : max ( ) ;
for ( const String & replica : active_replicas )
{
2021-05-08 10:59:55 +00:00
String current_replica_path = fs : : path ( from ) / " replicas " / replica ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
String log_pointer_str = zookeeper - > get ( fs : : path ( current_replica_path ) / " log_pointer " ) ;
2017-04-01 07:20:54 +00:00
Int64 log_pointer = log_pointer_str . empty ( ) ? 0 : parse < UInt64 > ( log_pointer_str ) ;
2018-08-25 01:58:14 +00:00
Coordination : : Stat stat ;
2021-05-08 10:59:55 +00:00
zookeeper - > get ( fs : : path ( current_replica_path ) / " queue " , & stat ) ;
2017-04-01 07:20:54 +00:00
size_t queue_size = stat . numChildren ;
if ( log_pointer > max_log_pointer
| | ( log_pointer = = max_log_pointer & & queue_size < min_queue_size ) )
{
max_log_pointer = log_pointer ;
min_queue_size = queue_size ;
best_replica = replica ;
}
}
}
if ( best_replica . empty ( ) )
throw Exception ( " Logical error: cannot choose best replica. " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Found {} replicas, {} of them are active. Selected {} to fetch from. " , replicas . size ( ) , active_replicas . size ( ) , best_replica ) ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
String best_replica_path = fs : : path ( from ) / " replicas " / best_replica ;
2017-04-01 07:20:54 +00:00
/// Let's find out which parts are on the best replica.
/** Trying to download these parts.
* Some of them could be deleted due to the merge .
* In this case , update the information about the available parts and try again .
*/
unsigned try_no = 0 ;
Strings missing_parts ;
do
{
if ( try_no )
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Some of parts ({}) are missing. Will try to fetch covering parts. " , missing_parts . size ( ) ) ;
2017-04-01 07:20:54 +00:00
2021-04-10 23:33:54 +00:00
if ( try_no > = query_context - > getSettings ( ) . max_fetch_partition_retries_count )
2018-03-09 23:23:15 +00:00
throw Exception ( " Too many retries to fetch parts from " + best_replica_path , ErrorCodes : : TOO_MANY_RETRIES_TO_FETCH_PARTS ) ;
2017-04-01 07:20:54 +00:00
2021-05-08 10:59:55 +00:00
Strings parts = zookeeper - > getChildren ( fs : : path ( best_replica_path ) / " parts " ) ;
2019-05-03 02:00:57 +00:00
ActiveDataPartSet active_parts_set ( format_version , parts ) ;
2017-04-01 07:20:54 +00:00
Strings parts_to_fetch ;
if ( missing_parts . empty ( ) )
{
parts_to_fetch = active_parts_set . getParts ( ) ;
/// Leaving only the parts of the desired partition.
Strings parts_to_fetch_partition ;
for ( const String & part : parts_to_fetch )
2017-08-14 18:16:11 +00:00
{
2019-05-03 02:00:57 +00:00
if ( MergeTreePartInfo : : fromPartName ( part , format_version ) . partition_id = = partition_id )
2017-04-01 07:20:54 +00:00
parts_to_fetch_partition . push_back ( part ) ;
2017-08-14 18:16:11 +00:00
}
2017-04-01 07:20:54 +00:00
parts_to_fetch = std : : move ( parts_to_fetch_partition ) ;
if ( parts_to_fetch . empty ( ) )
2017-08-14 18:16:11 +00:00
throw Exception ( " Partition " + partition_id + " on " + best_replica_path + " doesn't exist " , ErrorCodes : : PARTITION_DOESNT_EXIST ) ;
2017-04-01 07:20:54 +00:00
}
else
{
for ( const String & missing_part : missing_parts )
{
String containing_part = active_parts_set . getContainingPart ( missing_part ) ;
if ( ! containing_part . empty ( ) )
parts_to_fetch . push_back ( containing_part ) ;
else
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Part {} on replica {} has been vanished. " , missing_part , best_replica_path ) ;
2017-04-01 07:20:54 +00:00
}
}
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Parts to fetch: {} " , parts_to_fetch . size ( ) ) ;
2017-04-01 07:20:54 +00:00
missing_parts . clear ( ) ;
for ( const String & part : parts_to_fetch )
{
2020-08-28 00:53:22 +00:00
bool fetched = false ;
2017-04-01 07:20:54 +00:00
try
{
2020-09-17 10:21:00 +00:00
fetched = fetchPart ( part , metadata_snapshot , best_replica_path , true , 0 , zookeeper ) ;
2017-04-01 07:20:54 +00:00
}
catch ( const DB : : Exception & e )
{
2018-08-08 18:01:25 +00:00
if ( e . code ( ) ! = ErrorCodes : : RECEIVED_ERROR_FROM_REMOTE_IO_SERVER & & e . code ( ) ! = ErrorCodes : : RECEIVED_ERROR_TOO_MANY_REQUESTS
& & e . code ( ) ! = ErrorCodes : : CANNOT_READ_ALL_DATA )
2017-04-01 07:20:54 +00:00
throw ;
LOG_INFO ( log , e . displayText ( ) ) ;
}
2020-08-28 00:53:22 +00:00
if ( ! fetched )
missing_parts . push_back ( part ) ;
2017-04-01 07:20:54 +00:00
}
+ + try_no ;
} while ( ! missing_parts . empty ( ) ) ;
2014-10-09 20:28:33 +00:00
}
2021-04-10 23:33:54 +00:00
void StorageReplicatedMergeTree : : mutate ( const MutationCommands & commands , ContextPtr query_context )
2018-04-19 10:33:16 +00:00
{
2018-06-05 14:55:35 +00:00
/// Overview of the mutation algorithm.
///
/// When the client executes a mutation, this method is called. It acquires block numbers in all
/// partitions, saves them in the mutation entry and writes the mutation entry to a new ZK node in
/// the /mutations folder. This block numbers are needed to determine which parts should be mutated and
/// which shouldn't (parts inserted after the mutation will have the block number higher than the
/// block number acquired by the mutation in that partition and so will not be mutatied).
/// This block number is called "mutation version" in that partition.
///
/// Mutation versions are acquired atomically in all partitions, so the case when an insert in some
/// partition has the block number higher than the mutation version but the following insert into another
/// partition acquires the block number lower than the mutation version in that partition is impossible.
/// Another important invariant: mutation entries appear in /mutations in the order of their mutation
/// versions (in any partition). This means that mutations form a sequence and we can execute them in
/// the order of their mutation versions and not worry that some mutation with the smaller version
/// will suddenly appear.
///
/// During mutations individual parts are immutable - when we want to change the contents of a part
/// we prepare the new part and add it to MergeTreeData (the original part gets replaced). The fact that
/// we have mutated the part is recorded in the part->info.mutation field of MergeTreePartInfo.
/// The relation with the original part is preserved because the new part covers the same block range
/// as the original one.
///
/// We then can for each part determine its "mutation version": the version of the last mutation in
/// the mutation sequence that we regard as already applied to that part. All mutations with the greater
/// version number will still need to be applied to that part.
///
/// Execution of mutations is done asynchronously. All replicas watch the /mutations directory and
/// load new mutation entries as they appear (see mutationsUpdatingTask()). Next we need to determine
/// how to mutate individual parts consistently with part merges. This is done by the leader replica
/// (see mergeSelectingTask() and class ReplicatedMergeTreeMergePredicate for details). Important
/// invariants here are that a) all source parts for a single merge must have the same mutation version
/// and b) any part can be mutated only once or merged only once (e.g. once we have decided to mutate
/// a part then we need to execute that mutation and can assign merges only to the new part and not to the
/// original part). Multiple consecutive mutations can be executed at once (without writing the
/// intermediate result to a part).
///
/// Leader replica records its decisions to the replication log (/log directory in ZK) in the form of
/// MUTATE_PART entries and all replicas then execute them in the background pool
2021-09-16 21:19:58 +00:00
/// (see MutateTask class). When a replica encounters a MUTATE_PART command, it is
2018-06-05 14:55:35 +00:00
/// guaranteed that the corresponding mutation entry is already loaded (when we pull entries from
/// replication log into the replica queue, we also load mutation entries). Note that just as with merges
/// the replica can decide not to do the mutation locally and fetch the mutated part from another replica
/// instead.
///
/// Mutations of individual parts are in fact pretty similar to merges, e.g. their assignment and execution
2019-08-13 10:29:31 +00:00
/// is governed by the same storage_settings. TODO: support a single "merge-mutation" operation when the data
2018-06-05 14:55:35 +00:00
/// read from the the source parts is first mutated on the fly to some uniform mutation version and then
/// merged to a resulting part.
///
/// After all needed parts are mutated (i.e. all active parts have the mutation version greater than
/// the version of this mutation), the mutation is considered done and can be deleted.
2020-11-10 10:23:46 +00:00
ReplicatedMergeTreeMutationEntry mutation_entry ;
mutation_entry . source_replica = replica_name ;
mutation_entry . commands = commands ;
2018-04-19 10:33:16 +00:00
2021-05-08 10:59:55 +00:00
const String mutations_path = fs : : path ( zookeeper_path ) / " mutations " ;
2020-11-10 10:23:46 +00:00
const auto zookeeper = getZooKeeper ( ) ;
2018-04-19 10:33:16 +00:00
/// Update the mutations_path node when creating the mutation and check its version to ensure that
/// nodes for mutations are created in the same order as the corresponding block numbers.
/// Should work well if the number of concurrent mutation requests is small.
while ( true )
{
2018-08-25 01:58:14 +00:00
Coordination : : Stat mutations_stat ;
2018-04-19 10:33:16 +00:00
zookeeper - > get ( mutations_path , & mutations_stat ) ;
2020-11-10 10:23:46 +00:00
PartitionBlockNumbersHolder partition_block_numbers_holder =
allocateBlockNumbersInAffectedPartitions ( mutation_entry . commands , query_context , zookeeper ) ;
2018-04-19 10:33:16 +00:00
2020-11-10 10:23:46 +00:00
mutation_entry . block_numbers = partition_block_numbers_holder . getBlockNumbers ( ) ;
mutation_entry . create_time = time ( nullptr ) ;
2018-04-19 10:33:16 +00:00
2020-11-10 10:23:46 +00:00
/// The following version check guarantees the linearizability property for any pair of mutations:
/// mutation with higher sequence number is guaranteed to have higher block numbers in every partition
/// (and thus will be applied strictly according to sequence numbers of mutations)
2018-08-25 01:58:14 +00:00
Coordination : : Requests requests ;
2018-04-19 10:33:16 +00:00
requests . emplace_back ( zkutil : : makeSetRequest ( mutations_path , String ( ) , mutations_stat . version ) ) ;
requests . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( mutations_path ) / " " , mutation_entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2018-04-19 10:33:16 +00:00
2021-04-10 23:33:54 +00:00
if ( auto txn = query_context - > getZooKeeperMetadataTransaction ( ) )
2021-02-08 19:36:17 +00:00
txn - > moveOpsTo ( requests ) ;
2018-08-25 01:58:14 +00:00
Coordination : : Responses responses ;
2020-06-12 15:09:12 +00:00
Coordination : : Error rc = zookeeper - > tryMulti ( requests , responses ) ;
2018-04-19 10:33:16 +00:00
2020-11-10 10:23:46 +00:00
partition_block_numbers_holder . reset ( ) ;
2020-06-12 15:09:12 +00:00
if ( rc = = Coordination : : Error : : ZOK )
2018-06-21 13:27:36 +00:00
{
const String & path_created =
2018-08-25 01:58:14 +00:00
dynamic_cast < const Coordination : : CreateResponse * > ( responses [ 1 ] . get ( ) ) - > path_created ;
2020-11-10 10:23:46 +00:00
mutation_entry . znode_name = path_created . substr ( path_created . find_last_of ( ' / ' ) + 1 ) ;
LOG_TRACE ( log , " Created mutation with ID {} " , mutation_entry . znode_name ) ;
2018-04-19 10:33:16 +00:00
break ;
2018-06-21 13:27:36 +00:00
}
2020-06-12 15:09:12 +00:00
else if ( rc = = Coordination : : Error : : ZBADVERSION )
2018-04-19 10:33:16 +00:00
{
2021-07-30 16:34:18 +00:00
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if ( query_context - > getZooKeeperMetadataTransaction ( ) )
throw Exception ( " Cannot execute alter, because mutations version was suddenly changed due to concurrent alter " ,
ErrorCodes : : CANNOT_ASSIGN_ALTER ) ;
2018-04-19 10:33:16 +00:00
LOG_TRACE ( log , " Version conflict when trying to create a mutation node, retrying... " ) ;
continue ;
}
else
2018-08-25 01:58:14 +00:00
throw Coordination : : Exception ( " Unable to create a mutation znode " , rc ) ;
2018-04-19 10:33:16 +00:00
}
2019-12-16 15:51:15 +00:00
2021-04-10 23:33:54 +00:00
waitMutation ( mutation_entry . znode_name , query_context - > getSettingsRef ( ) . mutations_sync ) ;
2020-01-13 16:39:20 +00:00
}
2020-01-31 12:25:31 +00:00
void StorageReplicatedMergeTree : : waitMutation ( const String & znode_name , size_t mutations_sync ) const
2020-01-29 17:28:39 +00:00
{
2020-03-20 02:49:47 +00:00
if ( ! mutations_sync )
return ;
2019-12-19 15:27:56 +00:00
/// we have to wait
2020-03-20 02:49:47 +00:00
auto zookeeper = getZooKeeper ( ) ;
Strings replicas ;
if ( mutations_sync = = 2 ) /// wait for all replicas
2021-09-11 12:16:05 +00:00
{
2021-05-08 10:59:55 +00:00
replicas = zookeeper - > getChildren ( fs : : path ( zookeeper_path ) / " replicas " ) ;
2021-09-11 12:16:05 +00:00
/// This replica should be first, to ensure that the mutation will be loaded into memory
for ( auto it = replicas . begin ( ) ; it ! = replicas . end ( ) ; + + it )
{
if ( * it = = replica_name )
{
std : : iter_swap ( it , replicas . rbegin ( ) ) ;
break ;
}
}
}
2020-03-20 02:49:47 +00:00
else if ( mutations_sync = = 1 ) /// just wait for ourself
replicas . push_back ( replica_name ) ;
2019-12-19 15:27:56 +00:00
2020-03-20 02:49:47 +00:00
waitMutationToFinishOnReplicas ( replicas , znode_name ) ;
2018-04-19 10:33:16 +00:00
}
2018-06-07 13:28:39 +00:00
std : : vector < MergeTreeMutationStatus > StorageReplicatedMergeTree : : getMutationsStatus ( ) const
{
return queue . getMutationsStatus ( ) ;
}
2019-02-04 13:04:02 +00:00
CancellationCode StorageReplicatedMergeTree : : killMutation ( const String & mutation_id )
2019-01-10 18:19:29 +00:00
{
2019-02-04 12:53:25 +00:00
assertNotReadonly ( ) ;
zkutil : : ZooKeeperPtr zookeeper = getZooKeeper ( ) ;
2021-08-18 09:49:22 +00:00
LOG_INFO ( log , " Killing mutation {} " , mutation_id ) ;
2019-02-04 12:53:25 +00:00
auto mutation_entry = queue . removeMutation ( zookeeper , mutation_id ) ;
if ( ! mutation_entry )
2019-02-04 13:04:02 +00:00
return CancellationCode : : NotFound ;
2019-02-04 12:53:25 +00:00
/// After this point no new part mutations will start and part mutations that still exist
/// in the queue will be skipped.
/// Cancel already running part mutations.
for ( const auto & pair : mutation_entry - > block_numbers )
{
const String & partition_id = pair . first ;
Int64 block_number = pair . second ;
2021-06-06 12:24:49 +00:00
getContext ( ) - > getMergeList ( ) . cancelPartMutations ( getStorageID ( ) , partition_id , block_number ) ;
2019-02-04 12:53:25 +00:00
}
2019-02-04 13:04:02 +00:00
return CancellationCode : : CancelSent ;
2019-01-10 18:19:29 +00:00
}
2017-10-06 11:30:57 +00:00
void StorageReplicatedMergeTree : : clearOldPartsAndRemoveFromZK ( )
2017-05-24 20:19:29 +00:00
{
2020-06-18 16:10:47 +00:00
auto table_lock = lockForShare (
RWLockImpl : : NO_QUERY , getSettings ( ) - > lock_acquire_timeout_for_background_operations ) ;
2017-05-24 20:19:29 +00:00
auto zookeeper = getZooKeeper ( ) ;
2019-05-03 02:00:57 +00:00
DataPartsVector parts = grabOldParts ( ) ;
2017-10-03 14:44:10 +00:00
if ( parts . empty ( ) )
2017-05-24 20:19:29 +00:00
return ;
2019-05-03 02:00:57 +00:00
DataPartsVector parts_to_delete_only_from_filesystem ; // Only duplicates
DataPartsVector parts_to_delete_completely ; // All parts except duplicates
DataPartsVector parts_to_retry_deletion ; // Parts that should be retried due to network problems
DataPartsVector parts_to_remove_from_filesystem ; // Parts removed from ZK
2017-10-06 11:30:57 +00:00
for ( const auto & part : parts )
2017-08-29 11:58:33 +00:00
{
2017-10-06 11:30:57 +00:00
if ( ! part - > is_duplicate )
parts_to_delete_completely . emplace_back ( part ) ;
else
parts_to_delete_only_from_filesystem . emplace_back ( part ) ;
}
parts . clear ( ) ;
2017-08-29 11:58:33 +00:00
2017-10-06 11:30:57 +00:00
/// Delete duplicate parts from filesystem
if ( ! parts_to_delete_only_from_filesystem . empty ( ) )
{
2021-06-08 19:11:22 +00:00
clearPartsFromFilesystem ( parts_to_delete_only_from_filesystem ) ;
2019-05-03 02:00:57 +00:00
removePartsFinally ( parts_to_delete_only_from_filesystem ) ;
2017-10-06 11:30:57 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed {} old duplicate parts " , parts_to_delete_only_from_filesystem . size ( ) ) ;
2017-10-06 11:30:57 +00:00
}
/// Delete normal parts from ZooKeeper
2017-10-03 14:44:10 +00:00
NameSet part_names_to_retry_deletion ;
2017-05-24 20:19:29 +00:00
try
{
2017-10-06 11:30:57 +00:00
Strings part_names_to_delete_completely ;
for ( const auto & part : parts_to_delete_completely )
part_names_to_delete_completely . emplace_back ( part - > name ) ;
2017-08-09 21:09:44 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removing {} old parts from ZooKeeper " , parts_to_delete_completely . size ( ) ) ;
2017-10-06 11:30:57 +00:00
removePartsFromZooKeeper ( zookeeper , part_names_to_delete_completely , & part_names_to_retry_deletion ) ;
2017-05-24 20:19:29 +00:00
}
catch ( . . . )
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " There is a problem with deleting parts from ZooKeeper: {} " , getCurrentExceptionMessage ( true ) ) ;
2017-10-03 14:44:10 +00:00
}
2017-08-29 11:58:33 +00:00
2017-10-03 14:44:10 +00:00
/// Part names that were reliably deleted from ZooKeeper should be deleted from filesystem
2017-10-06 11:30:57 +00:00
auto num_reliably_deleted_parts = parts_to_delete_completely . size ( ) - part_names_to_retry_deletion . size ( ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed {} old parts from ZooKeeper. Removing them from filesystem. " , num_reliably_deleted_parts ) ;
2017-08-29 11:58:33 +00:00
2017-10-06 11:30:57 +00:00
/// Delete normal parts on two sets
for ( auto & part : parts_to_delete_completely )
2017-10-03 14:44:10 +00:00
{
if ( part_names_to_retry_deletion . count ( part - > name ) = = 0 )
parts_to_remove_from_filesystem . emplace_back ( part ) ;
else
parts_to_retry_deletion . emplace_back ( part ) ;
2017-05-24 20:19:29 +00:00
}
2017-10-03 14:44:10 +00:00
/// Will retry deletion
if ( ! parts_to_retry_deletion . empty ( ) )
2017-10-06 11:30:57 +00:00
{
2019-05-03 02:00:57 +00:00
rollbackDeletingParts ( parts_to_retry_deletion ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Will retry deletion of {} parts in the next time " , parts_to_retry_deletion . size ( ) ) ;
2017-10-06 11:30:57 +00:00
}
2017-08-29 11:58:33 +00:00
2017-10-03 14:44:10 +00:00
/// Remove parts from filesystem and finally from data_parts
2017-10-06 11:30:57 +00:00
if ( ! parts_to_remove_from_filesystem . empty ( ) )
2017-10-03 14:44:10 +00:00
{
2021-06-08 19:11:22 +00:00
clearPartsFromFilesystem ( parts_to_remove_from_filesystem ) ;
2019-05-03 02:00:57 +00:00
removePartsFinally ( parts_to_remove_from_filesystem ) ;
2017-08-29 11:58:33 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removed {} old parts " , parts_to_remove_from_filesystem . size ( ) ) ;
2017-10-06 11:30:57 +00:00
}
2017-05-24 20:19:29 +00:00
}
2019-05-03 02:00:57 +00:00
bool StorageReplicatedMergeTree : : tryRemovePartsFromZooKeeperWithRetries ( DataPartsVector & parts , size_t max_retries )
2018-05-21 13:49:54 +00:00
{
Strings part_names_to_remove ;
for ( const auto & part : parts )
part_names_to_remove . emplace_back ( part - > name ) ;
return tryRemovePartsFromZooKeeperWithRetries ( part_names_to_remove , max_retries ) ;
}
bool StorageReplicatedMergeTree : : tryRemovePartsFromZooKeeperWithRetries ( const Strings & part_names , size_t max_retries )
{
size_t num_tries = 0 ;
2018-12-11 13:30:20 +00:00
bool success = false ;
2018-05-21 13:49:54 +00:00
2018-12-11 13:30:20 +00:00
while ( ! success & & ( max_retries = = 0 | | num_tries < max_retries ) )
2018-05-21 13:49:54 +00:00
{
try
{
2018-12-11 13:30:20 +00:00
+ + num_tries ;
success = true ;
2018-05-21 13:49:54 +00:00
auto zookeeper = getZooKeeper ( ) ;
2018-12-11 13:30:20 +00:00
std : : vector < std : : future < Coordination : : ExistsResponse > > exists_futures ;
exists_futures . reserve ( part_names . size ( ) ) ;
2018-05-21 13:49:54 +00:00
for ( const String & part_name : part_names )
{
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2018-12-11 13:30:20 +00:00
exists_futures . emplace_back ( zookeeper - > asyncExists ( part_path ) ) ;
}
2018-05-21 13:49:54 +00:00
2018-12-11 13:30:20 +00:00
std : : vector < std : : future < Coordination : : MultiResponse > > remove_futures ;
remove_futures . reserve ( part_names . size ( ) ) ;
for ( size_t i = 0 ; i < part_names . size ( ) ; + + i )
{
Coordination : : ExistsResponse exists_resp = exists_futures [ i ] . get ( ) ;
2020-06-12 15:09:12 +00:00
if ( exists_resp . error = = Coordination : : Error : : ZOK )
2018-12-11 13:30:20 +00:00
{
Coordination : : Requests ops ;
removePartFromZooKeeper ( part_names [ i ] , ops , exists_resp . stat . numChildren > 0 ) ;
2021-06-01 07:30:06 +00:00
remove_futures . emplace_back ( zookeeper - > asyncTryMultiNoThrow ( ops ) ) ;
2018-12-11 13:30:20 +00:00
}
2018-05-21 13:49:54 +00:00
}
2018-12-11 13:30:20 +00:00
for ( auto & future : remove_futures )
2018-05-21 13:49:54 +00:00
{
auto response = future . get ( ) ;
2020-06-12 15:09:12 +00:00
if ( response . error = = Coordination : : Error : : ZOK | | response . error = = Coordination : : Error : : ZNONODE )
2018-05-21 13:49:54 +00:00
continue ;
2018-08-25 01:58:14 +00:00
if ( Coordination : : isHardwareError ( response . error ) )
2018-05-21 13:49:54 +00:00
{
2018-12-11 13:30:20 +00:00
success = false ;
2018-05-21 13:49:54 +00:00
continue ;
}
2018-08-25 01:58:14 +00:00
throw Coordination : : Exception ( response . error ) ;
2018-05-21 13:49:54 +00:00
}
}
2018-08-25 01:58:14 +00:00
catch ( Coordination : : Exception & e )
2018-05-21 13:49:54 +00:00
{
2018-12-11 13:30:20 +00:00
success = false ;
2018-05-21 13:49:54 +00:00
2018-08-25 01:58:14 +00:00
if ( Coordination : : isHardwareError ( e . code ) )
2018-05-21 13:49:54 +00:00
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
else
throw ;
}
2018-12-11 13:30:20 +00:00
if ( ! success & & num_tries < max_retries )
2018-05-21 13:49:54 +00:00
std : : this_thread : : sleep_for ( std : : chrono : : milliseconds ( 1000 ) ) ;
}
2018-12-11 13:30:20 +00:00
return success ;
2018-05-21 13:49:54 +00:00
}
2018-12-11 13:30:20 +00:00
void StorageReplicatedMergeTree : : removePartsFromZooKeeper (
zkutil : : ZooKeeperPtr & zookeeper , const Strings & part_names , NameSet * parts_should_be_retried )
2017-08-09 21:09:44 +00:00
{
2018-12-11 13:30:20 +00:00
std : : vector < std : : future < Coordination : : ExistsResponse > > exists_futures ;
std : : vector < std : : future < Coordination : : MultiResponse > > remove_futures ;
2019-06-03 14:58:19 +00:00
exists_futures . reserve ( part_names . size ( ) ) ;
2018-12-11 13:30:20 +00:00
remove_futures . reserve ( part_names . size ( ) ) ;
try
{
2019-06-03 14:58:19 +00:00
/// Exception can be thrown from loop
/// if zk session will be dropped
for ( const String & part_name : part_names )
{
2021-05-08 10:59:55 +00:00
String part_path = fs : : path ( replica_path ) / " parts " / part_name ;
2019-06-03 14:58:19 +00:00
exists_futures . emplace_back ( zookeeper - > asyncExists ( part_path ) ) ;
}
2018-12-11 13:30:20 +00:00
for ( size_t i = 0 ; i < part_names . size ( ) ; + + i )
2017-08-09 21:09:44 +00:00
{
2018-12-11 13:30:20 +00:00
Coordination : : ExistsResponse exists_resp = exists_futures [ i ] . get ( ) ;
2020-06-12 15:09:12 +00:00
if ( exists_resp . error = = Coordination : : Error : : ZOK )
2017-08-29 11:58:33 +00:00
{
2018-12-11 13:30:20 +00:00
Coordination : : Requests ops ;
removePartFromZooKeeper ( part_names [ i ] , ops , exists_resp . stat . numChildren > 0 ) ;
2021-06-01 07:30:06 +00:00
remove_futures . emplace_back ( zookeeper - > asyncTryMultiNoThrow ( ops ) ) ;
2017-10-03 14:44:10 +00:00
}
2018-12-11 13:30:20 +00:00
else
2017-08-29 11:58:33 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " There is no part {} in ZooKeeper, it was only in filesystem " , part_names [ i ] ) ;
2018-12-11 13:30:20 +00:00
// emplace invalid future so that the total number of futures is the same as part_names.size();
remove_futures . emplace_back ( ) ;
2017-08-29 11:58:33 +00:00
}
2018-12-11 13:30:20 +00:00
}
}
catch ( const Coordination : : Exception & e )
{
if ( parts_should_be_retried & & Coordination : : isHardwareError ( e . code ) )
parts_should_be_retried - > insert ( part_names . begin ( ) , part_names . end ( ) ) ;
throw ;
}
for ( size_t i = 0 ; i < remove_futures . size ( ) ; + + i )
{
auto & future = remove_futures [ i ] ;
if ( ! future . valid ( ) )
continue ;
2017-08-29 11:58:33 +00:00
2018-12-11 13:30:20 +00:00
auto response = future . get ( ) ;
2020-06-12 15:09:12 +00:00
if ( response . error = = Coordination : : Error : : ZOK )
2018-12-11 13:30:20 +00:00
continue ;
2020-06-12 15:09:12 +00:00
else if ( response . error = = Coordination : : Error : : ZNONODE )
2018-12-11 13:30:20 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " There is no part {} in ZooKeeper, it was only in filesystem " , part_names [ i ] ) ;
2018-12-11 13:30:20 +00:00
continue ;
2017-08-09 21:09:44 +00:00
}
2018-12-11 13:30:20 +00:00
else if ( Coordination : : isHardwareError ( response . error ) )
{
if ( parts_should_be_retried )
parts_should_be_retried - > insert ( part_names [ i ] ) ;
continue ;
2017-08-09 21:09:44 +00:00
}
2018-12-11 13:30:20 +00:00
else
2020-06-12 15:09:12 +00:00
LOG_WARNING ( log , " Cannot remove part {} from ZooKeeper: {} " , part_names [ i ] , Coordination : : errorMessage ( response . error ) ) ;
2017-08-09 21:09:44 +00:00
}
}
2020-11-02 17:30:53 +00:00
void StorageReplicatedMergeTree : : getClearBlocksInPartitionOps (
Coordination : : Requests & ops , zkutil : : ZooKeeper & zookeeper , const String & partition_id , Int64 min_block_num , Int64 max_block_num )
2017-11-15 16:32:47 +00:00
{
Strings blocks ;
2021-05-08 10:59:55 +00:00
if ( Coordination : : Error : : ZOK ! = zookeeper . tryGetChildren ( fs : : path ( zookeeper_path ) / " blocks " , blocks ) )
2017-11-15 16:32:47 +00:00
throw Exception ( zookeeper_path + " /blocks doesn't exist " , ErrorCodes : : NOT_FOUND_NODE ) ;
String partition_prefix = partition_id + " _ " ;
2018-08-25 01:58:14 +00:00
zkutil : : AsyncResponses < Coordination : : GetResponse > get_futures ;
2021-08-24 12:57:49 +00:00
2017-11-15 16:32:47 +00:00
for ( const String & block_id : blocks )
{
if ( startsWith ( block_id , partition_prefix ) )
{
2021-05-08 10:59:55 +00:00
String path = fs : : path ( zookeeper_path ) / " blocks " / block_id ;
2017-11-15 16:32:47 +00:00
get_futures . emplace_back ( path , zookeeper . asyncTryGet ( path ) ) ;
}
}
for ( auto & pair : get_futures )
{
const String & path = pair . first ;
2018-03-24 01:00:12 +00:00
auto result = pair . second . get ( ) ;
2017-11-15 16:32:47 +00:00
2020-06-12 15:09:12 +00:00
if ( result . error = = Coordination : : Error : : ZNONODE )
2017-11-15 16:32:47 +00:00
continue ;
2018-03-24 01:00:12 +00:00
ReadBufferFromString buf ( result . data ) ;
2021-08-24 12:57:49 +00:00
const auto part_info = MergeTreePartInfo : : tryParsePartName ( result . data , format_version ) ;
if ( ! part_info | | ( min_block_num < = part_info - > min_block & & part_info - > max_block < = max_block_num ) )
2020-11-02 17:30:53 +00:00
ops . emplace_back ( zkutil : : makeRemoveRequest ( path , - 1 ) ) ;
2017-11-15 16:32:47 +00:00
}
2020-11-02 17:30:53 +00:00
}
2017-11-15 16:32:47 +00:00
2020-11-02 17:30:53 +00:00
void StorageReplicatedMergeTree : : clearBlocksInPartition (
zkutil : : ZooKeeper & zookeeper , const String & partition_id , Int64 min_block_num , Int64 max_block_num )
{
Coordination : : Requests delete_requests ;
getClearBlocksInPartitionOps ( delete_requests , zookeeper , partition_id , min_block_num , max_block_num ) ;
Coordination : : Responses delete_responses ;
auto code = zookeeper . tryMulti ( delete_requests , delete_responses ) ;
if ( code ! = Coordination : : Error : : ZOK )
2017-11-15 16:32:47 +00:00
{
2020-11-02 17:30:53 +00:00
for ( size_t i = 0 ; i < delete_requests . size ( ) ; + + i )
if ( delete_responses [ i ] - > error ! = Coordination : : Error : : ZOK )
LOG_WARNING ( log , " Error while deleting ZooKeeper path `{}`: {}, ignoring. " , delete_requests [ i ] - > getPath ( ) , Coordination : : errorMessage ( delete_responses [ i ] - > error ) ) ;
2017-11-15 16:32:47 +00:00
}
2020-11-02 17:30:53 +00:00
LOG_TRACE ( log , " Deleted {} deduplication block IDs in partition ID {} " , delete_requests . size ( ) , partition_id ) ;
2017-11-15 16:32:47 +00:00
}
2020-06-26 11:30:23 +00:00
void StorageReplicatedMergeTree : : replacePartitionFrom (
2021-04-10 23:33:54 +00:00
const StoragePtr & source_table , const ASTPtr & partition , bool replace , ContextPtr query_context )
2018-05-21 13:49:54 +00:00
{
2020-03-17 18:07:54 +00:00
/// First argument is true, because we possibly will add new data to current table.
2021-04-10 23:33:54 +00:00
auto lock1 = lockForShare ( query_context - > getCurrentQueryId ( ) , query_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
auto lock2 = source_table - > lockForShare ( query_context - > getCurrentQueryId ( ) , query_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
2018-05-21 13:49:54 +00:00
2020-06-17 10:34:23 +00:00
auto source_metadata_snapshot = source_table - > getInMemoryMetadataPtr ( ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2018-05-21 13:49:54 +00:00
Stopwatch watch ;
2020-06-17 10:34:23 +00:00
MergeTreeData & src_data = checkStructureAndGetMergeTreeData ( source_table , source_metadata_snapshot , metadata_snapshot ) ;
2021-04-10 23:33:54 +00:00
String partition_id = getPartitionIDFromQuery ( partition , query_context ) ;
2018-05-21 13:49:54 +00:00
2021-08-23 17:50:50 +00:00
/// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet.
2019-05-03 02:00:57 +00:00
DataPartsVector src_all_parts = src_data . getDataPartsVectorInPartition ( MergeTreeDataPartState : : Committed , partition_id ) ;
2018-05-21 13:49:54 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Cloning {} parts " , src_all_parts . size ( ) ) ;
2018-05-21 13:49:54 +00:00
static const String TMP_PREFIX = " tmp_replace_from_ " ;
2021-11-03 07:46:33 +00:00
auto zookeeper = getZooKeeper ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
/// Retry if alter_partition_version changes
for ( size_t retry = 0 ; retry < 1000 ; + + retry )
2018-05-21 13:49:54 +00:00
{
2021-11-03 07:46:33 +00:00
DataPartsVector src_parts ;
MutableDataPartsVector dst_parts ;
Strings block_id_paths ;
Strings part_checksums ;
std : : vector < EphemeralLockInZooKeeper > ephemeral_locks ;
String alter_partition_version_path = zookeeper_path + " /alter_partition_version " ;
Coordination : : Stat alter_partition_version_stat ;
zookeeper - > get ( alter_partition_version_path , & alter_partition_version_stat ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
/// Firstly, generate last block number and compute drop_range
/// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block.
/// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop.
/// TODO why not to add normal DROP_RANGE entry to replication queue if `replace` is true?
MergeTreePartInfo drop_range ;
std : : optional < EphemeralLockInZooKeeper > delimiting_block_lock ;
bool partition_was_empty = ! getFakePartCoveringAllPartsInPartition ( partition_id , drop_range , delimiting_block_lock , true ) ;
if ( replace & & partition_was_empty )
{
/// Nothing to drop, will just attach new parts
LOG_INFO ( log , " Partition {} was empty, REPLACE PARTITION will work as ATTACH PARTITION FROM " , drop_range . partition_id ) ;
replace = false ;
}
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
if ( ! replace )
{
/// It's ATTACH PARTITION FROM, not REPLACE PARTITION. We have to reset drop range
drop_range = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom ( partition_id ) ;
}
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
assert ( replace = = ! LogEntry : : ReplaceRangeEntry : : isMovePartitionOrAttachFrom ( drop_range ) ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
String drop_range_fake_part_name = getPartNamePossiblyFake ( format_version , drop_range ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
for ( const auto & src_part : src_all_parts )
{
/// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION
/// Assume that merges in the partition are quite rare
/// Save deduplication block ids with special prefix replace_partition
2019-06-19 16:16:13 +00:00
2021-11-03 07:46:33 +00:00
if ( ! canReplacePartition ( src_part ) )
throw Exception (
" Cannot replace partition ' " + partition_id + " ' because part ' " + src_part - > name + " ' has inconsistent granularity with table " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2020-03-07 00:05:49 +00:00
2021-11-03 07:46:33 +00:00
String hash_hex = src_part - > checksums . getTotalChecksumHex ( ) ;
2020-03-07 00:05:49 +00:00
2021-11-03 07:46:33 +00:00
if ( replace )
LOG_INFO ( log , " Trying to replace {} with hash_hex {} " , src_part - > name , hash_hex ) ;
else
LOG_INFO ( log , " Trying to attach {} with hash_hex {} " , src_part - > name , hash_hex ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
String block_id_path = replace ? " " : ( fs : : path ( zookeeper_path ) / " blocks " / ( partition_id + " _replace_from_ " + hash_hex ) ) ;
auto lock = allocateBlockNumber ( partition_id , zookeeper , block_id_path ) ;
if ( ! lock )
{
LOG_INFO ( log , " Part {} (hash {}) has been already attached " , src_part - > name , hash_hex ) ;
continue ;
}
UInt64 index = lock - > getNumber ( ) ;
MergeTreePartInfo dst_part_info ( partition_id , index , index , src_part - > info . level ) ;
auto dst_part = cloneAndLoadDataPartOnSameDisk ( src_part , TMP_PREFIX , dst_part_info , metadata_snapshot ) ;
src_parts . emplace_back ( src_part ) ;
dst_parts . emplace_back ( dst_part ) ;
ephemeral_locks . emplace_back ( std : : move ( * lock ) ) ;
block_id_paths . emplace_back ( block_id_path ) ;
part_checksums . emplace_back ( hash_hex ) ;
2018-05-21 13:49:54 +00:00
}
2021-11-03 07:46:33 +00:00
ReplicatedMergeTreeLogEntryData entry ;
{
auto src_table_id = src_data . getStorageID ( ) ;
entry . type = ReplicatedMergeTreeLogEntryData : : REPLACE_RANGE ;
entry . source_replica = replica_name ;
entry . create_time = time ( nullptr ) ;
entry . replace_range_entry = std : : make_shared < ReplicatedMergeTreeLogEntryData : : ReplaceRangeEntry > ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
auto & entry_replace = * entry . replace_range_entry ;
entry_replace . drop_range_part_name = drop_range_fake_part_name ;
entry_replace . from_database = src_table_id . database_name ;
entry_replace . from_table = src_table_id . table_name ;
for ( const auto & part : src_parts )
entry_replace . src_part_names . emplace_back ( part - > name ) ;
for ( const auto & part : dst_parts )
entry_replace . new_part_names . emplace_back ( part - > name ) ;
for ( const String & checksum : part_checksums )
entry_replace . part_names_checksums . emplace_back ( checksum ) ;
entry_replace . columns_version = - 1 ;
}
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
/// Remove deduplication block_ids of replacing parts
if ( replace )
clearBlocksInPartition ( * zookeeper , drop_range . partition_id , drop_range . max_block , drop_range . max_block ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
DataPartsVector parts_to_remove ;
Coordination : : Responses op_results ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
try
2018-05-21 13:49:54 +00:00
{
2021-11-03 07:46:33 +00:00
Coordination : : Requests ops ;
for ( size_t i = 0 ; i < dst_parts . size ( ) ; + + i )
2018-05-21 13:49:54 +00:00
{
2021-11-03 07:46:33 +00:00
getCommitPartOps ( ops , dst_parts [ i ] , block_id_paths [ i ] ) ;
ephemeral_locks [ i ] . getUnlockOps ( ops ) ;
2018-05-21 13:49:54 +00:00
}
2021-11-03 07:46:33 +00:00
if ( auto txn = query_context - > getZooKeeperMetadataTransaction ( ) )
txn - > moveOpsTo ( ops ) ;
2021-02-08 19:36:17 +00:00
2021-11-03 07:46:33 +00:00
delimiting_block_lock - > getUnlockOps ( ops ) ;
/// Check and update version to avoid race with DROP_RANGE
ops . emplace_back ( zkutil : : makeSetRequest ( alter_partition_version_path , " " , alter_partition_version_stat . version ) ) ;
/// Just update version, because merges assignment relies on it
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " log " , " " , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( fs : : path ( zookeeper_path ) / " log/log- " , entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
Transaction transaction ( * this ) ;
{
auto data_parts_lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
for ( MutableDataPartPtr & part : dst_parts )
renameTempPartAndReplace ( part , nullptr , & transaction , data_parts_lock ) ;
}
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , op_results ) ;
if ( code = = Coordination : : Error : : ZOK )
delimiting_block_lock - > assumeUnlocked ( ) ;
else if ( code = = Coordination : : Error : : ZBADVERSION )
2021-11-09 10:02:17 +00:00
{
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if ( query_context - > getZooKeeperMetadataTransaction ( ) )
throw Exception (
2021-11-09 10:19:00 +00:00
" Cannot execute alter, because alter partition version was suddenly changed due to concurrent alter " ,
2021-11-09 10:02:17 +00:00
ErrorCodes : : CANNOT_ASSIGN_ALTER ) ;
2021-11-03 07:46:33 +00:00
continue ;
2021-11-09 10:02:17 +00:00
}
2021-11-03 07:46:33 +00:00
else
zkutil : : KeeperMultiException : : check ( code , ops , op_results ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
{
auto data_parts_lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
transaction . commit ( & data_parts_lock ) ;
if ( replace )
parts_to_remove = removePartsInRangeFromWorkingSet ( drop_range , true , data_parts_lock ) ;
}
PartLog : : addNewParts ( getContext ( ) , dst_parts , watch . elapsed ( ) ) ;
}
catch ( . . . )
{
PartLog : : addNewParts ( getContext ( ) , dst_parts , watch . elapsed ( ) , ExecutionStatus : : fromCurrentException ( ) ) ;
throw ;
2018-05-21 13:49:54 +00:00
}
2021-11-03 07:46:33 +00:00
String log_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * op_results . back ( ) ) . path_created ;
entry . znode_name = log_znode_path . substr ( log_znode_path . find_last_of ( ' / ' ) + 1 ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
for ( auto & lock : ephemeral_locks )
lock . assumeUnlocked ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
/// Forcibly remove replaced parts from ZooKeeper
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
/// Speedup removing of replaced parts from filesystem
parts_to_remove . clear ( ) ;
cleanup_thread . wakeup ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
lock2 . reset ( ) ;
lock1 . reset ( ) ;
2018-05-21 13:49:54 +00:00
2021-11-03 07:46:33 +00:00
waitForLogEntryToBeProcessedIfNecessary ( entry , query_context ) ;
2021-08-20 12:59:57 +00:00
2021-11-03 07:46:33 +00:00
return ;
}
throw Exception (
ErrorCodes : : CANNOT_ASSIGN_ALTER , " Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed " ) ;
2018-05-21 13:49:54 +00:00
}
2021-04-10 23:33:54 +00:00
void StorageReplicatedMergeTree : : movePartitionToTable ( const StoragePtr & dest_table , const ASTPtr & partition , ContextPtr query_context )
2019-07-26 08:42:17 +00:00
{
2021-04-10 23:33:54 +00:00
auto lock1 = lockForShare ( query_context - > getCurrentQueryId ( ) , query_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
auto lock2 = dest_table - > lockForShare ( query_context - > getCurrentQueryId ( ) , query_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
2019-09-17 09:00:20 +00:00
2019-08-28 08:24:17 +00:00
auto dest_table_storage = std : : dynamic_pointer_cast < StorageReplicatedMergeTree > ( dest_table ) ;
if ( ! dest_table_storage )
2020-02-21 16:57:40 +00:00
throw Exception ( " Table " + getStorageID ( ) . getNameForLogs ( ) + " supports movePartitionToTable only for ReplicatedMergeTree family of table engines. "
2019-08-28 08:24:17 +00:00
" Got " + dest_table - > getName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
2020-01-08 09:57:27 +00:00
if ( dest_table_storage - > getStoragePolicy ( ) ! = this - > getStoragePolicy ( ) )
2020-01-15 11:55:20 +00:00
throw Exception ( " Destination table " + dest_table_storage - > getStorageID ( ) . getNameForLogs ( ) +
" should have the same storage policy of source table " + getStorageID ( ) . getNameForLogs ( ) + " . " +
getStorageID ( ) . getNameForLogs ( ) + " : " + this - > getStoragePolicy ( ) - > getName ( ) + " , " +
2020-10-16 11:58:47 +00:00
getStorageID ( ) . getNameForLogs ( ) + " : " + dest_table_storage - > getStoragePolicy ( ) - > getName ( ) , ErrorCodes : : UNKNOWN_POLICY ) ;
2019-08-28 08:24:17 +00:00
2020-06-17 10:34:23 +00:00
auto dest_metadata_snapshot = dest_table - > getInMemoryMetadataPtr ( ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2019-08-28 08:24:17 +00:00
Stopwatch watch ;
2020-06-17 10:34:23 +00:00
MergeTreeData & src_data = dest_table_storage - > checkStructureAndGetMergeTreeData ( * this , metadata_snapshot , dest_metadata_snapshot ) ;
2020-01-15 11:55:20 +00:00
auto src_data_id = src_data . getStorageID ( ) ;
2020-04-13 15:21:05 +00:00
String partition_id = getPartitionIDFromQuery ( partition , query_context ) ;
2019-08-28 08:24:17 +00:00
2021-08-23 17:50:50 +00:00
/// A range for log entry to remove parts from the source table (myself).
auto zookeeper = getZooKeeper ( ) ;
2021-11-03 07:46:33 +00:00
/// Retry if alter_partition_version changes
for ( size_t retry = 0 ; retry < 1000 ; + + retry )
{
String alter_partition_version_path = zookeeper_path + " /alter_partition_version " ;
Coordination : : Stat alter_partition_version_stat ;
zookeeper - > get ( alter_partition_version_path , & alter_partition_version_stat ) ;
MergeTreePartInfo drop_range ;
std : : optional < EphemeralLockInZooKeeper > delimiting_block_lock ;
getFakePartCoveringAllPartsInPartition ( partition_id , drop_range , delimiting_block_lock , true ) ;
String drop_range_fake_part_name = getPartNamePossiblyFake ( format_version , drop_range ) ;
DataPartPtr covering_part ;
DataPartsVector src_all_parts ;
{
/// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet.
auto parts_lock = src_data . lockParts ( ) ;
src_all_parts = src_data . getActivePartsToReplace ( drop_range , drop_range_fake_part_name , covering_part , parts_lock ) ;
}
if ( covering_part )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Got part {} covering drop range {}, it's a bug " ,
covering_part - > name , drop_range_fake_part_name ) ;
/// After allocating block number for drop_range we must ensure that it does not intersect block numbers
/// allocated by concurrent REPLACE query.
/// We could check it in multi-request atomically with creation of DROP_RANGE entry in source table log,
/// but it's better to check it here and fail as early as possible (before we have done something to destination table).
Coordination : : Error version_check_code = zookeeper - > trySet ( alter_partition_version_path , " " , alter_partition_version_stat . version ) ;
if ( version_check_code ! = Coordination : : Error : : ZOK )
throw Exception ( ErrorCodes : : CANNOT_ASSIGN_ALTER , " Cannot DROP PARTITION in {} after copying partition to {}, "
" because another ALTER PARTITION query was concurrently executed " ,
getStorageID ( ) . getFullTableName ( ) , dest_table_storage - > getStorageID ( ) . getFullTableName ( ) ) ;
DataPartsVector src_parts ;
MutableDataPartsVector dst_parts ;
Strings block_id_paths ;
Strings part_checksums ;
std : : vector < EphemeralLockInZooKeeper > ephemeral_locks ;
LOG_DEBUG ( log , " Cloning {} parts " , src_all_parts . size ( ) ) ;
static const String TMP_PREFIX = " tmp_move_from_ " ;
/// Clone parts into destination table.
String dest_alter_partition_version_path = dest_table_storage - > zookeeper_path + " /alter_partition_version " ;
Coordination : : Stat dest_alter_partition_version_stat ;
zookeeper - > get ( dest_alter_partition_version_path , & dest_alter_partition_version_stat ) ;
for ( const auto & src_part : src_all_parts )
{
if ( ! dest_table_storage - > canReplacePartition ( src_part ) )
throw Exception (
" Cannot move partition ' " + partition_id + " ' because part ' " + src_part - > name + " ' has inconsistent granularity with table " ,
ErrorCodes : : LOGICAL_ERROR ) ;
String hash_hex = src_part - > checksums . getTotalChecksumHex ( ) ;
String block_id_path ;
auto lock = dest_table_storage - > allocateBlockNumber ( partition_id , zookeeper , block_id_path ) ;
if ( ! lock )
{
LOG_INFO ( log , " Part {} (hash {}) has been already attached " , src_part - > name , hash_hex ) ;
continue ;
}
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
UInt64 index = lock - > getNumber ( ) ;
MergeTreePartInfo dst_part_info ( partition_id , index , index , src_part - > info . level ) ;
auto dst_part = dest_table_storage - > cloneAndLoadDataPartOnSameDisk ( src_part , TMP_PREFIX , dst_part_info , dest_metadata_snapshot ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
src_parts . emplace_back ( src_part ) ;
dst_parts . emplace_back ( dst_part ) ;
ephemeral_locks . emplace_back ( std : : move ( * lock ) ) ;
block_id_paths . emplace_back ( block_id_path ) ;
part_checksums . emplace_back ( hash_hex ) ;
}
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
ReplicatedMergeTreeLogEntryData entry_delete ;
2019-08-28 08:24:17 +00:00
{
2021-11-03 07:46:33 +00:00
entry_delete . type = LogEntry : : DROP_RANGE ;
entry_delete . source_replica = replica_name ;
entry_delete . new_part_name = drop_range_fake_part_name ;
entry_delete . detach = false ; //-V1048
entry_delete . create_time = time ( nullptr ) ;
2019-08-28 08:24:17 +00:00
}
2021-11-03 07:46:33 +00:00
ReplicatedMergeTreeLogEntryData entry ;
{
MergeTreePartInfo drop_range_dest = makeDummyDropRangeForMovePartitionOrAttachPartitionFrom ( partition_id ) ;
2019-08-30 04:24:05 +00:00
2021-11-03 07:46:33 +00:00
entry . type = ReplicatedMergeTreeLogEntryData : : REPLACE_RANGE ;
entry . source_replica = dest_table_storage - > replica_name ;
entry . create_time = time ( nullptr ) ;
entry . replace_range_entry = std : : make_shared < ReplicatedMergeTreeLogEntryData : : ReplaceRangeEntry > ( ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
auto & entry_replace = * entry . replace_range_entry ;
entry_replace . drop_range_part_name = getPartNamePossiblyFake ( format_version , drop_range_dest ) ;
entry_replace . from_database = src_data_id . database_name ;
entry_replace . from_table = src_data_id . table_name ;
for ( const auto & part : src_parts )
entry_replace . src_part_names . emplace_back ( part - > name ) ;
for ( const auto & part : dst_parts )
entry_replace . new_part_names . emplace_back ( part - > name ) ;
for ( const String & checksum : part_checksums )
entry_replace . part_names_checksums . emplace_back ( checksum ) ;
entry_replace . columns_version = - 1 ;
}
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
clearBlocksInPartition ( * zookeeper , drop_range . partition_id , drop_range . max_block , drop_range . max_block ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
DataPartsVector parts_to_remove ;
Coordination : : Responses op_results ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
try
2019-08-28 08:24:17 +00:00
{
2021-11-03 07:46:33 +00:00
Coordination : : Requests ops ;
for ( size_t i = 0 ; i < dst_parts . size ( ) ; + + i )
2019-08-28 08:24:17 +00:00
{
2021-11-03 07:46:33 +00:00
dest_table_storage - > getCommitPartOps ( ops , dst_parts [ i ] , block_id_paths [ i ] ) ;
ephemeral_locks [ i ] . getUnlockOps ( ops ) ;
2019-08-28 08:24:17 +00:00
}
2021-11-03 07:46:33 +00:00
/// Check and update version to avoid race with DROP_RANGE
ops . emplace_back ( zkutil : : makeSetRequest ( dest_alter_partition_version_path , " " , dest_alter_partition_version_stat . version ) ) ;
/// Just update version, because merges assignment relies on it
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( dest_table_storage - > zookeeper_path ) / " log " , " " , - 1 ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest ( fs : : path ( dest_table_storage - > zookeeper_path ) / " log/log- " ,
entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
{
Transaction transaction ( * dest_table_storage ) ;
2019-10-04 04:37:19 +00:00
2021-11-03 07:46:33 +00:00
auto src_data_parts_lock = lockParts ( ) ;
auto dest_data_parts_lock = dest_table_storage - > lockParts ( ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
std : : mutex mutex ;
DataPartsLock lock ( mutex ) ;
2019-10-22 07:15:58 +00:00
2021-11-03 07:46:33 +00:00
for ( MutableDataPartPtr & part : dst_parts )
dest_table_storage - > renameTempPartAndReplace ( part , nullptr , & transaction , lock ) ;
2019-08-30 04:24:05 +00:00
2021-11-03 07:46:33 +00:00
Coordination : : Error code = zookeeper - > tryMulti ( ops , op_results ) ;
if ( code = = Coordination : : Error : : ZBADVERSION )
continue ;
else
zkutil : : KeeperMultiException : : check ( code , ops , op_results ) ;
parts_to_remove = removePartsInRangeFromWorkingSet ( drop_range , true , lock ) ;
transaction . commit ( & lock ) ;
}
2019-10-03 10:52:32 +00:00
2021-11-03 07:46:33 +00:00
PartLog : : addNewParts ( getContext ( ) , dst_parts , watch . elapsed ( ) ) ;
}
catch ( . . . )
{
PartLog : : addNewParts ( getContext ( ) , dst_parts , watch . elapsed ( ) , ExecutionStatus : : fromCurrentException ( ) ) ;
throw ;
2019-08-28 08:24:17 +00:00
}
2021-11-03 07:46:33 +00:00
String log_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * op_results . back ( ) ) . path_created ;
entry . znode_name = log_znode_path . substr ( log_znode_path . find_last_of ( ' / ' ) + 1 ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
for ( auto & lock : ephemeral_locks )
lock . assumeUnlocked ( ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
tryRemovePartsFromZooKeeperWithRetries ( parts_to_remove ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
parts_to_remove . clear ( ) ;
cleanup_thread . wakeup ( ) ;
lock2 . reset ( ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
dest_table_storage - > waitForLogEntryToBeProcessedIfNecessary ( entry , query_context ) ;
/// Create DROP_RANGE for the source table
Coordination : : Requests ops_src ;
ops_src . emplace_back ( zkutil : : makeCreateRequest (
fs : : path ( zookeeper_path ) / " log/log- " , entry_delete . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
/// Just update version, because merges assignment relies on it
ops_src . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " log " , " " , - 1 ) ) ;
delimiting_block_lock - > getUnlockOps ( ops_src ) ;
2019-08-28 08:24:17 +00:00
2021-11-03 07:46:33 +00:00
op_results = zookeeper - > multi ( ops_src ) ;
2019-08-30 04:24:05 +00:00
2021-11-03 07:46:33 +00:00
log_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * op_results . front ( ) ) . path_created ;
entry_delete . znode_name = log_znode_path . substr ( log_znode_path . find_last_of ( ' / ' ) + 1 ) ;
2019-08-30 04:24:05 +00:00
2021-11-03 07:46:33 +00:00
lock1 . reset ( ) ;
waitForLogEntryToBeProcessedIfNecessary ( entry_delete , query_context ) ;
2019-08-30 04:35:56 +00:00
2021-11-03 07:46:33 +00:00
/// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper.
cleanLastPartNode ( partition_id ) ;
2019-07-26 08:42:17 +00:00
2021-11-03 07:46:33 +00:00
return ;
}
2020-04-16 18:47:20 +00:00
2021-11-03 07:46:33 +00:00
throw Exception ( ErrorCodes : : CANNOT_ASSIGN_ALTER , " Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed " ) ;
2019-08-30 04:35:56 +00:00
}
2019-08-30 04:24:05 +00:00
2020-11-24 14:24:48 +00:00
void StorageReplicatedMergeTree : : movePartitionToShard (
2021-04-20 12:26:05 +00:00
const ASTPtr & partition , bool move_part , const String & to , ContextPtr /*query_context*/ )
2020-11-24 14:24:48 +00:00
{
/// This is a lightweight operation that only optimistically checks if it could succeed and queues tasks.
if ( ! move_part )
throw Exception ( " MOVE PARTITION TO SHARD is not supported, use MOVE PART instead " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2021-10-28 16:19:41 +00:00
if ( normalizeZooKeeperPath ( zookeeper_path , /* check_starts_with_slash */ true ) = = normalizeZooKeeperPath ( to , /* check_starts_with_slash */ true ) )
2020-11-24 14:24:48 +00:00
throw Exception ( " Source and destination are the same " , ErrorCodes : : BAD_ARGUMENTS ) ;
auto zookeeper = getZooKeeper ( ) ;
String part_name = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
auto part = getPartIfExists ( part_info , { MergeTreeDataPartState : : Committed } ) ;
if ( ! part )
throw Exception ( ErrorCodes : : NO_SUCH_DATA_PART , " Part {} not found locally " , part_name ) ;
if ( part - > uuid = = UUIDHelpers : : Nil )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Part {} does not have an uuid assigned and it can't be moved between shards " , part_name ) ;
ReplicatedMergeTreeMergePredicate merge_pred = queue . getMergePredicate ( zookeeper ) ;
/// The following block is pretty much copy & paste from StorageReplicatedMergeTree::dropPart to avoid conflicts while this is WIP.
/// Extract it to a common method and re-use it before merging.
{
if ( partIsLastQuorumPart ( part - > info ) )
{
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Part {} is last inserted part with quorum in partition. Would not be able to drop " , part_name ) ;
}
/// canMergeSinglePart is overlapping with dropPart, let's try to use the same code.
String out_reason ;
if ( ! merge_pred . canMergeSinglePart ( part , & out_reason ) )
throw Exception ( ErrorCodes : : PART_IS_TEMPORARILY_LOCKED , " Part is busy, reason: " + out_reason ) ;
}
{
/// Optimistic check that for compatible destination table structure.
checkTableStructure ( to , getInMemoryMetadataPtr ( ) ) ;
}
PinnedPartUUIDs src_pins ;
PinnedPartUUIDs dst_pins ;
{
String s = zookeeper - > get ( zookeeper_path + " /pinned_part_uuids " , & src_pins . stat ) ;
src_pins . fromString ( s ) ;
}
{
String s = zookeeper - > get ( to + " /pinned_part_uuids " , & dst_pins . stat ) ;
dst_pins . fromString ( s ) ;
}
if ( src_pins . part_uuids . contains ( part - > uuid ) | | dst_pins . part_uuids . contains ( part - > uuid ) )
throw Exception ( ErrorCodes : : PART_IS_TEMPORARILY_LOCKED , " Part {} has it's uuid ({}) already pinned . " , part_name, toString(part->uuid)) ;
src_pins . part_uuids . insert ( part - > uuid ) ;
dst_pins . part_uuids . insert ( part - > uuid ) ;
PartMovesBetweenShardsOrchestrator : : Entry part_move_entry ;
2021-09-16 16:03:31 +00:00
part_move_entry . state = PartMovesBetweenShardsOrchestrator : : EntryState : : SYNC_SOURCE ;
2020-11-24 14:24:48 +00:00
part_move_entry . create_time = std : : time ( nullptr ) ;
part_move_entry . update_time = part_move_entry . create_time ;
part_move_entry . task_uuid = UUIDHelpers : : generateV4 ( ) ;
part_move_entry . part_name = part - > name ;
part_move_entry . part_uuid = part - > uuid ;
part_move_entry . to_shard = to ;
Coordination : : Requests ops ;
ops . emplace_back ( zkutil : : makeCheckRequest ( zookeeper_path + " /log " , merge_pred . getVersion ( ) ) ) ; /// Make sure no new events were added to the log.
ops . emplace_back ( zkutil : : makeSetRequest ( zookeeper_path + " /pinned_part_uuids " , src_pins . toString ( ) , src_pins . stat . version ) ) ;
ops . emplace_back ( zkutil : : makeSetRequest ( to + " /pinned_part_uuids " , dst_pins . toString ( ) , dst_pins . stat . version ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest (
part_moves_between_shards_orchestrator . entries_znode_path + " /task- " ,
part_move_entry . toString ( ) ,
zkutil : : CreateMode : : PersistentSequential ) ) ;
Coordination : : Responses responses ;
Coordination : : Error rc = zookeeper - > tryMulti ( ops , responses ) ;
zkutil : : KeeperMultiException : : check ( rc , ops , responses ) ;
String task_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * responses . back ( ) ) . path_created ;
LOG_DEBUG ( log , " Created task for part movement between shards at " + task_znode_path ) ;
2021-09-16 16:03:31 +00:00
/// TODO(nv): Nice to have support for `replication_alter_partitions_sync`.
/// For now use the system.part_moves_between_shards table for status.
}
2020-11-24 14:24:48 +00:00
2021-09-16 16:03:31 +00:00
CancellationCode StorageReplicatedMergeTree : : killPartMoveToShard ( const UUID & task_uuid )
{
return part_moves_between_shards_orchestrator . killPartMoveToShard ( task_uuid ) ;
2020-11-24 14:24:48 +00:00
}
2018-05-21 13:49:54 +00:00
void StorageReplicatedMergeTree : : getCommitPartOps (
2018-08-25 01:58:14 +00:00
Coordination : : Requests & ops ,
2019-05-03 02:00:57 +00:00
MutableDataPartPtr & part ,
2018-05-21 13:49:54 +00:00
const String & block_id_path ) const
{
const String & part_name = part - > name ;
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
2018-05-21 13:49:54 +00:00
if ( ! block_id_path . empty ( ) )
{
/// Make final duplicate check and commit block_id
ops . emplace_back (
zkutil : : makeCreateRequest (
block_id_path ,
part_name , /// We will be able to know original part number for duplicate blocks, if we want.
zkutil : : CreateMode : : Persistent ) ) ;
}
2019-05-03 02:00:57 +00:00
/// Information about the part, in the replica
2019-08-26 18:08:58 +00:00
if ( storage_settings_ptr - > use_minimalistic_part_header_in_zookeeper )
2018-12-11 13:30:20 +00:00
{
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " parts " / part - > name ,
2020-01-16 16:15:01 +00:00
ReplicatedMergeTreePartHeader : : fromColumnsAndChecksums ( part - > getColumns ( ) , part - > checksums ) . toString ( ) ,
2018-12-11 13:30:20 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
}
else
{
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " parts " / part - > name ,
2018-12-11 13:30:20 +00:00
" " ,
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " parts " / part - > name / " columns " ,
2020-01-16 16:15:01 +00:00
part - > getColumns ( ) . toString ( ) ,
2018-12-11 13:30:20 +00:00
zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeCreateRequest (
2021-05-08 10:59:55 +00:00
fs : : path ( replica_path ) / " parts " / part - > name / " checksums " ,
2018-12-11 13:30:20 +00:00
getChecksumsForZooKeeper ( part - > checksums ) ,
zkutil : : CreateMode : : Persistent ) ) ;
}
}
2018-04-17 17:59:42 +00:00
ReplicatedMergeTreeAddress StorageReplicatedMergeTree : : getReplicatedMergeTreeAddress ( ) const
{
2021-04-10 23:33:54 +00:00
auto host_port = getContext ( ) - > getInterserverIOAddress ( ) ;
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID ( ) ;
2018-04-17 17:59:42 +00:00
ReplicatedMergeTreeAddress res ;
res . host = host_port . first ;
res . replication_port = host_port . second ;
2021-04-10 23:33:54 +00:00
res . queries_port = getContext ( ) - > getTCPPort ( ) ;
2019-12-03 16:25:32 +00:00
res . database = table_id . database_name ;
res . table = table_id . table_name ;
2021-04-10 23:33:54 +00:00
res . scheme = getContext ( ) - > getInterserverScheme ( ) ;
2018-04-17 17:59:42 +00:00
return res ;
}
2018-05-28 15:37:30 +00:00
ActionLock StorageReplicatedMergeTree : : getActionLock ( StorageActionBlockType action_type )
2018-05-21 13:49:54 +00:00
{
if ( action_type = = ActionLocks : : PartsMerge )
2019-08-01 15:36:12 +00:00
return merger_mutator . merges_blocker . cancel ( ) ;
if ( action_type = = ActionLocks : : PartsTTLMerge )
return merger_mutator . ttl_merges_blocker . cancel ( ) ;
2018-05-21 13:49:54 +00:00
if ( action_type = = ActionLocks : : PartsFetch )
return fetcher . blocker . cancel ( ) ;
if ( action_type = = ActionLocks : : PartsSend )
2020-10-21 19:24:16 +00:00
{
auto data_parts_exchange_ptr = std : : atomic_load ( & data_parts_exchange_endpoint ) ;
return data_parts_exchange_ptr ? data_parts_exchange_ptr - > blocker . cancel ( ) : ActionLock ( ) ;
}
2018-05-21 13:49:54 +00:00
if ( action_type = = ActionLocks : : ReplicationQueue )
2018-05-28 15:37:30 +00:00
return queue . actions_blocker . cancel ( ) ;
2018-05-21 13:49:54 +00:00
2019-09-03 14:50:49 +00:00
if ( action_type = = ActionLocks : : PartsMove )
return parts_mover . moves_blocker . cancel ( ) ;
2018-05-21 13:49:54 +00:00
return { } ;
}
2020-10-15 16:10:22 +00:00
void StorageReplicatedMergeTree : : onActionLockRemove ( StorageActionBlockType action_type )
{
if ( action_type = = ActionLocks : : PartsMerge | | action_type = = ActionLocks : : PartsTTLMerge
| | action_type = = ActionLocks : : PartsFetch | | action_type = = ActionLocks : : PartsSend
| | action_type = = ActionLocks : : ReplicationQueue )
2021-09-08 00:21:21 +00:00
background_operations_assignee . trigger ( ) ;
2020-10-15 16:10:22 +00:00
else if ( action_type = = ActionLocks : : PartsMove )
2021-09-08 00:21:21 +00:00
background_moves_assignee . trigger ( ) ;
2020-10-15 16:10:22 +00:00
}
2018-06-09 15:48:22 +00:00
2018-05-21 13:49:54 +00:00
bool StorageReplicatedMergeTree : : waitForShrinkingQueueSize ( size_t queue_size , UInt64 max_wait_milliseconds )
{
2019-09-19 11:04:57 +00:00
Stopwatch watch ;
2018-05-21 13:49:54 +00:00
/// Let's fetch new log entries firstly
2021-08-18 09:49:22 +00:00
queue . pullLogsToQueue ( getZooKeeper ( ) , { } , ReplicatedMergeTreeQueue : : SYNC ) ;
2020-06-15 22:54:19 +00:00
2020-10-15 08:29:18 +00:00
/// This is significant, because the execution of this task could be delayed at BackgroundPool.
/// And we force it to be executed.
2021-09-08 00:21:21 +00:00
background_operations_assignee . trigger ( ) ;
2018-05-21 13:49:54 +00:00
2019-09-19 11:04:57 +00:00
Poco : : Event target_size_event ;
auto callback = [ & target_size_event , queue_size ] ( size_t new_queue_size )
2018-05-21 13:49:54 +00:00
{
if ( new_queue_size < = queue_size )
2019-09-19 11:04:57 +00:00
target_size_event . set ( ) ;
2018-05-21 13:49:54 +00:00
} ;
2019-09-19 11:04:57 +00:00
const auto handler = queue . addSubscriber ( std : : move ( callback ) ) ;
2018-05-21 13:49:54 +00:00
2019-09-19 11:04:57 +00:00
while ( ! target_size_event . tryWait ( 50 ) )
2018-05-21 13:49:54 +00:00
{
if ( max_wait_milliseconds & & watch . elapsedMilliseconds ( ) > max_wait_milliseconds )
2019-09-19 11:04:57 +00:00
return false ;
2018-05-21 13:49:54 +00:00
2018-07-30 18:30:33 +00:00
if ( partial_shutdown_called )
2018-05-21 13:49:54 +00:00
throw Exception ( " Shutdown is called for table " , ErrorCodes : : ABORTED ) ;
}
2019-09-19 11:04:57 +00:00
return true ;
2018-04-02 12:45:55 +00:00
}
2017-11-15 16:32:47 +00:00
2021-05-27 23:10:44 +00:00
bool StorageReplicatedMergeTree : : dropPartImpl (
2020-11-12 17:36:02 +00:00
zkutil : : ZooKeeperPtr & zookeeper , String part_name , LogEntry & entry , bool detach , bool throw_if_noop )
2020-09-04 15:48:51 +00:00
{
LOG_TRACE ( log , " Will try to insert a log entry to DROP_RANGE for part: " + part_name ) ;
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
while ( true )
{
ReplicatedMergeTreeMergePredicate merge_pred = queue . getMergePredicate ( zookeeper ) ;
2020-11-02 17:30:53 +00:00
auto part = getPartIfExists ( part_info , { MergeTreeDataPartState : : Committed } ) ;
2020-09-04 15:48:51 +00:00
2020-11-02 17:30:53 +00:00
if ( ! part )
2020-11-12 17:36:02 +00:00
{
if ( throw_if_noop )
throw Exception ( " Part " + part_name + " not found locally, won't try to drop it. " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
return false ;
}
2020-09-04 15:48:51 +00:00
2021-07-05 19:58:36 +00:00
if ( merge_pred . hasDropRange ( part - > info ) )
{
if ( throw_if_noop )
throw Exception ( " Already has DROP RANGE for part " + part_name + " in queue. " , ErrorCodes : : PART_IS_TEMPORARILY_LOCKED ) ;
return false ;
}
2020-09-04 15:48:51 +00:00
/// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already
/// finished the merge.
2020-11-02 17:30:53 +00:00
if ( partIsAssignedToBackgroundOperation ( part ) )
2020-11-12 17:36:02 +00:00
{
if ( throw_if_noop )
throw Exception ( " Part " + part_name
+ " is currently participating in a background operation (mutation/merge) "
+ " , try again later " , ErrorCodes : : PART_IS_TEMPORARILY_LOCKED ) ;
return false ;
}
2020-11-03 09:24:10 +00:00
if ( partIsLastQuorumPart ( part - > info ) )
2020-11-12 17:36:02 +00:00
{
if ( throw_if_noop )
throw Exception ( " Part " + part_name + " is last inserted part with quorum in partition. Cannot drop " ,
ErrorCodes : : NOT_IMPLEMENTED ) ;
return false ;
}
2018-06-09 14:24:50 +00:00
2020-11-03 09:24:10 +00:00
if ( partIsInsertingWithParallelQuorum ( part - > info ) )
2020-11-12 17:36:02 +00:00
{
if ( throw_if_noop )
throw Exception ( " Part " + part_name + " is inserting with parallel quorum. Cannot drop " ,
ErrorCodes : : NOT_IMPLEMENTED ) ;
return false ;
}
2020-09-04 15:48:51 +00:00
2020-11-02 17:30:53 +00:00
Coordination : : Requests ops ;
getClearBlocksInPartitionOps ( ops , * zookeeper , part_info . partition_id , part_info . min_block , part_info . max_block ) ;
2021-01-12 10:55:02 +00:00
size_t clear_block_ops_size = ops . size ( ) ;
2020-11-02 17:30:53 +00:00
2020-09-04 15:48:51 +00:00
/// If `part_name` is result of a recent merge and source parts are still available then
/// DROP_RANGE with detach will move this part together with source parts to `detached/` dir.
entry . type = LogEntry : : DROP_RANGE ;
entry . source_replica = replica_name ;
2021-06-23 19:24:43 +00:00
/// We don't set fake drop level (999999999) for the single part DROP_RANGE.
/// First of all we don't guarantee anything other than the part will not be
/// active after DROP PART, but covering part (without data of dropped part) can exist.
/// If we add part with 9999999 level than we can break invariant in virtual_parts of
/// the queue.
entry . new_part_name = getPartNamePossiblyFake ( format_version , part - > info ) ;
2020-09-04 15:48:51 +00:00
entry . detach = detach ;
entry . create_time = time ( nullptr ) ;
2021-05-08 10:59:55 +00:00
ops . emplace_back ( zkutil : : makeCheckRequest ( fs : : path ( zookeeper_path ) / " log " , merge_pred . getVersion ( ) ) ) ; /// Make sure no new events were added to the log.
ops . emplace_back ( zkutil : : makeCreateRequest ( fs : : path ( zookeeper_path ) / " log/log- " , entry . toString ( ) , zkutil : : CreateMode : : PersistentSequential ) ) ;
2021-05-14 16:11:40 +00:00
/// Just update version, because merges assignment relies on it
2021-05-17 11:23:59 +00:00
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " log " , " " , - 1 ) ) ;
2020-09-04 15:48:51 +00:00
Coordination : : Responses responses ;
Coordination : : Error rc = zookeeper - > tryMulti ( ops , responses ) ;
if ( rc = = Coordination : : Error : : ZBADVERSION )
{
LOG_TRACE ( log , " A new log entry appeared while trying to commit DROP RANGE. Retry. " ) ;
continue ;
}
2021-01-14 08:07:13 +00:00
else if ( rc = = Coordination : : Error : : ZNONODE )
{
LOG_TRACE ( log , " Other replica already removing same part {} or part deduplication node was removed by background thread. Retry. " , part_name ) ;
continue ;
}
2020-09-04 15:48:51 +00:00
else
zkutil : : KeeperMultiException : : check ( rc , ops , responses ) ;
2021-01-12 10:55:02 +00:00
String log_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * responses [ clear_block_ops_size + 1 ] ) . path_created ;
2020-09-04 15:48:51 +00:00
entry . znode_name = log_znode_path . substr ( log_znode_path . find_last_of ( ' / ' ) + 1 ) ;
return true ;
}
}
2018-06-09 14:24:50 +00:00
2020-09-04 15:48:51 +00:00
bool StorageReplicatedMergeTree : : dropAllPartsInPartition (
2021-04-10 23:33:54 +00:00
zkutil : : ZooKeeper & zookeeper , String & partition_id , LogEntry & entry , ContextPtr query_context , bool detach )
2018-04-21 00:35:20 +00:00
{
2021-11-03 07:46:33 +00:00
/// Retry if alter_partition_version changes
for ( size_t retry = 0 ; retry < 1000 ; + + retry )
{
String alter_partition_version_path = zookeeper_path + " /alter_partition_version " ;
Coordination : : Stat alter_partition_version_stat ;
zookeeper . get ( alter_partition_version_path , & alter_partition_version_stat ) ;
2021-05-17 19:23:38 +00:00
2021-11-03 07:46:33 +00:00
MergeTreePartInfo drop_range_info ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
/// It would prevent other replicas from assigning merges which intersect locked block number.
std : : optional < EphemeralLockInZooKeeper > delimiting_block_lock ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
if ( ! getFakePartCoveringAllPartsInPartition ( partition_id , drop_range_info , delimiting_block_lock ) )
{
LOG_INFO ( log , " Will not drop partition {}, it is empty. " , partition_id ) ;
return false ;
}
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
clearBlocksInPartition ( zookeeper , partition_id , drop_range_info . min_block , drop_range_info . max_block ) ;
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
String drop_range_fake_part_name = getPartNamePossiblyFake ( format_version , drop_range_info ) ;
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
LOG_DEBUG ( log , " Disabled merges covered by range {} " , drop_range_fake_part_name ) ;
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
/// Finally, having achieved the necessary invariants, you can put an entry in the log.
entry . type = LogEntry : : DROP_RANGE ;
entry . source_replica = replica_name ;
entry . new_part_name = drop_range_fake_part_name ;
entry . detach = detach ;
entry . create_time = time ( nullptr ) ;
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
Coordination : : Requests ops ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
ops . emplace_back ( zkutil : : makeCreateRequest ( fs : : path ( zookeeper_path ) / " log/log- " , entry . toString ( ) ,
zkutil : : CreateMode : : PersistentSequential ) ) ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
/// Check and update version to avoid race with REPLACE_RANGE.
/// Otherwise new parts covered by drop_range_info may appear after execution of current DROP_RANGE entry
/// as a result of execution of concurrently created REPLACE_RANGE entry.
ops . emplace_back ( zkutil : : makeSetRequest ( alter_partition_version_path , " " , alter_partition_version_stat . version ) ) ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
/// Just update version, because merges assignment relies on it
ops . emplace_back ( zkutil : : makeSetRequest ( fs : : path ( zookeeper_path ) / " log " , " " , - 1 ) ) ;
delimiting_block_lock - > getUnlockOps ( ops ) ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
if ( auto txn = query_context - > getZooKeeperMetadataTransaction ( ) )
txn - > moveOpsTo ( ops ) ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
Coordination : : Responses responses ;
Coordination : : Error code = zookeeper . tryMulti ( ops , responses ) ;
2021-06-20 08:24:43 +00:00
2021-11-03 07:46:33 +00:00
if ( code = = Coordination : : Error : : ZOK )
delimiting_block_lock - > assumeUnlocked ( ) ;
else if ( code = = Coordination : : Error : : ZBADVERSION )
2021-11-09 10:02:17 +00:00
{
/// Cannot retry automatically, because some zookeeper ops were lost on the first attempt. Will retry on DDLWorker-level.
if ( query_context - > getZooKeeperMetadataTransaction ( ) )
throw Exception (
2021-11-09 10:19:00 +00:00
" Cannot execute alter, because alter partition version was suddenly changed due to concurrent alter " ,
2021-11-09 10:02:17 +00:00
ErrorCodes : : CANNOT_ASSIGN_ALTER ) ;
2021-11-03 07:46:33 +00:00
continue ;
2021-11-09 10:02:17 +00:00
}
2021-11-03 07:46:33 +00:00
else
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
2020-06-12 18:24:32 +00:00
2021-11-03 07:46:33 +00:00
String log_znode_path = dynamic_cast < const Coordination : : CreateResponse & > ( * responses . front ( ) ) . path_created ;
entry . znode_name = log_znode_path . substr ( log_znode_path . find_last_of ( ' / ' ) + 1 ) ;
2018-04-21 00:35:20 +00:00
2021-11-03 07:46:33 +00:00
getContext ( ) - > getMergeList ( ) . cancelInPartition ( getStorageID ( ) , partition_id , drop_range_info . max_block ) ;
2021-06-24 14:07:43 +00:00
2021-11-03 07:46:33 +00:00
return true ;
}
throw Exception ( ErrorCodes : : CANNOT_ASSIGN_ALTER ,
" Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed " ) ;
2018-05-21 13:49:54 +00:00
}
2019-07-03 13:17:19 +00:00
2021-04-10 23:33:54 +00:00
CheckResults StorageReplicatedMergeTree : : checkData ( const ASTPtr & query , ContextPtr local_context )
2019-07-03 13:17:19 +00:00
{
CheckResults results ;
DataPartsVector data_parts ;
if ( const auto & check_query = query - > as < ASTCheckQuery & > ( ) ; check_query . partition )
{
2021-04-10 23:33:54 +00:00
String partition_id = getPartitionIDFromQuery ( check_query . partition , local_context ) ;
2019-07-03 13:17:19 +00:00
data_parts = getDataPartsVectorInPartition ( MergeTreeDataPartState : : Committed , partition_id ) ;
}
else
data_parts = getDataPartsVector ( ) ;
for ( auto & part : data_parts )
{
try
{
results . push_back ( part_check_thread . checkPart ( part - > name ) ) ;
}
2019-07-09 09:02:52 +00:00
catch ( const Exception & ex )
2019-07-03 13:17:19 +00:00
{
2019-07-09 09:02:52 +00:00
results . emplace_back ( part - > name , false , " Check of part finished with error: ' " + ex . message ( ) + " ' " ) ;
2019-07-03 13:17:19 +00:00
}
}
return results ;
}
2021-02-26 09:48:57 +00:00
2019-08-12 13:30:29 +00:00
bool StorageReplicatedMergeTree : : canUseAdaptiveGranularity ( ) const
{
2019-08-26 18:08:58 +00:00
const auto storage_settings_ptr = getSettings ( ) ;
return storage_settings_ptr - > index_granularity_bytes ! = 0 & &
( storage_settings_ptr - > enable_mixed_granularity_parts | |
2019-08-12 13:30:29 +00:00
( ! has_non_adaptive_index_granularity_parts & & ! other_replicas_fixed_granularity ) ) ;
}
2020-11-28 08:17:20 +00:00
MutationCommands StorageReplicatedMergeTree : : getFirstAlterMutationCommandsForPart ( const DataPartPtr & part ) const
2020-02-01 11:47:09 +00:00
{
2020-03-24 17:05:38 +00:00
return queue . getFirstAlterMutationCommandsForPart ( part ) ;
2020-02-01 11:47:09 +00:00
}
2020-06-23 16:40:58 +00:00
2021-02-26 09:48:57 +00:00
2020-06-23 16:40:58 +00:00
void StorageReplicatedMergeTree : : startBackgroundMovesIfNeeded ( )
{
2020-10-20 11:27:50 +00:00
if ( areBackgroundMovesNeeded ( ) )
2021-09-08 00:21:21 +00:00
background_moves_assignee . start ( ) ;
2020-06-23 16:40:58 +00:00
}
2021-07-05 12:44:58 +00:00
std : : unique_ptr < MergeTreeSettings > StorageReplicatedMergeTree : : getDefaultSettings ( ) const
{
return std : : make_unique < MergeTreeSettings > ( getContext ( ) - > getReplicatedMergeTreeSettings ( ) ) ;
}
2021-02-26 09:48:57 +00:00
void StorageReplicatedMergeTree : : lockSharedData ( const IMergeTreeDataPart & part ) const
2021-01-14 16:26:56 +00:00
{
2021-02-26 09:48:57 +00:00
if ( ! part . volume )
return ;
DiskPtr disk = part . volume - > getDisk ( ) ;
2021-07-05 03:32:56 +00:00
if ( ! disk | | ! disk - > supportZeroCopyReplication ( ) )
2021-02-26 09:48:57 +00:00
return ;
2021-08-24 22:24:47 +00:00
String zero_copy = fmt : : format ( " zero_copy_{} " , toString ( disk - > getType ( ) ) ) ;
2021-02-26 09:48:57 +00:00
zkutil : : ZooKeeperPtr zookeeper = tryGetZooKeeper ( ) ;
if ( ! zookeeper )
return ;
String id = part . getUniqueId ( ) ;
boost : : replace_all ( id , " / " , " _ " ) ;
2021-06-24 08:25:05 +00:00
String zookeeper_node = fs : : path ( zookeeper_path ) / zero_copy / " shared " / part . name / id / replica_name ;
2021-02-26 09:48:57 +00:00
LOG_TRACE ( log , " Set zookeeper lock {} " , zookeeper_node ) ;
/// In rare case other replica can remove path between createAncestors and createIfNotExists
/// So we make up to 5 attempts
for ( int attempts = 5 ; attempts > 0 ; - - attempts )
{
try
{
zookeeper - > createAncestors ( zookeeper_node ) ;
zookeeper - > createIfNotExists ( zookeeper_node , " lock " ) ;
break ;
}
catch ( const zkutil : : KeeperException & e )
{
if ( e . code = = Coordination : : Error : : ZNONODE )
continue ;
throw ;
}
}
}
bool StorageReplicatedMergeTree : : unlockSharedData ( const IMergeTreeDataPart & part ) const
{
if ( ! part . volume )
return true ;
DiskPtr disk = part . volume - > getDisk ( ) ;
2021-07-05 03:32:56 +00:00
if ( ! disk | | ! disk - > supportZeroCopyReplication ( ) )
2021-02-26 09:48:57 +00:00
return true ;
2021-08-24 22:24:47 +00:00
String zero_copy = fmt : : format ( " zero_copy_{} " , toString ( disk - > getType ( ) ) ) ;
2021-02-26 09:48:57 +00:00
zkutil : : ZooKeeperPtr zookeeper = tryGetZooKeeper ( ) ;
if ( ! zookeeper )
return true ;
String id = part . getUniqueId ( ) ;
boost : : replace_all ( id , " / " , " _ " ) ;
2021-06-24 08:25:05 +00:00
String zookeeper_part_node = fs : : path ( zookeeper_path ) / zero_copy / " shared " / part . name ;
2021-05-08 10:59:55 +00:00
String zookeeper_part_uniq_node = fs : : path ( zookeeper_part_node ) / id ;
String zookeeper_node = fs : : path ( zookeeper_part_uniq_node ) / replica_name ;
2021-02-26 09:48:57 +00:00
LOG_TRACE ( log , " Remove zookeeper lock {} " , zookeeper_node ) ;
zookeeper - > tryRemove ( zookeeper_node ) ;
Strings children ;
zookeeper - > tryGetChildren ( zookeeper_part_uniq_node , children ) ;
if ( ! children . empty ( ) )
{
LOG_TRACE ( log , " Found zookeper locks for {} " , zookeeper_part_uniq_node ) ;
return false ;
}
zookeeper - > tryRemove ( zookeeper_part_uniq_node ) ;
/// Even when we have lock with same part name, but with different uniq, we can remove files on S3
children . clear ( ) ;
zookeeper - > tryGetChildren ( zookeeper_part_node , children ) ;
if ( children . empty ( ) )
/// Cleanup after last uniq removing
zookeeper - > tryRemove ( zookeeper_part_node ) ;
return true ;
}
bool StorageReplicatedMergeTree : : tryToFetchIfShared (
const IMergeTreeDataPart & part ,
const DiskPtr & disk ,
2021-03-05 17:24:06 +00:00
const String & path )
2021-02-26 09:48:57 +00:00
{
2021-06-24 08:25:05 +00:00
const auto settings = getSettings ( ) ;
auto disk_type = disk - > getType ( ) ;
2021-07-05 03:32:56 +00:00
if ( ! ( disk - > supportZeroCopyReplication ( ) & & settings - > allow_remote_fs_zero_copy_replication ) )
2021-02-26 09:48:57 +00:00
return false ;
2021-06-24 08:25:05 +00:00
String replica = getSharedDataReplica ( part , disk_type ) ;
2021-02-26 09:48:57 +00:00
2021-06-24 08:25:05 +00:00
/// We can't fetch part when none replicas have this part on a same type remote disk
2021-02-26 09:48:57 +00:00
if ( replica . empty ( ) )
return false ;
2021-03-05 17:24:06 +00:00
return executeFetchShared ( replica , part . name , disk , path ) ;
2021-02-26 09:48:57 +00:00
}
String StorageReplicatedMergeTree : : getSharedDataReplica (
2021-08-24 22:24:47 +00:00
const IMergeTreeDataPart & part , DiskType disk_type ) const
2021-02-26 09:48:57 +00:00
{
String best_replica ;
zkutil : : ZooKeeperPtr zookeeper = tryGetZooKeeper ( ) ;
if ( ! zookeeper )
return best_replica ;
2021-08-24 22:24:47 +00:00
String zero_copy = fmt : : format ( " zero_copy_{} " , toString ( disk_type ) ) ;
2021-06-24 08:25:05 +00:00
String zookeeper_part_node = fs : : path ( zookeeper_path ) / zero_copy / " shared " / part . name ;
2021-02-26 09:48:57 +00:00
Strings ids ;
zookeeper - > tryGetChildren ( zookeeper_part_node , ids ) ;
Strings replicas ;
for ( const auto & id : ids )
{
2021-05-08 10:59:55 +00:00
String zookeeper_part_uniq_node = fs : : path ( zookeeper_part_node ) / id ;
2021-03-04 23:10:20 +00:00
Strings id_replicas ;
zookeeper - > tryGetChildren ( zookeeper_part_uniq_node , id_replicas ) ;
LOG_TRACE ( log , " Found zookeper replicas for {}: {} " , zookeeper_part_uniq_node , id_replicas . size ( ) ) ;
replicas . insert ( replicas . end ( ) , id_replicas . begin ( ) , id_replicas . end ( ) ) ;
2021-02-26 09:48:57 +00:00
}
LOG_TRACE ( log , " Found zookeper replicas for part {}: {} " , part . name , replicas . size ( ) ) ;
Strings active_replicas ;
/// TODO: Move best replica choose in common method (here is the same code as in StorageReplicatedMergeTree::fetchPartition)
/// Leave only active replicas.
active_replicas . reserve ( replicas . size ( ) ) ;
for ( const String & replica : replicas )
2021-05-08 10:59:55 +00:00
if ( ( replica ! = replica_name ) & & ( zookeeper - > exists ( fs : : path ( zookeeper_path ) / " replicas " / replica / " is_active " ) ) )
2021-02-26 09:48:57 +00:00
active_replicas . push_back ( replica ) ;
LOG_TRACE ( log , " Found zookeper active replicas for part {}: {} " , part . name , active_replicas . size ( ) ) ;
if ( active_replicas . empty ( ) )
return best_replica ;
/** You must select the best (most relevant) replica.
* This is a replica with the maximum ` log_pointer ` , then with the minimum ` queue ` size .
* NOTE This is not exactly the best criteria . It does not make sense to download old partitions ,
* and it would be nice to be able to choose the replica closest by network .
* NOTE Of course , there are data races here . You can solve it by retrying .
*/
Int64 max_log_pointer = - 1 ;
UInt64 min_queue_size = std : : numeric_limits < UInt64 > : : max ( ) ;
for ( const String & replica : active_replicas )
{
2021-05-08 10:59:55 +00:00
String current_replica_path = fs : : path ( zookeeper_path ) / " replicas " / replica ;
2021-02-26 09:48:57 +00:00
2021-05-08 10:59:55 +00:00
String log_pointer_str = zookeeper - > get ( fs : : path ( current_replica_path ) / " log_pointer " ) ;
2021-02-26 09:48:57 +00:00
Int64 log_pointer = log_pointer_str . empty ( ) ? 0 : parse < UInt64 > ( log_pointer_str ) ;
Coordination : : Stat stat ;
2021-05-08 10:59:55 +00:00
zookeeper - > get ( fs : : path ( current_replica_path ) / " queue " , & stat ) ;
2021-02-26 09:48:57 +00:00
size_t queue_size = stat . numChildren ;
if ( log_pointer > max_log_pointer
| | ( log_pointer = = max_log_pointer & & queue_size < min_queue_size ) )
{
max_log_pointer = log_pointer ;
min_queue_size = queue_size ;
best_replica = replica ;
}
}
return best_replica ;
2021-01-14 16:26:56 +00:00
}
2021-04-13 04:40:33 +00:00
String StorageReplicatedMergeTree : : findReplicaHavingPart (
const String & part_name , const String & zookeeper_path_ , zkutil : : ZooKeeper : : Ptr zookeeper_ )
{
2021-05-08 10:59:55 +00:00
Strings replicas = zookeeper_ - > getChildren ( fs : : path ( zookeeper_path_ ) / " replicas " ) ;
2021-04-13 04:40:33 +00:00
/// Select replicas in uniformly random order.
std : : shuffle ( replicas . begin ( ) , replicas . end ( ) , thread_local_rng ) ;
for ( const String & replica : replicas )
{
2021-05-08 10:59:55 +00:00
if ( zookeeper_ - > exists ( fs : : path ( zookeeper_path_ ) / " replicas " / replica / " parts " / part_name )
& & zookeeper_ - > exists ( fs : : path ( zookeeper_path_ ) / " replicas " / replica / " is_active " ) )
return fs : : path ( zookeeper_path_ ) / " replicas " / replica ;
2021-04-13 04:40:33 +00:00
}
return { } ;
}
2021-04-14 02:05:41 +00:00
bool StorageReplicatedMergeTree : : checkIfDetachedPartExists ( const String & part_name )
2021-04-13 04:40:33 +00:00
{
2021-05-08 10:59:55 +00:00
fs : : directory_iterator dir_end ;
2021-04-13 04:40:33 +00:00
for ( const std : : string & path : getDataPaths ( ) )
2021-05-08 10:59:55 +00:00
for ( fs : : directory_iterator dir_it { fs : : path ( path ) / " detached/ " } ; dir_it ! = dir_end ; + + dir_it )
if ( dir_it - > path ( ) . filename ( ) . string ( ) = = part_name )
2021-04-13 04:40:33 +00:00
return true ;
return false ;
}
2021-04-14 02:05:41 +00:00
bool StorageReplicatedMergeTree : : checkIfDetachedPartitionExists ( const String & partition_name )
2021-04-13 04:40:33 +00:00
{
2021-05-08 10:59:55 +00:00
fs : : directory_iterator dir_end ;
2021-08-24 12:57:49 +00:00
2021-04-13 04:40:33 +00:00
for ( const std : : string & path : getDataPaths ( ) )
{
2021-05-08 10:59:55 +00:00
for ( fs : : directory_iterator dir_it { fs : : path ( path ) / " detached/ " } ; dir_it ! = dir_end ; + + dir_it )
2021-04-13 04:40:33 +00:00
{
2021-08-24 12:57:49 +00:00
const String file_name = dir_it - > path ( ) . filename ( ) . string ( ) ;
auto part_info = MergeTreePartInfo : : tryParsePartName ( file_name , format_version ) ;
if ( part_info & & part_info - > partition_id = = partition_name )
2021-04-13 04:40:33 +00:00
return true ;
}
}
return false ;
}
2021-06-29 15:14:44 +00:00
2021-06-30 15:24:51 +00:00
bool StorageReplicatedMergeTree : : createEmptyPartInsteadOfLost ( zkutil : : ZooKeeperPtr zookeeper , const String & lost_part_name )
2021-06-29 15:14:44 +00:00
{
LOG_INFO ( log , " Going to replace lost part {} with empty part " , lost_part_name ) ;
auto metadata_snapshot = getInMemoryMetadataPtr ( ) ;
2021-06-29 19:47:54 +00:00
auto settings = getSettings ( ) ;
2021-06-29 15:14:44 +00:00
constexpr static auto TMP_PREFIX = " tmp_empty_ " ;
auto new_part_info = MergeTreePartInfo : : fromPartName ( lost_part_name , format_version ) ;
auto block = metadata_snapshot - > getSampleBlock ( ) ;
DB : : IMergeTreeDataPart : : TTLInfos move_ttl_infos ;
NamesAndTypesList columns = metadata_snapshot - > getColumns ( ) . getAllPhysical ( ) . filter ( block . getNames ( ) ) ;
ReservationPtr reservation = reserveSpacePreferringTTLRules ( metadata_snapshot , 0 , move_ttl_infos , time ( nullptr ) , 0 , true ) ;
VolumePtr volume = getStoragePolicy ( ) - > getVolume ( 0 ) ;
2021-09-16 21:19:58 +00:00
auto minmax_idx = std : : make_shared < IMergeTreeDataPart : : MinMaxIndex > ( ) ;
minmax_idx - > update ( block , getMinMaxColumnsNames ( metadata_snapshot - > getPartitionKey ( ) ) ) ;
2021-06-29 15:14:44 +00:00
auto new_data_part = createPart (
lost_part_name ,
choosePartType ( 0 , block . rows ( ) ) ,
new_part_info ,
createVolumeFromReservation ( reservation , volume ) ,
TMP_PREFIX + lost_part_name ) ;
2021-06-29 19:47:54 +00:00
if ( settings - > assign_part_uuids )
2021-06-29 15:14:44 +00:00
new_data_part - > uuid = UUIDHelpers : : generateV4 ( ) ;
new_data_part - > setColumns ( columns ) ;
new_data_part - > rows_count = block . rows ( ) ;
{
2021-06-30 19:41:25 +00:00
auto lock = lockParts ( ) ;
auto parts_in_partition = getDataPartsPartitionRange ( new_part_info . partition_id ) ;
if ( parts_in_partition . empty ( ) )
{
LOG_WARNING ( log , " Empty part {} is not created instead of lost part because there are no parts in partition {} (it's empty), resolve this manually using DROP PARTITION. " , lost_part_name , new_part_info . partition_id ) ;
return false ;
}
new_data_part - > partition = ( * parts_in_partition . begin ( ) ) - > partition ;
2021-06-29 15:14:44 +00:00
}
new_data_part - > minmax_idx = std : : move ( minmax_idx ) ;
new_data_part - > is_temp = true ;
SyncGuardPtr sync_guard ;
if ( new_data_part - > isStoredOnDisk ( ) )
{
/// The name could be non-unique in case of stale files from previous runs.
String full_path = new_data_part - > getFullRelativePath ( ) ;
if ( new_data_part - > volume - > getDisk ( ) - > exists ( full_path ) )
{
LOG_WARNING ( log , " Removing old temporary directory {} " , fullPath ( new_data_part - > volume - > getDisk ( ) , full_path ) ) ;
new_data_part - > volume - > getDisk ( ) - > removeRecursive ( full_path ) ;
}
const auto disk = new_data_part - > volume - > getDisk ( ) ;
disk - > createDirectories ( full_path ) ;
if ( getSettings ( ) - > fsync_part_directory )
sync_guard = disk - > getDirectorySyncGuard ( full_path ) ;
}
/// This effectively chooses minimal compression method:
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
auto compression_codec = getContext ( ) - > chooseCompressionCodec ( 0 , 0 ) ;
const auto & index_factory = MergeTreeIndexFactory : : instance ( ) ;
MergedBlockOutputStream out ( new_data_part , metadata_snapshot , columns , index_factory . getMany ( metadata_snapshot - > getSecondaryIndices ( ) ) , compression_codec ) ;
2021-06-29 19:47:54 +00:00
bool sync_on_insert = settings - > fsync_after_insert ;
2021-06-29 15:14:44 +00:00
out . write ( block ) ;
2021-08-26 11:01:15 +00:00
/// TODO(ab): What projections should we add to the empty part? How can we make sure that it
/// won't block future merges? Perhaps we should also check part emptiness when selecting parts
/// to merge.
2021-06-29 15:14:44 +00:00
out . writeSuffixAndFinalizePart ( new_data_part , sync_on_insert ) ;
try
{
MergeTreeData : : Transaction transaction ( * this ) ;
2021-07-02 09:29:45 +00:00
auto replaced_parts = renameTempPartAndReplace ( new_data_part , nullptr , & transaction ) ;
if ( ! replaced_parts . empty ( ) )
{
Strings part_names ;
for ( const auto & part : replaced_parts )
part_names . emplace_back ( part - > name ) ;
2021-07-02 13:38:46 +00:00
/// Why this exception is not a LOGICAL_ERROR? Because it's possible
/// to have some source parts for the lost part if replica currently
/// cloning from another replica, but source replica lost covering
/// part and finished MERGE_PARTS before clone. It's an extremely
/// rare case and it's unclear how to resolve it better. Eventually
/// source replica will replace lost part with empty part and we
/// will fetch this empty part instead of our source parts. This
/// will make replicas consistent, but some data will be lost.
throw Exception ( ErrorCodes : : INCORRECT_DATA , " Tried to create empty part {}, but it replaces existing parts {}. " , lost_part_name , fmt : : join ( part_names , " , " ) ) ;
2021-07-02 09:29:45 +00:00
}
2021-06-29 15:14:44 +00:00
2021-06-30 15:24:51 +00:00
while ( true )
{
Coordination : : Requests ops ;
Coordination : : Stat replicas_stat ;
auto replicas_path = fs : : path ( zookeeper_path ) / " replicas " ;
Strings replicas = zookeeper - > getChildren ( replicas_path , & replicas_stat ) ;
/// In rare cases new replica can appear during check
ops . emplace_back ( zkutil : : makeCheckRequest ( replicas_path , replicas_stat . version ) ) ;
for ( const String & replica : replicas )
{
String current_part_path = fs : : path ( zookeeper_path ) / " replicas " / replica / " parts " / lost_part_name ;
/// We must be sure that this part doesn't exist on other replicas
if ( ! zookeeper - > exists ( current_part_path ) )
{
ops . emplace_back ( zkutil : : makeCreateRequest ( current_part_path , " " , zkutil : : CreateMode : : Persistent ) ) ;
ops . emplace_back ( zkutil : : makeRemoveRequest ( current_part_path , - 1 ) ) ;
}
else
{
throw Exception ( ErrorCodes : : DUPLICATE_DATA_PART , " Part {} already exists on replica {} on path {} " , lost_part_name , replica , current_part_path ) ;
}
}
getCommitPartOps ( ops , new_data_part ) ;
Coordination : : Responses responses ;
if ( auto code = zookeeper - > tryMulti ( ops , responses ) ; code = = Coordination : : Error : : ZOK )
{
transaction . commit ( ) ;
break ;
}
else if ( code = = Coordination : : Error : : ZBADVERSION )
{
LOG_INFO ( log , " Looks like new replica appearead while creating new empty part, will retry " ) ;
}
else
{
zkutil : : KeeperMultiException : : check ( code , ops , responses ) ;
}
}
2021-06-29 15:14:44 +00:00
}
catch ( const Exception & ex )
{
LOG_WARNING ( log , " Cannot commit empty part {} with error {} " , lost_part_name , ex . displayText ( ) ) ;
return false ;
}
LOG_INFO ( log , " Created empty part {} instead of lost part " , lost_part_name ) ;
return true ;
}
2014-03-21 13:42:14 +00:00
}