2014-03-13 12:48:07 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2018-05-13 00:24:52 +00:00
|
|
|
#include <Storages/MutationCommands.h>
|
2014-12-17 11:50:24 +00:00
|
|
|
#include <atomic>
|
2016-03-01 17:47:53 +00:00
|
|
|
#include <functional>
|
2017-10-06 16:53:55 +00:00
|
|
|
#include <Common/ActionBlocker.h>
|
|
|
|
|
2014-03-13 12:48:07 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2015-04-16 06:12:35 +00:00
|
|
|
class MergeListEntry;
|
2016-11-03 12:00:44 +00:00
|
|
|
class MergeProgressCallback;
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2019-01-13 22:02:33 +00:00
|
|
|
/// Auxiliary struct holding metainformation for the future merged or mutated part.
|
|
|
|
struct FutureMergedMutatedPart
|
|
|
|
{
|
|
|
|
String name;
|
2019-11-24 05:47:39 +00:00
|
|
|
String path;
|
2019-01-13 22:02:33 +00:00
|
|
|
MergeTreePartInfo part_info;
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
|
|
|
|
|
|
|
const MergeTreePartition & getPartition() const { return parts.front()->partition; }
|
|
|
|
|
|
|
|
FutureMergedMutatedPart() = default;
|
|
|
|
explicit FutureMergedMutatedPart(MergeTreeData::DataPartsVector parts_)
|
|
|
|
{
|
|
|
|
assign(std::move(parts_));
|
|
|
|
}
|
|
|
|
|
|
|
|
void assign(MergeTreeData::DataPartsVector parts_);
|
2019-12-09 13:35:02 +00:00
|
|
|
void updatePath(const MergeTreeData & storage, const ReservationPtr & reservation);
|
2019-01-13 22:02:33 +00:00
|
|
|
};
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2019-06-19 17:56:41 +00:00
|
|
|
|
2019-08-16 15:57:19 +00:00
|
|
|
/** Can select parts for background processes and do them.
|
|
|
|
* Currently helps with merges, mutations and moves
|
|
|
|
*/
|
2018-04-20 16:18:16 +00:00
|
|
|
class MergeTreeDataMergerMutator
|
2014-03-13 12:48:07 +00:00
|
|
|
{
|
2016-03-01 17:47:53 +00:00
|
|
|
public:
|
2018-01-12 17:30:21 +00:00
|
|
|
using AllowedMergingPredicate = std::function<bool (const MergeTreeData::DataPartPtr &, const MergeTreeData::DataPartPtr &, String * reason)>;
|
2016-03-01 17:47:53 +00:00
|
|
|
|
2014-03-13 12:48:07 +00:00
|
|
|
public:
|
2019-09-04 16:00:20 +00:00
|
|
|
MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** Get maximum total size of parts to do merge, at current moment of time.
|
|
|
|
* It depends on number of free threads in background_pool and amount of free space in disk.
|
|
|
|
*/
|
2019-06-17 19:41:48 +00:00
|
|
|
UInt64 getMaxSourcePartsSizeForMerge();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** For explicitly passed size of pool and number of used tasks.
|
|
|
|
* This method could be used to calculate threshold depending on number of tasks in replication queue.
|
|
|
|
*/
|
2019-06-17 19:41:48 +00:00
|
|
|
UInt64 getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used);
|
|
|
|
|
|
|
|
/** Get maximum total size of parts to do mutation, at current moment of time.
|
|
|
|
* It depends only on amount of free space in disk.
|
|
|
|
*/
|
|
|
|
UInt64 getMaxSourcePartSizeForMutation();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Selects which parts to merge. Uses a lot of heuristics.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* can_merge - a function that determines if it is possible to merge a pair of adjacent parts.
|
|
|
|
* This function must coordinate merge with inserts and other merges, ensuring that
|
|
|
|
* - Parts between which another part can still appear can not be merged. Refer to METR-7001.
|
|
|
|
* - A part that already merges with something in one place, you can not start to merge into something else in another place.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
bool selectPartsToMerge(
|
2019-01-13 22:02:33 +00:00
|
|
|
FutureMergedMutatedPart & future_part,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool aggressive,
|
|
|
|
size_t max_total_size_to_merge,
|
2018-01-12 17:30:21 +00:00
|
|
|
const AllowedMergingPredicate & can_merge,
|
|
|
|
String * out_disable_reason = nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-06-19 17:56:41 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Select all the parts in the specified partition for merge, if possible.
|
2017-07-04 12:46:31 +00:00
|
|
|
* final - choose to merge even a single part - that is, allow to merge one part "with itself".
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
bool selectAllPartsToMergeWithinPartition(
|
2019-01-13 22:02:33 +00:00
|
|
|
FutureMergedMutatedPart & future_part,
|
2018-12-17 14:10:23 +00:00
|
|
|
UInt64 & available_disk_space,
|
2017-04-01 07:20:54 +00:00
|
|
|
const AllowedMergingPredicate & can_merge,
|
2017-08-14 18:16:11 +00:00
|
|
|
const String & partition_id,
|
2018-01-12 17:30:21 +00:00
|
|
|
bool final,
|
|
|
|
String * out_disable_reason = nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Merge the parts.
|
|
|
|
* If `reservation != nullptr`, now and then reduces the size of the reserved space
|
|
|
|
* is approximately proportional to the amount of data already written.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* Creates and returns a temporary part.
|
2018-04-20 16:18:16 +00:00
|
|
|
* To end the merge, call the function renameMergedTemporaryPart.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* time_of_merge - the time when the merge was assigned.
|
|
|
|
* Important when using ReplicatedGraphiteMergeTree to provide the same merge on replicas.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart(
|
2019-01-13 22:02:33 +00:00
|
|
|
const FutureMergedMutatedPart & future_part,
|
2019-08-19 19:02:20 +00:00
|
|
|
MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge,
|
2019-11-27 09:39:44 +00:00
|
|
|
const ReservationPtr & disk_reservation, bool deduplication, bool force_ttl);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-18 16:07:08 +00:00
|
|
|
/// Mutate a single data part with the specified commands. Will create and return a temporary part.
|
2018-05-13 00:24:52 +00:00
|
|
|
MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart(
|
2019-01-13 22:02:33 +00:00
|
|
|
const FutureMergedMutatedPart & future_part,
|
2018-05-13 00:24:52 +00:00
|
|
|
const std::vector<MutationCommand> & commands,
|
2019-08-21 10:09:29 +00:00
|
|
|
MergeListEntry & merge_entry, const Context & context,
|
2019-11-27 09:39:44 +00:00
|
|
|
const ReservationPtr & disk_reservation,
|
2019-08-21 10:09:29 +00:00
|
|
|
TableStructureReadLockHolder & table_lock_holder);
|
2018-05-13 00:24:52 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData::DataPartPtr renameMergedTemporaryPart(
|
|
|
|
MergeTreeData::MutableDataPartPtr & new_data_part,
|
2017-08-16 19:24:50 +00:00
|
|
|
const MergeTreeData::DataPartsVector & parts,
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData::Transaction * out_transaction = nullptr);
|
|
|
|
|
2019-06-19 17:56:41 +00:00
|
|
|
|
2018-04-20 16:18:16 +00:00
|
|
|
/// The approximate amount of disk space needed for merge or mutation. With a surplus.
|
|
|
|
static size_t estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts);
|
2014-03-13 12:48:07 +00:00
|
|
|
|
2016-01-28 16:06:57 +00:00
|
|
|
private:
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Select all parts belonging to the same partition.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-08-14 18:16:11 +00:00
|
|
|
MergeTreeData::DataPartsVector selectAllPartsFromPartition(const String & partition_id);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2016-08-26 19:50:04 +00:00
|
|
|
public:
|
2018-04-20 16:18:16 +00:00
|
|
|
/** Is used to cancel all merges and mutations. On cancel() call all currently running actions will throw exception soon.
|
|
|
|
* All new attempts to start a merge or mutation will throw an exception until all 'LockHolder' objects will be destroyed.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2019-08-01 15:36:12 +00:00
|
|
|
ActionBlocker merges_blocker;
|
|
|
|
ActionBlocker ttl_merges_blocker;
|
2016-11-03 12:00:44 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
enum class MergeAlgorithm
|
|
|
|
{
|
2017-10-06 16:53:55 +00:00
|
|
|
Horizontal, /// per-row merge of all columns
|
2019-02-05 14:50:25 +00:00
|
|
|
Vertical /// per-row merge of PK and secondary indices columns, per-column gather for non-PK columns
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2016-11-03 12:00:44 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
|
2017-07-04 12:38:53 +00:00
|
|
|
MergeAlgorithm chooseMergeAlgorithm(
|
2019-01-04 12:10:00 +00:00
|
|
|
const MergeTreeData::DataPartsVector & parts,
|
2019-04-15 09:30:45 +00:00
|
|
|
size_t rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const;
|
2016-11-03 12:00:44 +00:00
|
|
|
|
2014-03-13 12:48:07 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData & data;
|
2019-09-04 16:00:20 +00:00
|
|
|
const size_t background_pool_size;
|
2014-03-13 12:48:07 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Logger * log;
|
2014-03-13 12:48:07 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// When the last time you wrote to the log that the disk space was running out (not to write about this too often).
|
2017-04-01 07:20:54 +00:00
|
|
|
time_t disk_space_warning_time = 0;
|
2019-04-15 09:30:45 +00:00
|
|
|
|
|
|
|
/// Last time when TTLMergeSelector has been used
|
|
|
|
time_t last_merge_with_ttl = 0;
|
2014-03-13 12:48:07 +00:00
|
|
|
};
|
|
|
|
|
2016-08-26 19:50:04 +00:00
|
|
|
|
2014-03-13 12:48:07 +00:00
|
|
|
}
|