2014-03-21 13:42:14 +00:00
|
|
|
#pragma once
|
|
|
|
|
2021-12-17 11:03:20 +00:00
|
|
|
#include <base/UUID.h>
|
2017-03-31 14:46:48 +00:00
|
|
|
#include <atomic>
|
2017-09-09 23:17:38 +00:00
|
|
|
#include <pcg_random.hpp>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2018-04-20 16:18:16 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
|
2019-08-19 17:59:16 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreePartsMover.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeQueue.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
2020-09-18 10:57:33 +00:00
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h>
|
2018-11-02 15:39:19 +00:00
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
2018-07-04 16:31:21 +00:00
|
|
|
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/DataPartsExchange.h>
|
2018-04-17 17:59:42 +00:00
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
|
2020-11-24 14:24:48 +00:00
|
|
|
#include <Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h>
|
2021-09-16 21:19:58 +00:00
|
|
|
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
|
|
|
|
#include <Storages/MergeTree/MergeFromLogEntryTask.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-12-28 17:11:52 +00:00
|
|
|
#include <Interpreters/Cluster.h>
|
2018-05-25 19:44:14 +00:00
|
|
|
#include <Interpreters/PartLog.h>
|
2017-09-08 23:31:18 +00:00
|
|
|
#include <Common/randomSeed.h>
|
2017-06-19 20:06:35 +00:00
|
|
|
#include <Common/ZooKeeper/ZooKeeper.h>
|
2021-05-26 20:37:44 +00:00
|
|
|
#include <Common/Throttler.h>
|
2018-08-20 15:34:37 +00:00
|
|
|
#include <Core/BackgroundSchedulePool.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2021-09-06 12:01:16 +00:00
|
|
|
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
|
2015-03-16 19:24:57 +00:00
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-06-20 08:24:43 +00:00
|
|
|
/** The engine that uses the merge tree (see MergeTreeData) and is replicated through ZooKeeper.
|
2015-09-20 05:21:43 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* ZooKeeper is used for the following things:
|
2020-02-14 13:17:50 +00:00
|
|
|
* - the structure of the table (/metadata, /columns)
|
2017-04-16 15:00:33 +00:00
|
|
|
* - action log with data (/log/log-...,/replicas/replica_name/queue/queue-...);
|
|
|
|
* - a replica list (/replicas), and replica activity tag (/replicas/replica_name/is_active), replica addresses (/replicas/replica_name/host);
|
2021-01-12 18:46:03 +00:00
|
|
|
* - the leader replica election (/leader_election) - these are the replicas that assign merges, mutations
|
|
|
|
* and partition manipulations.
|
2020-12-16 03:03:43 +00:00
|
|
|
* (after ClickHouse version 20.5 we allow multiple leaders to act concurrently);
|
2017-04-16 15:00:33 +00:00
|
|
|
* - a set of parts of data on each replica (/replicas/replica_name/parts);
|
|
|
|
* - list of the last N blocks of data with checksum, for deduplication (/blocks);
|
|
|
|
* - the list of incremental block numbers (/block_numbers) that we are about to insert,
|
|
|
|
* to ensure the linear order of data insertion and data merge only on the intervals in this sequence;
|
2021-01-12 18:46:03 +00:00
|
|
|
* - coordinate writes with quorum (/quorum).
|
2018-06-05 14:55:35 +00:00
|
|
|
* - Storage of mutation entries (ALTER DELETE, ALTER UPDATE etc.) to execute (/mutations).
|
|
|
|
* See comments in StorageReplicatedMergeTree::mutate() for details.
|
2014-03-21 19:17:59 +00:00
|
|
|
*/
|
2015-09-20 05:21:43 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** The replicated tables have a common log (/log/log-...).
|
|
|
|
* Log - a sequence of entries (LogEntry) about what to do.
|
|
|
|
* Each entry is one of:
|
|
|
|
* - normal data insertion (GET),
|
2021-06-20 08:24:43 +00:00
|
|
|
* - data insertion with a possible attach from local data (ATTACH),
|
2017-04-16 15:00:33 +00:00
|
|
|
* - merge (MERGE),
|
|
|
|
* - delete the partition (DROP).
|
2015-09-20 05:21:43 +00:00
|
|
|
*
|
2018-06-05 14:55:35 +00:00
|
|
|
* Each replica copies (queueUpdatingTask, pullLogsToQueue) entries from the log to its queue (/replicas/replica_name/queue/queue-...)
|
2017-04-16 15:00:33 +00:00
|
|
|
* and then executes them (queueTask).
|
|
|
|
* Despite the name of the "queue", execution can be reordered, if necessary (shouldExecuteLogEntry, executeLogEntry).
|
|
|
|
* In addition, the records in the queue can be generated independently (not from the log), in the following cases:
|
|
|
|
* - when creating a new replica, actions are put on GET from other replicas (createReplica);
|
2021-06-20 08:24:43 +00:00
|
|
|
* - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check
|
|
|
|
* (at start - checkParts, while running - searchForMissingPart), actions are put on GET from other replicas;
|
2015-09-20 05:21:43 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* The replica to which INSERT was made in the queue will also have an entry of the GET of this data.
|
|
|
|
* Such an entry is considered to be executed as soon as the queue handler sees it.
|
2015-09-20 05:21:43 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* The log entry has a creation time. This time is generated by the clock of server that created entry
|
|
|
|
* - the one on which the corresponding INSERT or ALTER query came.
|
2015-09-20 05:21:43 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* For the entries in the queue that the replica made for itself,
|
|
|
|
* as the time will take the time of creation the appropriate part on any of the replicas.
|
2015-09-20 05:21:43 +00:00
|
|
|
*/
|
|
|
|
|
2022-05-03 06:43:28 +00:00
|
|
|
class StorageReplicatedMergeTree final : public MergeTreeData
|
2014-03-21 13:42:14 +00:00
|
|
|
{
|
|
|
|
public:
|
2022-04-19 20:47:29 +00:00
|
|
|
enum RenamingRestrictions
|
|
|
|
{
|
|
|
|
ALLOW_ANY,
|
|
|
|
ALLOW_PRESERVING_UUID,
|
|
|
|
DO_NOT_ALLOW,
|
|
|
|
};
|
|
|
|
|
|
|
|
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
|
|
|
|
*/
|
|
|
|
StorageReplicatedMergeTree(
|
|
|
|
const String & zookeeper_path_,
|
|
|
|
const String & replica_name_,
|
|
|
|
bool attach,
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const String & relative_data_path_,
|
|
|
|
const StorageInMemoryMetadata & metadata_,
|
|
|
|
ContextMutablePtr context_,
|
|
|
|
const String & date_column_name,
|
|
|
|
const MergingParams & merging_params_,
|
|
|
|
std::unique_ptr<MergeTreeSettings> settings_,
|
|
|
|
bool has_force_restore_data_flag,
|
|
|
|
RenamingRestrictions renaming_restrictions_);
|
|
|
|
|
2017-06-06 17:06:14 +00:00
|
|
|
void startup() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
void shutdown() override;
|
2021-12-27 15:54:28 +00:00
|
|
|
void flush() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
~StorageReplicatedMergeTree() override;
|
|
|
|
|
2022-05-29 19:53:56 +00:00
|
|
|
static String getDefaultZooKeeperPath(const Poco::Util::AbstractConfiguration & config);
|
|
|
|
static String getDefaultReplicaName(const Poco::Util::AbstractConfiguration & config);
|
|
|
|
|
2019-05-03 02:00:57 +00:00
|
|
|
std::string getName() const override { return "Replicated" + merging_params.getModeName() + "MergeTree"; }
|
2019-03-29 20:31:06 +00:00
|
|
|
|
2019-12-13 03:09:22 +00:00
|
|
|
bool supportsParallelInsert() const override { return true; }
|
2017-04-25 15:21:03 +00:00
|
|
|
bool supportsReplication() const override { return true; }
|
2018-05-21 23:17:57 +00:00
|
|
|
bool supportsDeduplication() const override { return true; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-10-01 17:34:22 +00:00
|
|
|
void read(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-11-10 12:02:22 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2018-04-19 14:47:09 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2019-02-18 23:38:44 +00:00
|
|
|
size_t max_block_size,
|
2017-06-02 15:54:39 +00:00
|
|
|
unsigned num_streams) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
2021-04-10 23:33:54 +00:00
|
|
|
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override;
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> totalBytes(const Settings & settings) const override;
|
2019-10-28 17:27:43 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-27 20:13:16 +00:00
|
|
|
bool optimize(
|
|
|
|
const ASTPtr & query,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
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) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-10-25 17:49:49 +00:00
|
|
|
void alter(const AlterCommands & commands, ContextPtr query_context, AlterLockHolder & table_lock_holder) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void mutate(const MutationCommands & commands, ContextPtr context) override;
|
2020-03-09 01:22:33 +00:00
|
|
|
void waitMutation(const String & znode_name, size_t mutations_sync) const;
|
2019-05-03 02:00:57 +00:00
|
|
|
std::vector<MergeTreeMutationStatus> getMutationsStatus() const override;
|
2019-02-04 13:04:02 +00:00
|
|
|
CancellationCode killMutation(const String & mutation_id) override;
|
2018-06-07 13:28:39 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2020-01-22 11:30:11 +00:00
|
|
|
void drop() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr query_context, TableExclusiveLockHolder &) override;
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2022-04-13 14:51:59 +00:00
|
|
|
void checkTableCanBeRenamed(const StorageID & new_name) const override;
|
2020-09-26 19:18:28 +00:00
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
bool supportsIndexForIn() const override { return true; }
|
|
|
|
|
2018-08-03 09:39:01 +00:00
|
|
|
void checkTableCanBeDropped() const override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-05-28 15:37:30 +00:00
|
|
|
ActionLock getActionLock(StorageActionBlockType action_type) override;
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-10-15 16:10:22 +00:00
|
|
|
void onActionLockRemove(StorageActionBlockType action_type) override;
|
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Wait when replication queue size becomes less or equal than queue_size
|
|
|
|
/// If timeout is exceeded returns false
|
|
|
|
bool waitForShrinkingQueueSize(size_t queue_size = 0, UInt64 max_wait_milliseconds = 0);
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** For the system table replicas. */
|
2017-04-01 07:20:54 +00:00
|
|
|
struct Status
|
|
|
|
{
|
|
|
|
bool is_leader;
|
2019-02-13 13:05:58 +00:00
|
|
|
bool can_become_leader;
|
2017-04-01 07:20:54 +00:00
|
|
|
bool is_readonly;
|
|
|
|
bool is_session_expired;
|
|
|
|
ReplicatedMergeTreeQueue::Status queue;
|
|
|
|
UInt32 parts_to_check;
|
|
|
|
String zookeeper_path;
|
|
|
|
String replica_name;
|
|
|
|
String replica_path;
|
|
|
|
Int32 columns_version;
|
|
|
|
UInt64 log_max_index;
|
|
|
|
UInt64 log_pointer;
|
2017-04-17 15:06:12 +00:00
|
|
|
UInt64 absolute_delay;
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt8 total_replicas;
|
|
|
|
UInt8 active_replicas;
|
2021-07-27 14:35:20 +00:00
|
|
|
String last_queue_update_exception;
|
2020-02-15 00:11:09 +00:00
|
|
|
/// If the error has happened fetching the info from ZooKeeper, this field will be set.
|
|
|
|
String zookeeper_exception;
|
2021-08-04 13:19:42 +00:00
|
|
|
std::unordered_map<std::string, bool> replica_is_active;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK.
|
2017-04-01 07:20:54 +00:00
|
|
|
void getStatus(Status & res, bool with_zk_fields = true);
|
|
|
|
|
|
|
|
using LogEntriesData = std::vector<ReplicatedMergeTreeLogEntryData>;
|
|
|
|
void getQueue(LogEntriesData & res, String & replica_name);
|
|
|
|
|
2020-11-24 14:24:48 +00:00
|
|
|
std::vector<PartMovesBetweenShardsOrchestrator::Entry> getPartMovesBetweenShardsEntries();
|
|
|
|
|
2017-04-17 15:06:12 +00:00
|
|
|
/// Get replica delay relative to current time.
|
|
|
|
time_t getAbsoluteDelay() const;
|
|
|
|
|
2020-06-19 14:18:58 +00:00
|
|
|
/// If the absolute delay is greater than min_relative_delay_to_measure,
|
2017-04-17 15:06:12 +00:00
|
|
|
/// will also calculate the difference from the unprocessed time of the best replica.
|
|
|
|
/// NOTE: Will communicate to ZooKeeper to calculate relative delay.
|
2017-04-01 07:20:54 +00:00
|
|
|
void getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay);
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Add a part to the queue of parts whose data you want to check in the background thread.
|
2022-04-12 12:14:26 +00:00
|
|
|
void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0);
|
2016-04-09 03:50:02 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
2019-07-03 13:17:19 +00:00
|
|
|
|
2019-08-12 13:30:29 +00:00
|
|
|
/// Checks ability to use granularity
|
|
|
|
bool canUseAdaptiveGranularity() const override;
|
|
|
|
|
2022-05-29 19:53:56 +00:00
|
|
|
/// Returns the default path to the table in ZooKeeper.
|
|
|
|
/// It's used if not set in engine's arguments while creating a replicated table.
|
|
|
|
static String getDefaultReplicaPath(const ContextPtr & context_);
|
|
|
|
|
|
|
|
/// Returns the default replica name in ZooKeeper.
|
|
|
|
/// It's used if not set in engine's arguments while creating a replicated table.
|
|
|
|
static String getDefaultReplicaName(const ContextPtr & context_);
|
|
|
|
|
2020-02-17 12:47:34 +00:00
|
|
|
int getMetadataVersion() const { return metadata_version; }
|
2020-02-05 11:18:11 +00:00
|
|
|
|
2022-06-22 22:56:41 +00:00
|
|
|
/// Modify a CREATE TABLE query to make a variant which must be written to a backup.
|
2022-06-23 10:17:54 +00:00
|
|
|
void adjustCreateQueryForBackup(ASTPtr & create_query) const override;
|
2022-05-29 19:53:56 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Makes backup entries to backup the data of the storage.
|
|
|
|
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
|
|
|
|
|
|
|
/// Extract data from the backup and put it to the storage.
|
|
|
|
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
|
|
|
|
2020-06-22 15:01:40 +00:00
|
|
|
/** Remove a specific replica from zookeeper.
|
2020-06-05 07:03:51 +00:00
|
|
|
*/
|
2022-04-05 15:36:53 +00:00
|
|
|
static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica,
|
|
|
|
Poco::Logger * logger, MergeTreeSettingsPtr table_settings = nullptr);
|
2020-06-05 07:03:51 +00:00
|
|
|
|
2021-04-19 08:21:42 +00:00
|
|
|
/// Removes table from ZooKeeper after the last replica was dropped
|
|
|
|
static bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path,
|
2021-04-19 10:40:20 +00:00
|
|
|
const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger);
|
2021-04-19 08:21:42 +00:00
|
|
|
|
2021-06-21 13:36:21 +00:00
|
|
|
/// Schedules job to execute in background pool (merge, mutate, drop range and so on)
|
2021-09-08 00:21:21 +00:00
|
|
|
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
2019-10-28 17:27:43 +00:00
|
|
|
|
2020-11-09 09:14:20 +00:00
|
|
|
/// Checks that fetches are not disabled with action blocker and pool for fetches
|
|
|
|
/// is not overloaded
|
2020-10-26 11:02:47 +00:00
|
|
|
bool canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const;
|
|
|
|
|
2021-01-14 16:26:56 +00:00
|
|
|
/// Fetch part only when it stored on shared storage like S3
|
2022-04-22 16:58:09 +00:00
|
|
|
DataPartStoragePtr 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-07-05 03:32:56 +00:00
|
|
|
/// Lock part in zookeeper for use shared data in several nodes
|
2022-04-20 22:30:13 +00:00
|
|
|
void lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional<HardlinkedFiles> hardlinked_files) const override;
|
2022-02-14 09:20:27 +00:00
|
|
|
|
|
|
|
void lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const;
|
2021-02-26 09:48:57 +00:00
|
|
|
|
2021-07-05 03:32:56 +00:00
|
|
|
/// Unlock shared data part in zookeeper
|
2021-02-26 09:48:57 +00:00
|
|
|
/// Return true if data unlocked
|
|
|
|
/// Return false if data is still used by another node
|
2022-04-18 23:09:09 +00:00
|
|
|
std::pair<bool, NameSet> unlockSharedData(const IMergeTreeDataPart & part) const override;
|
2021-12-01 13:11:26 +00:00
|
|
|
|
2021-11-23 13:57:24 +00:00
|
|
|
/// Unlock shared data part in zookeeper by part id
|
|
|
|
/// Return true if data unlocked
|
|
|
|
/// Return false if data is still used by another node
|
2022-04-18 23:09:09 +00:00
|
|
|
static std::pair<bool, NameSet> unlockSharedDataByID(String part_id, const String & table_uuid, const String & part_name, const String & replica_name_,
|
2022-04-21 19:19:13 +00:00
|
|
|
std::string disk_type, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger,
|
2021-12-01 13:11:26 +00:00
|
|
|
const String & zookeeper_path_old);
|
2021-11-23 13:57:24 +00:00
|
|
|
|
2021-02-26 09:48:57 +00:00
|
|
|
/// Fetch part only if some replica has it on shared storage like S3
|
2022-04-22 16:58:09 +00:00
|
|
|
DataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override;
|
2021-02-26 09:48:57 +00:00
|
|
|
|
2021-06-24 08:25:05 +00:00
|
|
|
/// Get best replica having this partition on a same type remote disk
|
2021-08-24 22:24:47 +00:00
|
|
|
String getSharedDataReplica(const IMergeTreeDataPart & part, DiskType disk_type) const;
|
2021-02-26 09:48:57 +00:00
|
|
|
|
2021-06-20 08:24:43 +00:00
|
|
|
inline String getReplicaName() const { return replica_name; }
|
|
|
|
|
|
|
|
/// Restores table metadata if ZooKeeper lost it.
|
2021-12-30 14:27:22 +00:00
|
|
|
/// Used only on restarted readonly replicas (not checked). All active (Active) parts are moved to detached/
|
2021-06-20 08:24:43 +00:00
|
|
|
/// folder and attached. Parts in all other states are just moved to detached/ folder.
|
|
|
|
void restoreMetadataInZooKeeper();
|
|
|
|
|
2021-05-27 12:54:47 +00:00
|
|
|
/// Get throttler for replicated fetches
|
2021-05-26 20:37:44 +00:00
|
|
|
ThrottlerPtr getFetchesThrottler() const
|
|
|
|
{
|
|
|
|
return replicated_fetches_throttler;
|
|
|
|
}
|
|
|
|
|
2021-05-27 12:54:47 +00:00
|
|
|
/// Get throttler for replicated sends
|
2021-05-26 20:37:44 +00:00
|
|
|
ThrottlerPtr getSendsThrottler() const
|
|
|
|
{
|
|
|
|
return replicated_sends_throttler;
|
|
|
|
}
|
|
|
|
|
2021-06-30 15:24:51 +00:00
|
|
|
bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name);
|
2021-06-29 15:14:44 +00:00
|
|
|
|
2021-12-27 16:27:06 +00:00
|
|
|
// Return default or custom zookeeper name for table
|
2021-12-21 14:29:50 +00:00
|
|
|
String getZooKeeperName() const { return zookeeper_name; }
|
2021-11-23 13:57:24 +00:00
|
|
|
|
2022-04-19 18:15:27 +00:00
|
|
|
String getZooKeeperPath() const { return zookeeper_path; }
|
|
|
|
|
2021-12-27 16:27:06 +00:00
|
|
|
// Return table id, common for different replicas
|
2022-04-20 19:08:26 +00:00
|
|
|
String getTableSharedID() const override;
|
2021-12-17 11:03:20 +00:00
|
|
|
|
2022-06-24 19:29:38 +00:00
|
|
|
/// Returns the same as getTableSharedID(), but extracts it from a create query.
|
|
|
|
static std::optional<String> tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context);
|
|
|
|
|
2022-03-13 12:23:51 +00:00
|
|
|
static String getDefaultZooKeeperName() { return default_zookeeper_name; }
|
2021-12-15 11:30:57 +00:00
|
|
|
|
2022-01-31 20:47:04 +00:00
|
|
|
/// Check if there are new broken disks and enqueue part recovery tasks.
|
|
|
|
void checkBrokenDisks();
|
|
|
|
|
2022-06-08 12:09:59 +00:00
|
|
|
static bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
|
|
|
|
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path, ContextPtr local_context);
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
private:
|
2021-06-20 08:24:43 +00:00
|
|
|
std::atomic_bool are_restoring_replica {false};
|
|
|
|
|
2019-10-28 17:27:43 +00:00
|
|
|
/// Get a sequential consistent view of current parts.
|
|
|
|
ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const;
|
|
|
|
|
2017-12-15 18:23:05 +00:00
|
|
|
/// Delete old parts from disk and from ZooKeeper.
|
2017-10-06 11:30:57 +00:00
|
|
|
void clearOldPartsAndRemoveFromZK();
|
2017-05-24 20:19:29 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
friend class ReplicatedMergeTreeSink;
|
2017-04-01 07:20:54 +00:00
|
|
|
friend class ReplicatedMergeTreePartCheckThread;
|
|
|
|
friend class ReplicatedMergeTreeCleanupThread;
|
|
|
|
friend class ReplicatedMergeTreeAlterThread;
|
|
|
|
friend class ReplicatedMergeTreeRestartingThread;
|
2020-09-18 10:57:33 +00:00
|
|
|
friend class ReplicatedMergeTreeMergeStrategyPicker;
|
2017-04-01 07:20:54 +00:00
|
|
|
friend struct ReplicatedMergeTreeLogEntry;
|
|
|
|
friend class ScopedPartitionMergeLock;
|
2018-05-20 19:56:03 +00:00
|
|
|
friend class ReplicatedMergeTreeQueue;
|
2020-11-24 14:24:48 +00:00
|
|
|
friend class PartMovesBetweenShardsOrchestrator;
|
2018-05-21 13:49:54 +00:00
|
|
|
friend class MergeTreeData;
|
2021-09-16 21:19:58 +00:00
|
|
|
friend class MergeFromLogEntryTask;
|
|
|
|
friend class MutateFromLogEntryTask;
|
|
|
|
friend class ReplicatedMergeMutateTaskBase;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
2020-09-18 10:57:33 +00:00
|
|
|
using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker;
|
2017-04-01 07:20:54 +00:00
|
|
|
using LogEntry = ReplicatedMergeTreeLogEntry;
|
|
|
|
using LogEntryPtr = LogEntry::Ptr;
|
2014-08-05 13:49:44 +00:00
|
|
|
|
2021-02-16 13:19:21 +00:00
|
|
|
using MergeTreeData::MutableDataPartPtr;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
zkutil::ZooKeeperPtr current_zookeeper; /// Use only the methods below.
|
2019-10-28 17:27:43 +00:00
|
|
|
mutable std::mutex current_zookeeper_mutex; /// To recreate the session in the background thread.
|
2014-12-12 20:50:32 +00:00
|
|
|
|
2019-10-28 17:27:43 +00:00
|
|
|
zkutil::ZooKeeperPtr tryGetZooKeeper() const;
|
|
|
|
zkutil::ZooKeeperPtr getZooKeeper() const;
|
2022-02-03 10:10:05 +00:00
|
|
|
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
|
2020-11-16 08:27:33 +00:00
|
|
|
void setZooKeeper();
|
2014-03-22 14:44:44 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// If true, the table is offline and can not be written to it.
|
2022-02-03 10:10:05 +00:00
|
|
|
/// This flag is managed by RestartingThread.
|
|
|
|
std::atomic_bool is_readonly {true};
|
2022-01-24 09:16:13 +00:00
|
|
|
/// If nullopt - ZooKeeper is not available, so we don't know if there is table metadata.
|
2020-06-14 01:23:53 +00:00
|
|
|
/// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case.
|
2022-01-20 18:55:59 +00:00
|
|
|
std::optional<bool> has_metadata_in_zookeeper;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-16 08:27:33 +00:00
|
|
|
static constexpr auto default_zookeeper_name = "default";
|
|
|
|
String zookeeper_name;
|
2017-04-01 07:20:54 +00:00
|
|
|
String zookeeper_path;
|
|
|
|
String replica_name;
|
|
|
|
String replica_path;
|
|
|
|
|
|
|
|
/** /replicas/me/is_active.
|
|
|
|
*/
|
|
|
|
zkutil::EphemeralNodeHolderPtr replica_is_active_node;
|
|
|
|
|
2017-07-11 13:48:26 +00:00
|
|
|
/** Is this replica "leading". The leader replica selects the parts to merge.
|
2020-06-15 02:14:59 +00:00
|
|
|
* It can be false only when old ClickHouse versions are working on the same cluster, because now we allow multiple leaders.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-04-06 16:06:07 +00:00
|
|
|
std::atomic<bool> is_leader {false};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-14 14:27:48 +00:00
|
|
|
InterserverIOEndpointPtr data_parts_exchange_endpoint;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
MergeTreeDataSelectExecutor reader;
|
|
|
|
MergeTreeDataWriter writer;
|
2018-04-20 16:18:16 +00:00
|
|
|
MergeTreeDataMergerMutator merger_mutator;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-18 10:57:33 +00:00
|
|
|
MergeStrategyPicker merge_strategy_picker;
|
|
|
|
|
2017-08-25 20:41:45 +00:00
|
|
|
/** The queue of what needs to be done on this replica to catch up with everyone. It is taken from ZooKeeper (/replicas/me/queue/).
|
|
|
|
* In ZK entries in chronological order. Here it is not necessary.
|
|
|
|
*/
|
|
|
|
ReplicatedMergeTreeQueue queue;
|
|
|
|
std::atomic<time_t> last_queue_update_start_time{0};
|
|
|
|
std::atomic<time_t> last_queue_update_finish_time{0};
|
|
|
|
|
2021-08-09 12:58:23 +00:00
|
|
|
mutable std::mutex last_queue_update_exception_lock;
|
|
|
|
String last_queue_update_exception;
|
|
|
|
String getLastQueueUpdateException() const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
DataPartsExchange::Fetcher fetcher;
|
|
|
|
|
2018-03-21 23:30:20 +00:00
|
|
|
/// When activated, replica is initialized and startup() method could exit
|
|
|
|
Poco::Event startup_event;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Do I need to complete background threads (except restarting_thread)?
|
2018-07-30 18:30:33 +00:00
|
|
|
std::atomic<bool> partial_shutdown_called {false};
|
2018-07-30 20:21:45 +00:00
|
|
|
|
2018-07-30 18:30:33 +00:00
|
|
|
/// Event that is signalled (and is reset) by the restarting_thread when the ZooKeeper session expires.
|
2018-07-30 20:21:45 +00:00
|
|
|
Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-12-27 15:54:28 +00:00
|
|
|
std::atomic<bool> shutdown_called {false};
|
|
|
|
std::atomic<bool> flush_called {false};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-02-05 11:18:11 +00:00
|
|
|
int metadata_version = 0;
|
2017-11-15 16:32:47 +00:00
|
|
|
/// Threads.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-29 22:32:04 +00:00
|
|
|
/// A task that keeps track of the updates in the logs of all replicas and loads them into the queue.
|
|
|
|
bool queue_update_in_progress = false;
|
2018-05-31 13:05:05 +00:00
|
|
|
BackgroundSchedulePool::TaskHolder queue_updating_task;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-05-31 13:05:05 +00:00
|
|
|
BackgroundSchedulePool::TaskHolder mutations_updating_task;
|
2018-04-19 14:20:18 +00:00
|
|
|
|
2017-12-29 22:32:04 +00:00
|
|
|
/// A task that selects parts to merge.
|
2018-05-31 13:05:05 +00:00
|
|
|
BackgroundSchedulePool::TaskHolder merge_selecting_task;
|
2018-07-30 17:34:55 +00:00
|
|
|
/// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query.
|
|
|
|
std::mutex merge_selecting_mutex;
|
2018-04-02 12:45:55 +00:00
|
|
|
|
2018-06-21 13:27:36 +00:00
|
|
|
/// A task that marks finished mutations as done.
|
|
|
|
BackgroundSchedulePool::TaskHolder mutations_finalizing_task;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// A thread that removes old parts, log entries, and blocks.
|
2018-07-30 17:34:55 +00:00
|
|
|
ReplicatedMergeTreeCleanupThread cleanup_thread;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// A thread that checks the data of the parts, as well as the queue of the parts to be checked.
|
2017-04-01 07:20:54 +00:00
|
|
|
ReplicatedMergeTreePartCheckThread part_check_thread;
|
|
|
|
|
2018-07-30 17:34:55 +00:00
|
|
|
/// A thread that processes reconnection to ZooKeeper when the session expires.
|
2018-08-21 14:03:06 +00:00
|
|
|
ReplicatedMergeTreeRestartingThread restarting_thread;
|
2018-07-30 17:34:55 +00:00
|
|
|
|
2020-11-24 14:24:48 +00:00
|
|
|
PartMovesBetweenShardsOrchestrator part_moves_between_shards_orchestrator;
|
|
|
|
|
2019-08-12 13:30:29 +00:00
|
|
|
/// True if replica was created for existing table with fixed granularity
|
|
|
|
bool other_replicas_fixed_granularity = false;
|
|
|
|
|
2020-09-26 19:18:28 +00:00
|
|
|
/// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro
|
2022-04-13 14:51:59 +00:00
|
|
|
const RenamingRestrictions renaming_restrictions;
|
2020-09-26 19:18:28 +00:00
|
|
|
|
2020-10-26 11:02:47 +00:00
|
|
|
const size_t replicated_fetches_pool_size;
|
|
|
|
|
2021-05-27 12:54:47 +00:00
|
|
|
/// Throttlers used in DataPartsExchange to lower maximum fetch/sends
|
|
|
|
/// speed.
|
2021-05-26 20:37:44 +00:00
|
|
|
ThrottlerPtr replicated_fetches_throttler;
|
|
|
|
ThrottlerPtr replicated_sends_throttler;
|
|
|
|
|
2021-12-30 09:57:38 +00:00
|
|
|
/// Global ID, synced via ZooKeeper between replicas
|
|
|
|
UUID table_shared_id;
|
|
|
|
|
2022-01-31 20:47:04 +00:00
|
|
|
std::mutex last_broken_disks_mutex;
|
|
|
|
std::set<String> last_broken_disks;
|
|
|
|
|
2020-03-29 08:50:27 +00:00
|
|
|
template <class Func>
|
2021-12-30 14:27:22 +00:00
|
|
|
void foreachActiveParts(Func && func, bool select_sequential_consistency) const;
|
2020-03-29 08:50:27 +00:00
|
|
|
|
2020-06-11 03:24:52 +00:00
|
|
|
/** Creates the minimum set of nodes in ZooKeeper and create first replica.
|
|
|
|
* Returns true if was created, false if exists.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2020-06-16 16:55:04 +00:00
|
|
|
bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-06-20 08:24:43 +00:00
|
|
|
/**
|
|
|
|
* Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas.
|
|
|
|
*/
|
2020-06-16 16:55:04 +00:00
|
|
|
void createReplica(const StorageMetadataPtr & metadata_snapshot);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Create nodes in the ZK, which must always be, but which might not exist when older versions of the server are running.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void createNewZooKeeperNodes();
|
|
|
|
|
2020-06-16 16:55:04 +00:00
|
|
|
void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-02 11:53:05 +00:00
|
|
|
/// A part of ALTER: apply metadata changes only (data parts are altered separately).
|
2020-06-18 16:10:47 +00:00
|
|
|
/// Must be called under IStorage::lockForAlter() lock.
|
2018-11-02 15:39:19 +00:00
|
|
|
void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff);
|
2018-11-02 11:53:05 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/).
|
|
|
|
* If any parts described in ZK are not locally, throw an exception.
|
|
|
|
* If any local parts are not mentioned in ZK, remove them.
|
|
|
|
* But if there are too many, throw an exception just in case - it's probably a configuration error.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void checkParts(bool skip_sanity_checks);
|
|
|
|
|
2020-11-24 14:24:48 +00:00
|
|
|
/// Synchronize the list of part uuids which are currently pinned. These should be sent to root query executor
|
|
|
|
/// to be used for deduplication.
|
|
|
|
void syncPinnedPartUUIDs();
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Check that the part's checksum is the same as the checksum of the same part on some other replica.
|
|
|
|
* If no one has such a part, nothing checks.
|
|
|
|
* Not very reliable: if two replicas add a part almost at the same time, no checks will occur.
|
|
|
|
* Adds actions to `ops` that add data about the part into ZooKeeper.
|
2020-06-22 09:49:21 +00:00
|
|
|
* Call under lockForShare.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2019-05-03 02:00:57 +00:00
|
|
|
void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const DataPartPtr & part,
|
2018-08-25 01:58:14 +00:00
|
|
|
Coordination::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr);
|
2018-03-21 23:30:20 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const;
|
2018-03-21 23:30:20 +00:00
|
|
|
|
2021-12-30 14:27:22 +00:00
|
|
|
/// Accepts a PreActive part, atomically checks its checksums with ones on other replicas and commit the part
|
2022-04-19 13:53:10 +00:00
|
|
|
DataPartsVector checkPartChecksumsAndCommit(Transaction & transaction, const DataPartPtr & part, std::optional<HardlinkedFiles> hardlinked_files = {});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-09-05 13:12:29 +00:00
|
|
|
bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override;
|
2019-08-16 15:57:19 +00:00
|
|
|
|
2019-09-05 13:12:29 +00:00
|
|
|
void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const;
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Adds actions to `ops` that remove a part from ZooKeeper.
|
2018-12-11 13:30:20 +00:00
|
|
|
/// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes).
|
|
|
|
void removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops, bool has_children);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-01-27 10:07:18 +00:00
|
|
|
/// Just removes part from ZooKeeper using previous method
|
|
|
|
void removePartFromZooKeeper(const String & part_name);
|
|
|
|
|
2017-08-09 21:09:44 +00:00
|
|
|
/// Quickly removes big set of parts from ZooKeeper (using async multi queries)
|
2017-10-03 14:44:10 +00:00
|
|
|
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
|
2018-04-06 19:48:54 +00:00
|
|
|
NameSet * parts_should_be_retried = nullptr);
|
2017-06-01 12:53:54 +00:00
|
|
|
|
2021-03-05 09:50:26 +00:00
|
|
|
/// Remove parts from ZooKeeper, throw exception if unable to do so after max_retries.
|
|
|
|
void removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries = 5);
|
|
|
|
void removePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries = 5);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Removes a part from ZooKeeper and adds a task to the queue to download it. It is supposed to do this with broken parts.
|
2017-04-01 07:20:54 +00:00
|
|
|
void removePartAndEnqueueFetch(const String & part_name);
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Running jobs from the queue.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Execute the action from the queue. Throws an exception if something is wrong.
|
|
|
|
* Returns whether or not it succeeds. If it did not work, write it to the end of the queue.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-05-23 14:33:55 +00:00
|
|
|
bool executeLogEntry(LogEntry & entry);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-02-15 15:06:48 +00:00
|
|
|
/// Lookup the part for the entry in the detached/ folder.
|
|
|
|
/// returns nullptr if the part is corrupt or missing.
|
|
|
|
MutableDataPartPtr attachPartHelperFoundValidPart(const LogEntry& entry) const;
|
2018-05-25 19:44:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void executeDropRange(const LogEntry & entry);
|
|
|
|
|
2020-08-08 00:47:03 +00:00
|
|
|
/// Execute alter of table metadata. Set replica/metadata and replica/columns
|
2020-02-17 16:33:05 +00:00
|
|
|
/// nodes in zookeeper and also changes in memory metadata.
|
|
|
|
/// New metadata and columns values stored in entry.
|
2020-01-17 13:54:22 +00:00
|
|
|
bool executeMetadataAlter(const LogEntry & entry);
|
2020-01-13 16:39:20 +00:00
|
|
|
|
2020-02-17 16:33:05 +00:00
|
|
|
/// Fetch part from other replica (inserted or merged/mutated)
|
|
|
|
/// NOTE: Attention! First of all tries to find covering part on other replica
|
|
|
|
/// and set it into entry.actual_new_part_name. After that tries to fetch this new covering part.
|
|
|
|
/// If fetch was not successful, clears entry.actual_new_part_name.
|
2022-03-19 16:31:33 +00:00
|
|
|
bool executeFetch(LogEntry & entry, bool need_to_check_missing_part=true);
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
bool executeReplaceRange(const LogEntry & entry);
|
2020-11-24 14:24:48 +00:00
|
|
|
void executeClonePartFromShard(const LogEntry & entry);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Updates the queue.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-05-31 13:05:05 +00:00
|
|
|
void queueUpdatingTask();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-05-31 13:05:05 +00:00
|
|
|
void mutationsUpdatingTask();
|
2018-04-19 14:20:18 +00:00
|
|
|
|
2018-08-20 17:15:04 +00:00
|
|
|
/** Clone data from another replica.
|
2018-08-27 23:59:49 +00:00
|
|
|
* If replica can not be cloned throw Exception.
|
|
|
|
*/
|
2018-08-27 13:51:22 +00:00
|
|
|
void cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper);
|
2018-08-07 15:21:42 +00:00
|
|
|
|
2021-05-07 17:09:39 +00:00
|
|
|
/// Repairs metadata of staled replica. Called from cloneReplica(...)
|
|
|
|
void cloneMetadataIfNeeded(const String & source_replica, const String & source_path, zkutil::ZooKeeperPtr & zookeeper);
|
|
|
|
|
2018-08-07 15:21:42 +00:00
|
|
|
/// Clone replica if it is lost.
|
2018-08-22 14:01:54 +00:00
|
|
|
void cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper);
|
2018-08-07 15:21:42 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
ReplicatedMergeTreeQueue::SelectedEntryPtr selectQueueEntry();
|
2019-08-19 17:59:16 +00:00
|
|
|
|
|
|
|
|
2021-09-16 21:19:58 +00:00
|
|
|
MergeFromLogEntryTaskPtr getTaskToProcessMergeQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry);
|
2019-08-19 17:59:16 +00:00
|
|
|
|
2021-12-07 16:55:55 +00:00
|
|
|
/// Start being leader (if not disabled by setting).
|
|
|
|
/// Since multi-leaders are allowed, it just sets is_leader flag.
|
|
|
|
void startBeingLeader();
|
|
|
|
void stopBeingLeader();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Selects the parts to merge and writes to the log.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-05-31 13:05:05 +00:00
|
|
|
void mergeSelectingTask();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-21 13:27:36 +00:00
|
|
|
/// Checks if some mutations are done and marks them as done.
|
|
|
|
void mutationsFinalizingTask();
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Write the selected parts to merge into the log,
|
|
|
|
* Call when merge_selecting_mutex is locked.
|
|
|
|
* Returns false if any part is not in ZK.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2020-06-12 18:24:32 +00:00
|
|
|
enum class CreateMergeEntryResult { Ok, MissingPart, LogUpdated, Other };
|
|
|
|
|
|
|
|
CreateMergeEntryResult createLogEntryToMergeParts(
|
2018-05-10 15:01:10 +00:00
|
|
|
zkutil::ZooKeeperPtr & zookeeper,
|
2019-05-03 02:00:57 +00:00
|
|
|
const DataPartsVector & parts,
|
2017-04-01 07:20:54 +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,
|
2017-04-17 15:14:56 +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);
|
2020-06-12 18:24:32 +00:00
|
|
|
|
|
|
|
CreateMergeEntryResult createLogEntryToMutatePart(
|
|
|
|
const IMergeTreeDataPart & part,
|
2020-11-02 14:38:18 +00:00
|
|
|
const UUID & new_part_uuid,
|
2020-06-12 18:24:32 +00:00
|
|
|
Int64 mutation_version,
|
|
|
|
int32_t alter_version,
|
|
|
|
int32_t log_version);
|
2018-04-20 16:18:16 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Exchange parts.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
ConnectionTimeouts getFetchPartHTTPTimeouts(ContextPtr context);
|
2021-02-04 17:25:10 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Returns an empty string if no one has a part.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
String findReplicaHavingPart(const String & part_name, bool active);
|
2021-04-13 09:34:04 +00:00
|
|
|
static String findReplicaHavingPart(const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-18 10:57:33 +00:00
|
|
|
bool checkReplicaHavePart(const String & replica, const String & part_name);
|
2021-04-14 02:05:41 +00:00
|
|
|
bool checkIfDetachedPartExists(const String & part_name);
|
|
|
|
bool checkIfDetachedPartitionExists(const String & partition_name);
|
2020-09-18 10:57:33 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Find replica having specified part or any part that covers it.
|
|
|
|
* If active = true, consider only active replicas.
|
2017-05-12 13:47:42 +00:00
|
|
|
* If found, returns replica name and set 'entry->actual_new_part_name' to name of found largest covering part.
|
2017-04-01 07:20:54 +00:00
|
|
|
* If not found, returns empty string.
|
|
|
|
*/
|
2018-05-23 14:33:55 +00:00
|
|
|
String findReplicaHavingCoveringPart(LogEntry & entry, bool active);
|
2018-05-21 13:49:54 +00:00
|
|
|
String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Download the specified part from the specified replica.
|
|
|
|
* If `to_detached`, the part is placed in the `detached` directory.
|
|
|
|
* If quorum != 0, then the node for tracking the quorum is updated.
|
2017-04-01 07:20:54 +00:00
|
|
|
* Returns false if part is already fetching right now.
|
|
|
|
*/
|
2020-08-27 14:19:18 +00:00
|
|
|
bool fetchPart(
|
|
|
|
const String & part_name,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const String & replica_path,
|
|
|
|
bool to_detached,
|
|
|
|
size_t quorum,
|
2022-04-22 17:18:18 +00:00
|
|
|
zkutil::ZooKeeper::Ptr zookeeper_ = nullptr,
|
|
|
|
bool try_fetch_shared = true);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-03-09 17:49:50 +00:00
|
|
|
/** Download the specified part from the specified replica.
|
|
|
|
* Used for replace local part on the same s3-shared part in hybrid storage.
|
|
|
|
* Returns false if part is already fetching right now.
|
|
|
|
*/
|
2022-04-22 16:58:09 +00:00
|
|
|
DataPartStoragePtr fetchExistsPart(
|
2021-03-09 17:49:50 +00:00
|
|
|
const String & part_name,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const String & replica_path,
|
|
|
|
DiskPtr replaced_disk,
|
|
|
|
String replaced_part_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-12 13:47:42 +00:00
|
|
|
/// Required only to avoid races between executeLogEntry and fetchPartition
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unordered_set<String> currently_fetching_parts;
|
|
|
|
std::mutex currently_fetching_parts_mutex;
|
|
|
|
|
2017-05-12 13:47:42 +00:00
|
|
|
/// With the quorum being tracked, add a replica to the quorum for the part.
|
2020-10-07 11:28:48 +00:00
|
|
|
void updateQuorum(const String & part_name, bool is_parallel);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-13 15:21:05 +00:00
|
|
|
/// Deletes info from quorum/last_part node for particular partition_id.
|
2020-04-20 10:56:59 +00:00
|
|
|
void cleanLastPartNode(const String & partition_id);
|
2020-04-10 21:29:54 +00:00
|
|
|
|
2020-11-03 09:24:10 +00:00
|
|
|
/// Part name is stored in quorum/last_part for corresponding partition_id.
|
|
|
|
bool partIsLastQuorumPart(const MergeTreePartInfo & part_info) const;
|
|
|
|
|
|
|
|
/// Part currently inserting with quorum (node quorum/parallel/part_name exists)
|
|
|
|
bool partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const;
|
2020-10-01 10:38:50 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Creates new block number if block with such block_id does not exist
|
2021-05-21 09:30:49 +00:00
|
|
|
/// If zookeeper_path_prefix specified then allocate block number on this path
|
|
|
|
/// (can be used if we want to allocate blocks on other replicas)
|
2018-07-04 16:31:21 +00:00
|
|
|
std::optional<EphemeralLockInZooKeeper> allocateBlockNumber(
|
2020-11-10 10:23:46 +00:00
|
|
|
const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper,
|
2021-05-21 09:30:49 +00:00
|
|
|
const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Wait until all replicas, including this, execute the specified action from the log.
|
2021-08-23 12:57:50 +00:00
|
|
|
* If replicas are added at the same time, it can not wait the added replica.
|
|
|
|
*
|
|
|
|
* Waits for inactive replicas no more than wait_for_inactive_timeout.
|
|
|
|
* Returns list of inactive replicas that have not executed entry or throws exception.
|
2019-08-20 01:46:48 +00:00
|
|
|
*
|
|
|
|
* NOTE: This method must be called without table lock held.
|
|
|
|
* Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2021-08-23 12:57:50 +00:00
|
|
|
void waitForAllReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry,
|
|
|
|
Int64 wait_for_inactive_timeout, const String & error_context = {});
|
|
|
|
Strings tryWaitForAllReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry,
|
|
|
|
Int64 wait_for_inactive_timeout);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Wait until the specified replica executes the specified action from the log.
|
2019-08-20 01:46:48 +00:00
|
|
|
* NOTE: See comment about locks above.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2021-08-23 12:57:50 +00:00
|
|
|
bool tryWaitForReplicaToProcessLogEntry(const String & table_zookeeper_path, const String & replica_name,
|
|
|
|
const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout = 0);
|
|
|
|
|
|
|
|
/// Depending on settings, do nothing or wait for this replica or all replicas process log entry.
|
|
|
|
void waitForLogEntryToBeProcessedIfNecessary(const ReplicatedMergeTreeLogEntryData & entry, ContextPtr query_context, const String & error_context = {});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Throw an exception if the table is readonly.
|
2017-04-01 07:20:54 +00:00
|
|
|
void assertNotReadonly() const;
|
2016-01-21 16:30:05 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Produce an imaginary part info covering all parts in the specified partition (at the call moment).
|
|
|
|
/// Returns false if the partition doesn't exist yet.
|
2021-05-13 14:04:36 +00:00
|
|
|
/// Caller must hold delimiting_block_lock until creation of drop/replace entry in log.
|
|
|
|
/// Otherwise some replica may assign merge which intersects part_info.
|
|
|
|
bool getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info,
|
|
|
|
std::optional<EphemeralLockInZooKeeper> & delimiting_block_lock, bool for_replace_range = false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Check for a node in ZK. If it is, remember this information, and then immediately answer true.
|
2020-11-10 10:23:46 +00:00
|
|
|
mutable std::unordered_set<std::string> existing_nodes_cache;
|
|
|
|
mutable std::mutex existing_nodes_cache_mutex;
|
|
|
|
bool existsNodeCached(const std::string & path) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-02 17:30:53 +00:00
|
|
|
void 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
|
|
|
/// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range.
|
|
|
|
void clearBlocksInPartition(
|
|
|
|
zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-17 17:59:42 +00:00
|
|
|
/// Info about how other replicas can access this one.
|
|
|
|
ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const;
|
2018-06-09 15:48:22 +00:00
|
|
|
|
2022-05-06 14:12:31 +00:00
|
|
|
bool addOpsToDropAllPartsInPartition(
|
|
|
|
zkutil::ZooKeeper & zookeeper, const String & partition_id, bool detach,
|
|
|
|
Coordination::Requests & ops, std::vector<LogEntryPtr> & entries,
|
|
|
|
std::vector<EphemeralLockInZooKeeper> & delimiting_block_locks,
|
|
|
|
std::vector<size_t> & log_entry_ops_idx);
|
|
|
|
void dropAllPartsInPartitions(
|
|
|
|
zkutil::ZooKeeper & zookeeper, const Strings partition_ids, std::vector<LogEntryPtr> & entries, ContextPtr query_context, bool detach);
|
|
|
|
|
|
|
|
LogEntryPtr dropAllPartsInPartition(
|
|
|
|
zkutil::ZooKeeper & zookeeper, const String & partition_id, ContextPtr query_context, bool detach);
|
|
|
|
|
|
|
|
|
|
|
|
void dropAllPartitionsImpl(const zkutil::ZooKeeperPtr & zookeeper, bool detach, ContextPtr query_context);
|
2018-04-17 17:59:42 +00:00
|
|
|
|
2021-05-17 14:26:36 +00:00
|
|
|
void dropPartNoWaitNoThrow(const String & part_name) override;
|
|
|
|
void dropPart(const String & part_name, bool detach, ContextPtr query_context) override;
|
2018-04-17 17:59:42 +00:00
|
|
|
|
2018-11-13 13:48:53 +00:00
|
|
|
// Partition helpers
|
2021-05-17 14:26:36 +00:00
|
|
|
void dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context) override;
|
2021-04-10 23:33:54 +00:00
|
|
|
PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr query_context) override;
|
|
|
|
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override;
|
|
|
|
void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override;
|
2021-04-20 12:26:05 +00:00
|
|
|
void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override;
|
2021-09-16 16:03:31 +00:00
|
|
|
CancellationCode killPartMoveToShard(const UUID & task_uuid) override;
|
2021-04-13 04:40:33 +00:00
|
|
|
void fetchPartition(
|
|
|
|
const ASTPtr & partition,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const String & from,
|
|
|
|
bool fetch_part,
|
|
|
|
ContextPtr query_context) override;
|
2018-11-13 13:48:53 +00:00
|
|
|
|
2021-06-30 12:29:09 +00:00
|
|
|
/// NOTE: there are no guarantees for concurrent merges. Dropping part can
|
|
|
|
/// be concurrently merged into some covering part and dropPart will do
|
|
|
|
/// nothing. There are some fundamental problems with it. But this is OK
|
|
|
|
/// because:
|
|
|
|
///
|
|
|
|
/// dropPart used in the following cases:
|
|
|
|
/// 1) Remove empty parts after TTL.
|
|
|
|
/// 2) Remove parts after move between shards.
|
|
|
|
/// 3) User queries: ALTER TABLE DROP PART 'part_name'.
|
|
|
|
///
|
|
|
|
/// In the first case merge of empty part is even better than DROP. In the
|
|
|
|
/// second case part UUIDs used to forbid merges for moving parts so there
|
|
|
|
/// is no problem with concurrent merges. The third case is quite rare and
|
|
|
|
/// we give very weak guarantee: there will be no active part with this
|
|
|
|
/// name, but possibly it was merged to some other part.
|
|
|
|
///
|
|
|
|
/// NOTE: don't rely on dropPart if you 100% need to remove non-empty part
|
|
|
|
/// and don't use any explicit locking mechanism for merges.
|
2021-05-27 23:10:44 +00:00
|
|
|
bool dropPartImpl(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop);
|
2018-11-13 13:48:53 +00:00
|
|
|
|
2019-08-12 13:30:29 +00:00
|
|
|
/// Check granularity of already existing replicated table in zookeeper if it exists
|
|
|
|
/// return true if it's fixed
|
2021-06-28 17:02:22 +00:00
|
|
|
bool checkFixedGranularityInZookeeper();
|
2019-08-12 13:30:29 +00:00
|
|
|
|
2019-12-19 15:27:56 +00:00
|
|
|
/// Wait for timeout seconds mutation is finished on replicas
|
|
|
|
void waitMutationToFinishOnReplicas(
|
|
|
|
const Strings & replicas, const String & mutation_id) const;
|
2019-12-16 15:51:15 +00:00
|
|
|
|
2020-11-28 08:17:20 +00:00
|
|
|
MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override;
|
2020-02-01 11:47:09 +00:00
|
|
|
|
2020-06-23 16:40:58 +00:00
|
|
|
void startBackgroundMovesIfNeeded() override;
|
|
|
|
|
2022-05-19 12:36:27 +00:00
|
|
|
/// Attaches restored parts to the storage.
|
|
|
|
void attachRestoredParts(MutableDataPartsVector && parts) override;
|
|
|
|
|
2021-07-05 12:44:58 +00:00
|
|
|
std::unique_ptr<MergeTreeSettings> getDefaultSettings() const override;
|
|
|
|
|
2020-11-10 10:23:46 +00:00
|
|
|
PartitionBlockNumbersHolder 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
|
|
|
|
2022-04-21 19:19:13 +00:00
|
|
|
static Strings getZeroCopyPartPath(const MergeTreeSettings & settings, std::string disk_type, const String & table_uuid,
|
2021-12-17 11:03:20 +00:00
|
|
|
const String & part_name, const String & zookeeper_path_old);
|
2021-11-23 13:57:24 +00:00
|
|
|
|
2022-04-19 12:01:30 +00:00
|
|
|
static void createZeroCopyLockNode(
|
|
|
|
const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node,
|
|
|
|
int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false,
|
|
|
|
const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {});
|
2021-11-23 13:57:24 +00:00
|
|
|
|
2021-12-21 14:29:50 +00:00
|
|
|
bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override;
|
|
|
|
|
2021-12-30 09:57:38 +00:00
|
|
|
/// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled.
|
2021-12-27 16:27:06 +00:00
|
|
|
void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override;
|
|
|
|
|
|
|
|
// Create table id if needed
|
|
|
|
void createTableSharedID();
|
2021-12-21 14:29:50 +00:00
|
|
|
|
2022-02-10 19:45:52 +00:00
|
|
|
|
|
|
|
bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk);
|
|
|
|
|
|
|
|
std::optional<String> getZeroCopyPartPath(const String & part_name, const DiskPtr & disk);
|
|
|
|
|
2022-01-17 11:52:51 +00:00
|
|
|
/// Create ephemeral lock in zookeeper for part and disk which support zero copy replication.
|
|
|
|
/// If somebody already holding the lock -- return std::nullopt.
|
2022-02-10 19:45:52 +00:00
|
|
|
std::optional<ZeroCopyLock> tryCreateZeroCopyExclusiveLock(const String & part_name, const DiskPtr & disk) override;
|
2016-01-21 16:30:05 +00:00
|
|
|
};
|
|
|
|
|
2021-09-16 16:03:31 +00:00
|
|
|
String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info);
|
|
|
|
|
2016-04-07 21:35:01 +00:00
|
|
|
|
2020-02-25 18:58:28 +00:00
|
|
|
/** There are three places for each part, where it should be
|
|
|
|
* 1. In the RAM, data_parts, all_data_parts.
|
|
|
|
* 2. In the filesystem (FS), the directory with the data of the table.
|
|
|
|
* 3. in ZooKeeper (ZK).
|
|
|
|
*
|
|
|
|
* When adding a part, it must be added immediately to these three places.
|
|
|
|
* This is done like this
|
|
|
|
* - [FS] first write the part into a temporary directory on the filesystem;
|
|
|
|
* - [FS] rename the temporary part to the result on the filesystem;
|
|
|
|
* - [RAM] immediately afterwards add it to the `data_parts`, and remove from `data_parts` any parts covered by this one;
|
|
|
|
* - [RAM] also set the `Transaction` object, which in case of an exception (in next point),
|
|
|
|
* rolls back the changes in `data_parts` (from the previous point) back;
|
|
|
|
* - [ZK] then send a transaction (multi) to add a part to ZooKeeper (and some more actions);
|
|
|
|
* - [FS, ZK] by the way, removing the covered (old) parts from filesystem, from ZooKeeper and from `all_data_parts`
|
|
|
|
* is delayed, after a few minutes.
|
|
|
|
*
|
|
|
|
* There is no atomicity here.
|
|
|
|
* It could be possible to achieve atomicity using undo/redo logs and a flag in `DataPart` when it is completely ready.
|
|
|
|
* But it would be inconvenient - I would have to write undo/redo logs for each `Part` in ZK, and this would increase already large number of interactions.
|
|
|
|
*
|
|
|
|
* Instead, we are forced to work in a situation where at any time
|
|
|
|
* (from another thread, or after server restart), there may be an unfinished transaction.
|
|
|
|
* (note - for this the part should be in RAM)
|
|
|
|
* From these cases the most frequent one is when the part is already in the data_parts, but it's not yet in ZooKeeper.
|
|
|
|
* This case must be distinguished from the case where such a situation is achieved due to some kind of damage to the state.
|
|
|
|
*
|
|
|
|
* Do this with the threshold for the time.
|
|
|
|
* If the part is young enough, its lack in ZooKeeper will be perceived optimistically - as if it just did not have time to be added there
|
|
|
|
* - as if the transaction has not yet been executed, but will soon be executed.
|
|
|
|
* And if the part is old, its absence in ZooKeeper will be perceived as an unfinished transaction that needs to be rolled back.
|
|
|
|
*
|
|
|
|
* PS. Perhaps it would be better to add a flag to the DataPart that a part is inserted into ZK.
|
|
|
|
* But here it's too easy to get confused with the consistency of this flag.
|
|
|
|
*/
|
2022-03-13 12:23:51 +00:00
|
|
|
/// NOLINTNEXTLINE
|
2020-02-25 18:58:28 +00:00
|
|
|
#define MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER (5 * 60)
|
2016-04-07 21:35:01 +00:00
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
}
|