2012-07-16 20:25:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2018-11-19 14:31:16 +00:00
|
|
|
#include <Core/Names.h>
|
2019-05-09 14:25:18 +00:00
|
|
|
#include <Storages/AlterCommands.h>
|
2018-03-06 20:18:34 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
2018-04-20 16:18:16 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
|
2019-08-19 14:40:12 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreePartsMover.h>
|
2018-03-30 19:25:37 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeMutationEntry.h>
|
2018-07-06 19:04:54 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
|
2021-04-02 11:46:42 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDeduplicationLog.h>
|
2021-09-16 21:19:58 +00:00
|
|
|
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
|
|
|
|
#include <Storages/MergeTree/MergePlainMergeTreeTask.h>
|
|
|
|
#include <Storages/MergeTree/MutatePlainMergeTreeTask.h>
|
2021-04-02 11:46:42 +00:00
|
|
|
|
2020-05-04 20:15:38 +00:00
|
|
|
#include <Disks/StoragePolicy.h>
|
2017-05-10 06:49:19 +00:00
|
|
|
#include <Common/SimpleIncrement.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2012-07-16 20:25:19 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** See the description of the data structure in MergeTreeData.
|
2012-07-16 20:25:19 +00:00
|
|
|
*/
|
2022-05-03 06:43:28 +00:00
|
|
|
class StorageMergeTree final : public MergeTreeData
|
2012-07-16 20:25:19 +00:00
|
|
|
{
|
|
|
|
public:
|
2022-04-19 20:47:29 +00:00
|
|
|
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
|
|
|
* (correctness of names and paths are not checked)
|
|
|
|
* consisting of the specified columns.
|
|
|
|
*
|
|
|
|
* See MergeTreeData constructor for comments on parameters.
|
|
|
|
*/
|
|
|
|
StorageMergeTree(
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const String & relative_data_path_,
|
|
|
|
const StorageInMemoryMetadata & metadata,
|
|
|
|
bool attach,
|
|
|
|
ContextMutablePtr context_,
|
|
|
|
const String & date_column_name,
|
|
|
|
const MergingParams & merging_params_,
|
|
|
|
std::unique_ptr<MergeTreeSettings> settings_,
|
|
|
|
bool has_force_restore_data_flag);
|
|
|
|
|
2017-06-06 17:06:14 +00:00
|
|
|
void startup() override;
|
2021-12-14 08:31:17 +00:00
|
|
|
void flush() override;
|
2014-10-03 17:55:36 +00:00
|
|
|
void shutdown() override;
|
2022-04-19 20:47:29 +00:00
|
|
|
|
2014-10-03 17:55:36 +00:00
|
|
|
~StorageMergeTree() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-05-03 02:00:57 +00:00
|
|
|
std::string getName() const override { return merging_params.getModeName() + "MergeTree"; }
|
2018-03-19 14:29:40 +00:00
|
|
|
|
2019-12-12 10:49:15 +00:00
|
|
|
bool supportsParallelInsert() const override { return true; }
|
|
|
|
|
2018-03-19 14:29:40 +00:00
|
|
|
bool supportsIndexForIn() const override { return true; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-01-31 22:27:55 +00:00
|
|
|
bool supportsTransactions() const override { return true; }
|
|
|
|
|
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,
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams) override;
|
2020-10-01 17:34:22 +00:00
|
|
|
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> totalRows(const Settings &) const override;
|
2021-04-10 23:33:54 +00:00
|
|
|
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override;
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> totalBytes(const 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
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Perform the next step in combining the parts.
|
2015-04-14 14:58:59 +00:00
|
|
|
*/
|
2020-06-17 13:39:26 +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,
|
2023-02-16 13:03:16 +00:00
|
|
|
bool cleanup,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-01-30 17:38:28 +00:00
|
|
|
void mutate(const MutationCommands & commands, ContextPtr context) override;
|
2019-12-12 16:24:03 +00:00
|
|
|
|
2022-07-06 10:29:29 +00:00
|
|
|
bool hasLightweightDeletedMask() const override;
|
2022-06-08 02:31:11 +00:00
|
|
|
|
2019-12-12 16:24:03 +00:00
|
|
|
/// Return introspection information about currently processing or recently processed mutations.
|
2019-05-03 02:00:57 +00:00
|
|
|
std::vector<MergeTreeMutationStatus> getMutationsStatus() const override;
|
2019-12-12 16:24:03 +00:00
|
|
|
|
2019-02-04 13:04:02 +00:00
|
|
|
CancellationCode killMutation(const String & mutation_id) override;
|
2018-07-06 19:04:54 +00:00
|
|
|
|
2022-07-05 07:39:52 +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;
|
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
void drop() override;
|
2021-04-10 23:33:54 +00:00
|
|
|
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-10-25 17:49:49 +00:00
|
|
|
void alter(const AlterCommands & commands, ContextPtr context, AlterLockHolder & table_lock_holder) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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;
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
2019-07-03 08:49:52 +00:00
|
|
|
|
2021-09-08 00:21:21 +00:00
|
|
|
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
2021-04-02 11:46:42 +00:00
|
|
|
|
2021-04-02 16:45:18 +00:00
|
|
|
MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); }
|
2021-08-30 19:37:03 +00:00
|
|
|
|
2012-07-16 20:25:19 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-12-16 15:51:15 +00:00
|
|
|
/// Mutex and condvar for synchronous mutations wait
|
|
|
|
std::mutex mutation_wait_mutex;
|
|
|
|
std::condition_variable mutation_wait_event;
|
|
|
|
|
2014-03-13 12:48:07 +00:00
|
|
|
MergeTreeDataSelectExecutor reader;
|
|
|
|
MergeTreeDataWriter writer;
|
2018-03-30 19:25:37 +00:00
|
|
|
MergeTreeDataMergerMutator merger_mutator;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-02 16:45:18 +00:00
|
|
|
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;
|
2021-04-02 11:46:42 +00:00
|
|
|
|
2016-11-18 01:03:57 +00:00
|
|
|
/// For block numbers.
|
2020-12-21 07:48:15 +00:00
|
|
|
SimpleIncrement increment;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-07-14 10:18:29 +00:00
|
|
|
/// For clearOldParts
|
|
|
|
AtomicStopwatch time_after_previous_cleanup_parts;
|
|
|
|
/// For clearOldTemporaryDirectories.
|
|
|
|
AtomicStopwatch time_after_previous_cleanup_temporary_directories;
|
2022-06-09 14:59:12 +00:00
|
|
|
/// For clearOldBrokenDetachedParts
|
|
|
|
AtomicStopwatch time_after_previous_cleanup_broken_detached_parts;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-09-02 11:35:53 +00:00
|
|
|
/// Mutex for parts currently processing in background
|
|
|
|
/// merging (also with TTL), mutating or moving.
|
2019-08-15 09:43:31 +00:00
|
|
|
mutable std::mutex currently_processing_in_background_mutex;
|
2020-06-02 00:41:52 +00:00
|
|
|
mutable std::condition_variable currently_processing_in_background_condition;
|
2019-09-02 11:35:53 +00:00
|
|
|
|
|
|
|
/// Parts that currently participate in merge or mutation.
|
|
|
|
/// This set have to be used with `currently_processing_in_background_mutex`.
|
|
|
|
DataParts currently_merging_mutating_parts;
|
|
|
|
|
2021-11-19 16:51:03 +00:00
|
|
|
std::map<UInt64, MergeTreeMutationEntry> current_mutations_by_version;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
std::atomic<bool> shutdown_called {false};
|
2021-12-27 15:54:28 +00:00
|
|
|
std::atomic<bool> flush_called {false};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-03-02 19:19:58 +00:00
|
|
|
/// PreparedSets cache for one executing mutation.
|
|
|
|
/// NOTE: we only store weak_ptr to PreparedSetsCache, so that the cache is shared between mutation tasks that are executed in parallel.
|
|
|
|
/// The goal is to avoiding consuming a lot of memory when the same big sets are used by multiple tasks at the same time.
|
|
|
|
/// If the tasks are executed without time overlap, we will destroy the cache to free memory, and the next task might rebuild the same sets.
|
|
|
|
std::mutex mutation_prepared_sets_cache_mutex;
|
2023-04-13 14:04:30 +00:00
|
|
|
std::map<Int64, PreparedSetsCachePtr::weak_type> mutation_prepared_sets_cache;
|
2023-03-02 19:19:58 +00:00
|
|
|
|
2018-07-11 12:43:55 +00:00
|
|
|
void loadMutations();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-06 10:14:44 +00:00
|
|
|
/// Load and initialize deduplication logs. Even if deduplication setting
|
|
|
|
/// equals zero creates object with deduplication window equals zero.
|
|
|
|
void loadDeduplicationLog();
|
|
|
|
|
2017-11-04 03:20:18 +00:00
|
|
|
/** Determines what parts should be merged and merges it.
|
|
|
|
* If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query).
|
|
|
|
* Returns true if merge is finished successfully.
|
|
|
|
*/
|
2021-05-17 11:14:09 +00:00
|
|
|
bool merge(
|
|
|
|
bool aggressive,
|
|
|
|
const String & partition_id,
|
|
|
|
bool final, bool deduplicate,
|
|
|
|
const Names & deduplicate_by_columns,
|
2023-02-16 13:03:16 +00:00
|
|
|
bool cleanup,
|
2021-05-17 11:14:09 +00:00
|
|
|
const MergeTreeTransactionPtr & txn,
|
|
|
|
String * out_disable_reason = nullptr,
|
|
|
|
bool optimize_skip_merged_partitions = false);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2022-10-13 16:07:25 +00:00
|
|
|
void renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction);
|
2022-09-09 14:03:05 +00:00
|
|
|
|
2020-12-22 13:46:09 +00:00
|
|
|
/// Make part state outdated and queue it to remove without timeout
|
|
|
|
/// If force, then stop merges and block them until part state became outdated. Throw exception if part doesn't exists
|
2020-12-22 13:48:10 +00:00
|
|
|
/// If not force, then take merges selector and check that part is not participating in background operations.
|
2023-01-05 16:23:20 +00:00
|
|
|
MergeTreeDataPartPtr outdatePart(MergeTreeTransaction * txn, const String & part_name, bool force, bool clear_without_timeout = true);
|
2020-08-15 05:21:02 +00:00
|
|
|
ActionLock stopMergesAndWait();
|
|
|
|
|
2020-04-03 11:09:27 +00:00
|
|
|
/// Allocate block number for new mutation, write mutation to disk
|
|
|
|
/// and into in-memory structures. Wake up merge-mutation task.
|
2022-07-21 07:32:29 +00:00
|
|
|
Int64 startMutation(const MutationCommands & commands, ContextPtr query_context);
|
2020-04-03 11:09:27 +00:00
|
|
|
/// Wait until mutation with version will finish mutation for all parts
|
2021-12-14 20:06:34 +00:00
|
|
|
void waitForMutation(Int64 version);
|
|
|
|
void waitForMutation(const String & mutation_id) override;
|
2022-07-13 04:34:12 +00:00
|
|
|
void waitForMutation(Int64 version, const String & mutation_id);
|
2022-03-17 17:26:18 +00:00
|
|
|
void setMutationCSN(const String & mutation_id, CSN csn) override;
|
|
|
|
|
2020-03-18 10:02:57 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
friend struct CurrentlyMergingPartsTagger;
|
2020-09-30 12:40:46 +00:00
|
|
|
|
2022-10-18 11:35:34 +00:00
|
|
|
MergeMutateSelectedEntryPtr selectPartsToMerge(
|
2020-11-10 20:01:43 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
bool aggressive,
|
|
|
|
const String & partition_id,
|
|
|
|
bool final,
|
|
|
|
String * disable_reason,
|
|
|
|
TableLockHolder & table_lock_holder,
|
Fix race between DETACH and merges
CI reports failure of the 01442_merge_detach_attach test [1]:
<details>
2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference:
2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300
2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300
2021-06-21 02:25:43 @@ -0,0 +1 @@
2021-06-21 02:25:43 +2
2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple()
2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL
2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t
2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t
2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0
2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0
2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB.
2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0
2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact
2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts.
2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal
2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x
2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x
2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B.
2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec.
2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection.
2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec.
2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1.
2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2
2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0
2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB.
2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB.
...
2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0
2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t
2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete
2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown
2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges
2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0
2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating
2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key
2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.)
2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data
2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec.
2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B.
</details>
[1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1
The problem here is that there is a tiny race window between DETACH
and OPTIMIZE.
DETACH cancel merges, and checks that there no merges for parts
currently under lock, and later changes the state of parts. After it
allows merges.
However OPTIMIZE get parts for processing under lock and only when it
starts writing temporary part it checks that merges are not canceled.
So suppose the following scenario:
T#OPTIMIZE T#DETACH
- cancel merges
- lock
- check that no merges are in progress
- unlock
- lock
- get parts
*<--Here, state of the parts are not changed yet,
and lock is not held,
so OPTIMIZE may (and will) get those parts. -->*
- write temporary part
- check merges not canceled
- unlock
- change the parts state
- allows merges
Plus this patch will also fail merges earlier in case of concurrent
DETACH.
Refs: #13746
Refs: #23315
2021-06-22 23:19:06 +00:00
|
|
|
std::unique_lock<std::mutex> & lock,
|
2021-05-17 11:14:09 +00:00
|
|
|
const MergeTreeTransactionPtr & txn,
|
2020-12-04 14:01:59 +00:00
|
|
|
bool optimize_skip_merged_partitions = false,
|
2020-11-11 10:38:00 +00:00
|
|
|
SelectPartsDecision * select_decision_out = nullptr);
|
2020-12-01 09:10:12 +00:00
|
|
|
|
2020-09-30 13:49:22 +00:00
|
|
|
|
2022-10-18 11:35:34 +00:00
|
|
|
MergeMutateSelectedEntryPtr selectPartsToMutate(
|
2022-08-24 18:58:59 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot, String * disable_reason,
|
|
|
|
TableLockHolder & table_lock_holder, std::unique_lock<std::mutex> & currently_processing_in_background_mutex_lock);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2021-12-01 08:18:07 +00:00
|
|
|
/// For current mutations queue, returns maximum version of mutation for a part,
|
|
|
|
/// with respect of mutations which would not change it.
|
|
|
|
/// Returns 0 if there is no such mutation in active status.
|
|
|
|
UInt64 getCurrentMutationVersion(
|
2019-05-03 02:00:57 +00:00
|
|
|
const DataPartPtr & part,
|
2020-06-02 00:41:52 +00:00
|
|
|
std::unique_lock<std::mutex> & /* currently_processing_in_background_mutex_lock */) const;
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2023-01-20 11:10:31 +00:00
|
|
|
/// Returns the maximum level of all outdated parts in a range (left; right), or 0 in case if empty range.
|
|
|
|
/// Merges have to be aware of the outdated part's levels inside designated merge range.
|
|
|
|
/// When two parts all_1_1_0, all_3_3_0 are merged into all_1_3_1, the gap between those parts have to be verified.
|
2023-01-20 12:35:23 +00:00
|
|
|
/// There should not be an unactive part all_1_1_1. Otherwise it is impossible to load parts after restart, they intersects.
|
2023-01-20 13:55:58 +00:00
|
|
|
/// Therefore this function is used in merge predicate in order to prevent merges over the gaps with high level outdated parts.
|
2023-01-05 16:23:20 +00:00
|
|
|
UInt32 getMaxLevelInBetween(
|
|
|
|
const DataPartPtr & left,
|
|
|
|
const DataPartPtr & right) const;
|
|
|
|
|
2021-11-09 12:26:51 +00:00
|
|
|
size_t clearOldMutations(bool truncate = false);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2018-11-13 13:48:53 +00:00
|
|
|
// Partition helpers
|
2021-05-17 14:26:36 +00:00
|
|
|
void dropPartNoWaitNoThrow(const String & part_name) override;
|
|
|
|
void dropPart(const String & part_name, bool detach, ContextPtr context) override;
|
|
|
|
void dropPartition(const ASTPtr & partition, bool detach, ContextPtr context) override;
|
2021-04-10 23:33:54 +00:00
|
|
|
PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override;
|
2020-07-28 15:10:36 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override;
|
|
|
|
void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) override;
|
2019-09-05 13:12:29 +00:00
|
|
|
bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override;
|
2020-07-22 19:29:54 +00:00
|
|
|
/// Update mutation entries after part mutation execution. May reset old
|
|
|
|
/// errors if mutation was successful. Otherwise update last_failed* fields
|
|
|
|
/// in mutation entries.
|
2021-09-16 21:19:58 +00:00
|
|
|
void updateMutationEntriesErrors(FutureMergedMutatedPartPtr result_part, bool is_successful, const String & exception_message);
|
2019-09-05 13:12:29 +00:00
|
|
|
|
2020-07-22 12:36:19 +00:00
|
|
|
/// Return empty optional if mutation was killed. Otherwise return partially
|
|
|
|
/// filled mutation status with information about error (latest_fail*) and
|
2020-07-31 12:22:32 +00:00
|
|
|
/// is_done. mutation_ids filled with mutations with the same errors,
|
|
|
|
/// because we can execute several mutations at once. Order is important for
|
|
|
|
/// better readability of exception message. If mutation was killed doesn't
|
|
|
|
/// return any ids.
|
2020-07-31 11:37:16 +00:00
|
|
|
std::optional<MergeTreeMutationStatus> getIncompleteMutationsStatus(Int64 mutation_version, std::set<String> * mutation_ids = nullptr) const;
|
2018-11-13 13:48:53 +00:00
|
|
|
|
2022-06-24 15:19:59 +00:00
|
|
|
void fillNewPartName(MutableDataPartPtr & part, DataPartsLock & lock);
|
|
|
|
|
2020-06-23 16:40:58 +00:00
|
|
|
void startBackgroundMovesIfNeeded() override;
|
|
|
|
|
2022-07-05 07:39:52 +00:00
|
|
|
BackupEntries backupMutations(UInt64 version, const String & data_path_in_backup) const;
|
|
|
|
|
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;
|
|
|
|
|
2023-03-02 19:19:58 +00:00
|
|
|
PreparedSetsCachePtr getPreparedSetsCache(Int64 mutation_id);
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
friend class MergeTreeSink;
|
2018-05-21 13:49:54 +00:00
|
|
|
friend class MergeTreeData;
|
2021-09-16 21:19:58 +00:00
|
|
|
friend class MergePlainMergeTreeTask;
|
|
|
|
friend class MutatePlainMergeTreeTask;
|
2020-09-30 12:40:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2017-11-04 03:20:18 +00:00
|
|
|
protected:
|
2019-07-24 12:56:39 +00:00
|
|
|
|
2023-02-27 11:27:57 +00:00
|
|
|
std::map<int64_t, MutationCommands> getAlterMutationCommandsForPart(const DataPartPtr & part) const override;
|
2012-07-16 20:25:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|