ClickHouse/src/Storages/StorageMergeTree.h

245 lines
10 KiB
C++
Raw Normal View History

2012-07-16 20:25:19 +00:00
#pragma once
2021-10-02 07:13:14 +00:00
#include <base/shared_ptr_helper.h>
#include <Core/Names.h>
2019-05-09 14:25:18 +00:00
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
2019-08-19 14:40:12 +00:00
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Storages/MergeTree/MergeTreeMutationEntry.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
2021-04-02 11:46:42 +00:00
#include <Storages/MergeTree/MergeTreeDeduplicationLog.h>
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
#include <Storages/MergeTree/MergePlainMergeTreeTask.h>
#include <Storages/MergeTree/MutatePlainMergeTreeTask.h>
2021-04-02 11:46:42 +00:00
#include <Disks/StoragePolicy.h>
#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
*/
2021-06-15 19:55:21 +00:00
class StorageMergeTree final : public shared_ptr_helper<StorageMergeTree>, public MergeTreeData
2012-07-16 20:25:19 +00:00
{
2021-06-15 19:55:21 +00:00
friend struct shared_ptr_helper<StorageMergeTree>;
2012-07-16 20:25:19 +00:00
public:
void startup() override;
void shutdown() override;
~StorageMergeTree() override;
2019-05-03 02:00:57 +00:00
std::string getName() const override { return merging_params.getModeName() + "MergeTree"; }
2019-12-12 10:49:15 +00:00
bool supportsParallelInsert() const override { return true; }
bool supportsIndexForIn() const override { return true; }
2020-08-03 13:54:14 +00:00
Pipe read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
2017-06-02 15:54:39 +00:00
unsigned num_streams) override;
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
2020-11-25 13:47:32 +00:00
std::optional<UInt64> totalRows(const Settings &) const override;
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;
2021-07-23 19:33:59 +00:00
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
2017-04-16 15:00:33 +00:00
/** Perform the next step in combining the parts.
*/
2020-06-17 13:39:26 +00:00
bool optimize(
const ASTPtr & query,
const StorageMetadataPtr & /*metadata_snapshot*/,
const ASTPtr & partition,
bool final,
bool deduplicate,
const Names & deduplicate_by_columns,
ContextPtr context) override;
void mutate(const MutationCommands & commands, ContextPtr context) override;
/// Return introspection information about currently processing or recently processed mutations.
2019-05-03 02:00:57 +00:00
std::vector<MergeTreeMutationStatus> getMutationsStatus() const override;
CancellationCode killMutation(const String & mutation_id) override;
2020-01-22 11:30:11 +00:00
void drop() override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
2021-10-25 17:49:49 +00:00
void alter(const AlterCommands & commands, ContextPtr context, AlterLockHolder & table_lock_holder) override;
void checkTableCanBeDropped() const override;
ActionLock getActionLock(StorageActionBlockType action_type) override;
2020-10-15 16:10:22 +00:00
void onActionLockRemove(StorageActionBlockType action_type) override;
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
2019-07-03 08:49:52 +00:00
2021-08-18 22:19:14 +00:00
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
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:
/// Mutex and condvar for synchronous mutations wait
std::mutex mutation_wait_mutex;
std::condition_variable mutation_wait_event;
MergeTreeDataSelectExecutor reader;
MergeTreeDataWriter writer;
MergeTreeDataMergerMutator merger_mutator;
2021-04-02 16:45:18 +00:00
std::unique_ptr<MergeTreeDeduplicationLog> deduplication_log;
2021-04-02 11:46:42 +00:00
/// For block numbers.
2020-12-21 07:48:15 +00:00
SimpleIncrement increment;
/// For clearOldParts
AtomicStopwatch time_after_previous_cleanup_parts;
/// For clearOldTemporaryDirectories.
AtomicStopwatch time_after_previous_cleanup_temporary_directories;
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;
std::map<String, MergeTreeMutationEntry> current_mutations_by_id;
std::multimap<Int64, MergeTreeMutationEntry &> current_mutations_by_version;
std::atomic<bool> shutdown_called {false};
private:
void loadMutations();
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();
/** 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.
*/
bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, const Names & deduplicate_by_columns, String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false);
/// 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.
MergeTreeDataPartPtr outdatePart(const String & part_name, bool force);
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.
Int64 startMutation(const MutationCommands & commands, String & mutation_file_name);
/// Wait until mutation with version will finish mutation for all parts
2020-04-03 16:33:25 +00:00
void waitForMutation(Int64 version, const String & file_name);
2020-03-18 10:02:57 +00:00
2020-10-23 08:54:00 +00:00
friend struct CurrentlyMergingPartsTagger;
2020-11-10 20:01:43 +00:00
std::shared_ptr<MergeMutateSelectedEntry> selectPartsToMerge(
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,
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-10-23 08:54:00 +00:00
std::shared_ptr<MergeMutateSelectedEntry> selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder);
Int64 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;
2019-07-29 09:15:46 +00:00
void clearOldMutations(bool truncate = false);
// 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-22 20:07:00 +00:00
void dropPartsImpl(DataPartsVector && parts_to_remove, bool detach);
PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override;
2020-07-28 15:10:36 +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.
void updateMutationEntriesErrors(FutureMergedMutatedPartPtr result_part, bool is_successful, const String & exception_message);
2019-09-05 13:12:29 +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;
void startBackgroundMovesIfNeeded() override;
2021-07-05 12:44:58 +00:00
std::unique_ptr<MergeTreeSettings> getDefaultSettings() const override;
friend class MergeTreeProjectionBlockOutputStream;
2021-07-23 19:33:59 +00:00
friend class MergeTreeSink;
friend class MergeTreeData;
friend class MergePlainMergeTreeTask;
friend class MutatePlainMergeTreeTask;
protected:
2019-07-24 12:56:39 +00:00
2018-11-26 00:56:50 +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(
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
const String & relative_data_path_,
2019-12-27 16:34:50 +00:00
const StorageInMemoryMetadata & metadata,
bool attach,
2021-05-31 14:49:02 +00:00
ContextMutablePtr context_,
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_,
bool has_force_restore_data_flag);
2020-03-24 17:05:38 +00:00
2020-11-28 08:17:20 +00:00
MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override;
2012-07-16 20:25:19 +00:00
};
}