2012-07-16 20:25:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-06-06 17:18:32 +00:00
|
|
|
#include <ext/shared_ptr_helper.h>
|
2016-08-26 21:25:05 +00:00
|
|
|
|
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>
|
|
|
|
|
2020-05-04 20:15:38 +00:00
|
|
|
#include <Disks/StoragePolicy.h>
|
2017-05-10 06:49:19 +00:00
|
|
|
#include <Common/SimpleIncrement.h>
|
2020-10-13 14:25:42 +00:00
|
|
|
#include <Storages/MergeTree/BackgroundJobsExecutor.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
|
|
|
*/
|
2020-03-19 23:48:53 +00:00
|
|
|
class StorageMergeTree final : public ext::shared_ptr_helper<StorageMergeTree>, public MergeTreeData
|
2012-07-16 20:25:19 +00:00
|
|
|
{
|
2019-08-26 19:07:29 +00:00
|
|
|
friend struct ext::shared_ptr_helper<StorageMergeTree>;
|
2012-07-16 20:25:19 +00:00
|
|
|
public:
|
2017-06-06 17:06:14 +00:00
|
|
|
void startup() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
void shutdown() override;
|
|
|
|
~StorageMergeTree() override;
|
|
|
|
|
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
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-15 19:08:58 +00:00
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & 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-10-01 17:34:22 +00:00
|
|
|
void read(
|
|
|
|
QueryPlan & query_plan,
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
2020-11-10 12:02:22 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2020-10-01 17:34:22 +00:00
|
|
|
const Context & 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;
|
2020-09-21 10:13:01 +00:00
|
|
|
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, const Context &) 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
|
|
|
|
2020-06-15 19:08:58 +00:00
|
|
|
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & 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.
|
2017-04-01 07:20:54 +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,
|
2020-06-17 13:39:26 +00:00
|
|
|
const Context & context) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-30 19:25:37 +00:00
|
|
|
void mutate(const MutationCommands & commands, const Context & context) override;
|
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
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
void drop() override;
|
2020-06-18 16:10:47 +00:00
|
|
|
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-18 16:10:47 +00:00
|
|
|
void alter(const AlterCommands & commands, const Context & context, TableLockHolder & 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;
|
|
|
|
|
2019-07-03 13:17:19 +00:00
|
|
|
CheckResults checkData(const ASTPtr & query, const Context & context) override;
|
2019-07-03 08:49:52 +00:00
|
|
|
|
2020-10-16 10:12:31 +00:00
|
|
|
std::optional<JobAndPool> getDataProcessingJob() override;
|
2021-04-02 11:46:42 +00:00
|
|
|
|
2021-04-02 12:37:42 +00:00
|
|
|
std::optional<MergeTreeDeduplicationLog> & getDeduplicationLog() { return deduplication_log; }
|
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;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeDataSelectExecutor reader;
|
|
|
|
MergeTreeDataWriter writer;
|
2018-03-30 19:25:37 +00:00
|
|
|
MergeTreeDataMergerMutator merger_mutator;
|
2020-10-14 07:22:48 +00:00
|
|
|
BackgroundJobsExecutor background_executor;
|
2020-10-14 14:56:42 +00:00
|
|
|
BackgroundMovesExecutor background_moves_executor;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-04-02 11:46:42 +00:00
|
|
|
std::optional<MergeTreeDeduplicationLog> deduplication_log;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For block numbers.
|
2020-12-21 07:48:15 +00:00
|
|
|
SimpleIncrement increment;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// For clearOldParts, clearOldTemporaryDirectories.
|
2018-02-25 02:43:27 +00:00
|
|
|
AtomicStopwatch time_after_previous_cleanup;
|
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;
|
|
|
|
|
|
|
|
|
2019-01-14 12:25:25 +00:00
|
|
|
std::map<String, MergeTreeMutationEntry> current_mutations_by_id;
|
|
|
|
std::multimap<Int64, MergeTreeMutationEntry &> current_mutations_by_version;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
std::atomic<bool> shutdown_called {false};
|
|
|
|
|
2018-07-11 12:43:55 +00:00
|
|
|
void loadMutations();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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.
|
|
|
|
*/
|
2020-12-01 09:10:12 +00:00
|
|
|
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);
|
2017-11-04 03:20:18 +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.
|
2020-12-22 13:46:09 +00:00
|
|
|
MergeTreeDataPartPtr outdatePart(const String & part_name, bool force);
|
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.
|
|
|
|
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
|
|
|
struct CurrentlyMergingPartsTagger
|
|
|
|
{
|
|
|
|
FutureMergedMutatedPart future_part;
|
|
|
|
ReservationPtr reserved_space;
|
|
|
|
StorageMergeTree & storage;
|
2021-03-08 09:38:07 +00:00
|
|
|
// Optional tagger to maintain volatile parts for the JBOD balancer
|
2021-02-18 08:50:31 +00:00
|
|
|
std::optional<CurrentlySubmergingEmergingTagger> tagger;
|
2020-09-30 12:40:46 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
CurrentlyMergingPartsTagger(
|
|
|
|
FutureMergedMutatedPart & future_part_,
|
|
|
|
size_t total_size,
|
|
|
|
StorageMergeTree & storage_,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
bool is_mutation);
|
|
|
|
|
|
|
|
~CurrentlyMergingPartsTagger();
|
|
|
|
};
|
2018-07-09 15:34:11 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
using CurrentlyMergingPartsTaggerPtr = std::unique_ptr<CurrentlyMergingPartsTagger>;
|
|
|
|
friend struct CurrentlyMergingPartsTagger;
|
2020-09-30 12:40:46 +00:00
|
|
|
|
|
|
|
struct MergeMutateSelectedEntry
|
|
|
|
{
|
|
|
|
FutureMergedMutatedPart future_part;
|
|
|
|
CurrentlyMergingPartsTaggerPtr tagger;
|
|
|
|
MutationCommands commands;
|
2020-10-23 08:54:00 +00:00
|
|
|
MergeMutateSelectedEntry(const FutureMergedMutatedPart & future_part_, CurrentlyMergingPartsTaggerPtr && tagger_, const MutationCommands & commands_)
|
|
|
|
: future_part(future_part_)
|
|
|
|
, tagger(std::move(tagger_))
|
|
|
|
, commands(commands_)
|
|
|
|
{}
|
2020-09-30 12:40:46 +00:00
|
|
|
};
|
|
|
|
|
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,
|
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
|
|
|
|
|
|
|
bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, const Names & deduplicate_by_columns, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder);
|
2020-09-30 13:49:22 +00:00
|
|
|
|
2020-10-23 08:54:00 +00:00
|
|
|
std::shared_ptr<MergeMutateSelectedEntry> selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder);
|
2020-10-20 21:10:55 +00:00
|
|
|
bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2018-03-30 19:25:37 +00:00
|
|
|
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;
|
2018-03-30 19:25:37 +00:00
|
|
|
|
2019-07-29 09:15:46 +00:00
|
|
|
void clearOldMutations(bool truncate = false);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
2018-11-13 13:48:53 +00:00
|
|
|
// Partition helpers
|
2020-11-12 17:36:02 +00:00
|
|
|
void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context, bool throw_if_noop) override;
|
2020-11-11 13:34:07 +00:00
|
|
|
PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) override;
|
2020-07-28 15:10:36 +00:00
|
|
|
|
2020-11-11 13:34:07 +00:00
|
|
|
void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) override;
|
|
|
|
void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & 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(FutureMergedMutatedPart 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
|
|
|
|
2020-06-23 16:40:58 +00:00
|
|
|
void startBackgroundMovesIfNeeded() override;
|
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
friend class MergeTreeBlockOutputStream;
|
|
|
|
friend class MergeTreeData;
|
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
|
|
|
|
2018-11-26 00:56:50 +00:00
|
|
|
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
2017-06-06 18:36:13 +00:00
|
|
|
* (correctness of names and paths are not checked)
|
|
|
|
* consisting of the specified columns.
|
|
|
|
*
|
2018-11-06 18:25:36 +00:00
|
|
|
* See MergeTreeData constructor for comments on parameters.
|
2017-06-06 18:36:13 +00:00
|
|
|
*/
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageMergeTree(
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2019-10-28 20:12:14 +00:00
|
|
|
const String & relative_data_path_,
|
2019-12-27 16:34:50 +00:00
|
|
|
const StorageInMemoryMetadata & metadata,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool attach,
|
|
|
|
Context & context_,
|
2017-09-08 18:11:09 +00:00
|
|
|
const String & date_column_name,
|
2019-05-03 02:00:57 +00:00
|
|
|
const MergingParams & merging_params_,
|
2019-08-26 14:24:29 +00:00
|
|
|
std::unique_ptr<MergeTreeSettings> settings_,
|
2017-09-19 20:42:42 +00:00
|
|
|
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
|
|
|
};
|
|
|
|
|
|
|
|
}
|