2016-02-14 04:58:47 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-08-21 15:35:29 +00:00
|
|
|
#include <Core/Block.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/types.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
2019-10-10 16:30:30 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2019-03-30 13:44:23 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
2019-06-19 10:07:56 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
|
2019-02-05 14:50:25 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndices.h>
|
2017-08-14 18:16:11 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
2017-09-11 17:55:41 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreePartition.h>
|
2018-03-21 20:21:34 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
2019-04-15 09:30:45 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
|
2019-12-18 15:54:45 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIOSettings.h>
|
2018-06-04 16:20:02 +00:00
|
|
|
#include <Storages/MergeTree/KeyCondition.h>
|
2021-11-08 18:56:09 +00:00
|
|
|
#include <Interpreters/TransactionVersionMetadata.h>
|
2021-03-09 17:25:23 +00:00
|
|
|
#include <DataTypes/Serializations/SerializationInfo.h>
|
2018-06-04 16:20:02 +00:00
|
|
|
|
2017-07-28 17:34:02 +00:00
|
|
|
#include <shared_mutex>
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2020-10-09 14:24:10 +00:00
|
|
|
namespace zkutil
|
|
|
|
{
|
|
|
|
class ZooKeeper;
|
|
|
|
using ZooKeeperPtr = std::shared_ptr<ZooKeeper>;
|
|
|
|
}
|
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
struct ColumnSize;
|
|
|
|
class MergeTreeData;
|
2019-12-07 19:30:37 +00:00
|
|
|
struct FutureMergedMutatedPart;
|
2019-12-12 08:57:25 +00:00
|
|
|
class IReservation;
|
|
|
|
using ReservationPtr = std::unique_ptr<IReservation>;
|
2019-10-21 15:33:59 +00:00
|
|
|
|
2020-05-09 21:24:15 +00:00
|
|
|
class IVolume;
|
|
|
|
using VolumePtr = std::shared_ptr<IVolume>;
|
2019-10-21 15:33:59 +00:00
|
|
|
|
|
|
|
class IMergeTreeReader;
|
2019-10-21 17:23:06 +00:00
|
|
|
class IMergeTreeDataPartWriter;
|
2020-05-20 20:16:32 +00:00
|
|
|
class MarkCache;
|
2020-07-09 01:00:16 +00:00
|
|
|
class UncompressedCache;
|
2021-04-08 17:20:45 +00:00
|
|
|
class MergeTreeTransaction;
|
2020-07-09 01:00:16 +00:00
|
|
|
|
2020-02-19 14:07:36 +00:00
|
|
|
/// Description of the data part.
|
2019-10-10 16:30:30 +00:00
|
|
|
class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPart>
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2019-10-10 16:30:30 +00:00
|
|
|
public:
|
2021-03-09 14:46:52 +00:00
|
|
|
static constexpr auto DATA_FILE_EXTENSION = ".bin";
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using Checksums = MergeTreeDataPartChecksums;
|
2017-03-24 13:52:50 +00:00
|
|
|
using Checksum = MergeTreeDataPartChecksums::Checksum;
|
2019-10-10 16:30:30 +00:00
|
|
|
using ValueSizeMap = std::map<std::string, double>;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
2019-10-21 17:23:06 +00:00
|
|
|
using MergeTreeWriterPtr = std::unique_ptr<IMergeTreeDataPartWriter>;
|
2019-10-21 15:33:59 +00:00
|
|
|
|
2019-12-03 00:23:11 +00:00
|
|
|
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
2021-03-09 14:46:52 +00:00
|
|
|
using NameToNumber = std::unordered_map<std::string, size_t>;
|
2019-12-03 00:23:11 +00:00
|
|
|
|
2021-10-08 13:13:56 +00:00
|
|
|
using IndexSizeByName = std::unordered_map<std::string, ColumnSize>;
|
|
|
|
|
2020-02-10 20:27:06 +00:00
|
|
|
using Type = MergeTreeDataPartType;
|
|
|
|
|
|
|
|
|
|
|
|
IMergeTreeDataPart(
|
|
|
|
const MergeTreeData & storage_,
|
|
|
|
const String & name_,
|
|
|
|
const MergeTreePartInfo & info_,
|
2020-05-09 21:24:15 +00:00
|
|
|
const VolumePtr & volume,
|
2020-02-10 20:27:06 +00:00
|
|
|
const std::optional<String> & relative_path,
|
2021-02-10 14:12:49 +00:00
|
|
|
Type part_type_,
|
|
|
|
const IMergeTreeDataPart * parent_part_);
|
2020-02-10 20:27:06 +00:00
|
|
|
|
|
|
|
IMergeTreeDataPart(
|
2021-03-12 16:33:41 +00:00
|
|
|
const MergeTreeData & storage_,
|
2020-02-10 20:27:06 +00:00
|
|
|
const String & name_,
|
2020-05-09 21:24:15 +00:00
|
|
|
const VolumePtr & volume,
|
2020-02-10 20:27:06 +00:00
|
|
|
const std::optional<String> & relative_path,
|
2021-02-10 14:12:49 +00:00
|
|
|
Type part_type_,
|
|
|
|
const IMergeTreeDataPart * parent_part_);
|
2020-02-10 20:27:06 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
virtual MergeTreeReaderPtr getReader(
|
|
|
|
const NamesAndTypesList & columns_,
|
2020-06-17 16:39:58 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2019-10-10 16:30:30 +00:00
|
|
|
const MarkRanges & mark_ranges,
|
|
|
|
UncompressedCache * uncompressed_cache,
|
|
|
|
MarkCache * mark_cache,
|
2019-12-18 15:54:45 +00:00
|
|
|
const MergeTreeReaderSettings & reader_settings_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const ValueSizeMap & avg_value_size_hints_ = ValueSizeMap{},
|
|
|
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = ReadBufferFromFileBase::ProfileCallback{}) const = 0;
|
2019-10-20 23:36:27 +00:00
|
|
|
|
|
|
|
virtual MergeTreeWriterPtr getWriter(
|
|
|
|
const NamesAndTypesList & columns_list,
|
2020-06-17 12:39:20 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2019-11-05 11:53:22 +00:00
|
|
|
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
2019-10-20 23:36:27 +00:00
|
|
|
const CompressionCodecPtr & default_codec_,
|
2019-12-18 15:54:45 +00:00
|
|
|
const MergeTreeWriterSettings & writer_settings,
|
2019-11-05 11:53:22 +00:00
|
|
|
const MergeTreeIndexGranularity & computed_index_granularity = {}) const = 0;
|
2019-12-18 16:41:11 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
virtual bool isStoredOnDisk() const = 0;
|
|
|
|
|
2021-10-10 20:22:58 +00:00
|
|
|
virtual bool isStoredOnRemoteDisk() const = 0;
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
virtual bool supportsVerticalMerge() const { return false; }
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2018-03-26 14:18:04 +00:00
|
|
|
/// NOTE: Returns zeros if column files are not found in checksums.
|
2020-03-23 13:40:14 +00:00
|
|
|
/// Otherwise return information about column size on disk.
|
2021-12-09 10:39:28 +00:00
|
|
|
ColumnSize getColumnSize(const String & column_name) const;
|
2018-03-26 14:18:04 +00:00
|
|
|
|
2021-10-08 13:13:56 +00:00
|
|
|
/// NOTE: Returns zeros if secondary indexes are not found in checksums.
|
|
|
|
/// Otherwise return information about secondary index size on disk.
|
|
|
|
IndexSize getSecondaryIndexSize(const String & secondary_index_name) const;
|
|
|
|
|
2020-03-23 13:40:14 +00:00
|
|
|
/// Return information about column size on disk for all columns in part
|
2020-03-23 12:19:43 +00:00
|
|
|
ColumnSize getTotalColumnsSize() const { return total_columns_size; }
|
2018-03-26 14:18:04 +00:00
|
|
|
|
2021-10-08 13:13:56 +00:00
|
|
|
/// Return information about secondary indexes size on disk for all indexes in part
|
|
|
|
IndexSize getTotalSeconaryIndicesSize() const { return total_secondary_indices_size; }
|
|
|
|
|
2019-12-19 13:10:57 +00:00
|
|
|
virtual String getFileNameForColumn(const NameAndTypePair & column) const = 0;
|
|
|
|
|
2019-12-03 00:23:11 +00:00
|
|
|
virtual ~IMergeTreeDataPart();
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
using ColumnToSize = std::map<std::string, UInt64>;
|
2020-06-03 21:30:10 +00:00
|
|
|
/// Populates columns_to_size map (compressed size).
|
2020-03-23 12:19:43 +00:00
|
|
|
void accumulateColumnSizes(ColumnToSize & /* column_to_size */) const;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2020-01-14 13:23:51 +00:00
|
|
|
Type getType() const { return part_type; }
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2020-02-11 13:41:26 +00:00
|
|
|
String getTypeName() const { return getType().toString(); }
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2022-01-21 00:20:41 +00:00
|
|
|
void setColumns(const NamesAndTypesList & new_columns);
|
2020-01-16 16:15:01 +00:00
|
|
|
|
|
|
|
const NamesAndTypesList & getColumns() const { return columns; }
|
2022-01-21 00:20:41 +00:00
|
|
|
|
|
|
|
void setSerializationInfos(const SerializationInfoByName & new_infos);
|
2022-01-20 23:22:16 +00:00
|
|
|
|
2021-10-29 17:21:02 +00:00
|
|
|
const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; }
|
2022-01-21 00:20:41 +00:00
|
|
|
|
2021-11-02 03:03:52 +00:00
|
|
|
SerializationPtr getSerialization(const NameAndTypePair & column) const;
|
2020-01-16 16:15:01 +00:00
|
|
|
|
2020-06-03 21:30:10 +00:00
|
|
|
/// Throws an exception if part is not stored in on-disk format.
|
2019-10-10 16:30:30 +00:00
|
|
|
void assertOnDisk() const;
|
|
|
|
|
2021-06-09 12:36:47 +00:00
|
|
|
void remove() const;
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2021-06-11 12:41:48 +00:00
|
|
|
void projectionRemove(const String & parent_to, bool keep_shared_data = false) const;
|
2021-02-10 14:12:49 +00:00
|
|
|
|
2019-12-03 00:23:11 +00:00
|
|
|
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
|
|
|
/// Load checksums from checksums.txt if exists. Load index if required.
|
|
|
|
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency);
|
2017-05-16 15:40:32 +00:00
|
|
|
|
2019-10-28 11:00:29 +00:00
|
|
|
String getMarksFileExtension() const { return index_granularity_info.marks_file_extension; }
|
|
|
|
|
2018-06-04 11:43:09 +00:00
|
|
|
/// Generate the new name for this part according to `new_part_info` and min/max dates from the old name.
|
|
|
|
/// This is useful when you want to change e.g. block numbers or the mutation version of the part.
|
2018-05-23 19:34:37 +00:00
|
|
|
String getNewName(const MergeTreePartInfo & new_part_info) const;
|
|
|
|
|
2020-02-10 20:27:06 +00:00
|
|
|
/// Returns column position in part structure or std::nullopt if it's missing in part.
|
2020-05-15 10:26:44 +00:00
|
|
|
///
|
|
|
|
/// NOTE: Doesn't take column renames into account, if some column renames
|
|
|
|
/// take place, you must take original name of column for this part from
|
|
|
|
/// storage and pass it to this method.
|
2019-12-03 00:23:11 +00:00
|
|
|
std::optional<size_t> getColumnPosition(const String & column_name) const;
|
2019-10-31 14:44:17 +00:00
|
|
|
|
2020-01-16 16:15:01 +00:00
|
|
|
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
|
|
|
/// If no checksums are present returns the name of the first physically existing column.
|
2020-10-27 11:04:03 +00:00
|
|
|
String getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const;
|
2020-01-16 16:15:01 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); }
|
2017-08-14 18:16:11 +00:00
|
|
|
|
2021-03-22 22:16:41 +00:00
|
|
|
/// If the partition key includes date column (a common case), this function will return min and max values for that column.
|
|
|
|
std::pair<DayNum, DayNum> getMinMaxDate() const;
|
2017-05-16 15:40:32 +00:00
|
|
|
|
2021-03-22 22:16:41 +00:00
|
|
|
/// otherwise, if the partition key includes dateTime column (also a common case), this function will return min and max values for that column.
|
|
|
|
std::pair<time_t, time_t> getMinMaxTime() const;
|
2018-12-10 10:20:19 +00:00
|
|
|
|
2018-05-23 19:34:37 +00:00
|
|
|
bool isEmpty() const { return rows_count == 0; }
|
|
|
|
|
2021-04-01 08:07:56 +00:00
|
|
|
/// Compute part block id for zero level part. Otherwise throws an exception.
|
insert_deduplication_token setting for INSERT statement
The setting allows a user to provide own deduplication semantic in Replicated*MergeTree
If provided, it's used instead of data digest to generate block ID
So, for example, by providing a unique value for the setting in each INSERT statement,
user can avoid the same inserted data being deduplicated
Inserting data within the same INSERT statement are split into blocks
according to the *insert_block_size* settings
(max_insert_block_size, min_insert_block_size_rows, min_insert_block_size_bytes).
Each block with the same INSERT statement will get an ordinal number.
The ordinal number is added to insert_deduplication_token to get block dedup token
i.e. <token>_0, <token>_1, ... Deduplication is done per block
So, to guarantee deduplication for two same INSERT queries,
dedup token and number of blocks to have to be the same
Issue: #7461
2021-11-21 20:39:42 +00:00
|
|
|
/// If token is not empty, block id is calculated based on it instead of block data
|
2022-01-03 23:04:56 +00:00
|
|
|
String getZeroLevelPartBlockID(std::string_view token) const;
|
2021-04-01 08:07:56 +00:00
|
|
|
|
2018-10-17 03:13:00 +00:00
|
|
|
const MergeTreeData & storage;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2017-08-14 18:16:11 +00:00
|
|
|
String name;
|
|
|
|
MergeTreePartInfo info;
|
|
|
|
|
2020-10-15 16:17:16 +00:00
|
|
|
/// Part unique identifier.
|
|
|
|
/// The intention is to use it for identifying cases where the same part is
|
|
|
|
/// processed by multiple shards.
|
2020-11-02 14:38:18 +00:00
|
|
|
UUID uuid = UUIDHelpers::Nil;
|
2020-10-15 16:17:16 +00:00
|
|
|
|
2020-05-09 21:24:15 +00:00
|
|
|
VolumePtr volume;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2020-06-03 21:30:10 +00:00
|
|
|
/// A directory path (relative to storage's path) where part data is actually stored
|
|
|
|
/// Examples: 'detached/tmp_fetch_<name>', 'tmp_<name>', '<name>'
|
2021-10-11 11:12:08 +00:00
|
|
|
/// NOTE: Cannot have trailing slash.
|
2017-05-16 15:40:32 +00:00
|
|
|
mutable String relative_path;
|
2020-01-14 13:23:51 +00:00
|
|
|
MergeTreeIndexGranularityInfo index_granularity_info;
|
2017-05-16 15:40:32 +00:00
|
|
|
|
2017-10-24 14:11:53 +00:00
|
|
|
size_t rows_count = 0;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
time_t modification_time = 0;
|
2018-03-14 15:57:13 +00:00
|
|
|
/// When the part is removed from the working set. Changes once.
|
2018-03-14 16:04:19 +00:00
|
|
|
mutable std::atomic<time_t> remove_time { std::numeric_limits<time_t>::max() };
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// If true, the destructor will delete the directory with the part.
|
2021-12-10 13:29:51 +00:00
|
|
|
/// FIXME Why do we need this flag? What's difference from Temporary and DeleteOnDestroy state? Can we get rid of this?
|
2017-04-01 07:20:54 +00:00
|
|
|
bool is_temp = false;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2017-10-06 11:30:57 +00:00
|
|
|
/// If true it means that there are no ZooKeeper node for this part, so it should be deleted only from filesystem
|
|
|
|
bool is_duplicate = false;
|
|
|
|
|
2019-08-21 03:16:59 +00:00
|
|
|
/// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table.
|
|
|
|
mutable std::atomic<bool> is_frozen {false};
|
2019-05-31 04:03:46 +00:00
|
|
|
|
2021-05-19 15:46:27 +00:00
|
|
|
/// Flag for keep S3 data when zero-copy replication over S3 turned on.
|
2021-06-09 12:36:47 +00:00
|
|
|
mutable bool force_keep_shared_data = false;
|
2021-05-19 15:46:27 +00:00
|
|
|
|
2017-09-05 19:03:51 +00:00
|
|
|
/**
|
|
|
|
* Part state is a stage of its lifetime. States are ordered and state of a part could be increased only.
|
|
|
|
* Part state should be modified under data_parts mutex.
|
|
|
|
*
|
|
|
|
* Possible state transitions:
|
2021-12-30 14:27:22 +00:00
|
|
|
* Temporary -> PreActive: we are trying to add a fetched, inserted or merged part to active set
|
|
|
|
* PreActive -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part)
|
|
|
|
* PreActive -> Active: we successfully added a part to active dataset
|
|
|
|
* PreActive -> Outdated: a part was replaced by a covering part or DROP PARTITION
|
2021-05-19 15:46:27 +00:00
|
|
|
* Outdated -> Deleting: a cleaner selected this part for deletion
|
|
|
|
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
|
2021-12-30 14:27:22 +00:00
|
|
|
* Active -> DeleteOnDestroy: if part was moved to another disk
|
2017-09-05 19:03:51 +00:00
|
|
|
*/
|
|
|
|
enum class State
|
|
|
|
{
|
2019-09-04 18:26:18 +00:00
|
|
|
Temporary, /// the part is generating now, it is not in data_parts list
|
2021-12-30 14:27:22 +00:00
|
|
|
PreActive, /// the part is in data_parts, but not used for SELECTs
|
|
|
|
Active, /// active data part, used by current and upcoming SELECTs
|
2019-09-04 18:26:18 +00:00
|
|
|
Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes
|
|
|
|
Deleting, /// not active data part with identity refcounter, it is deleting right now by a cleaner
|
|
|
|
DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor
|
2017-09-05 19:03:51 +00:00
|
|
|
};
|
|
|
|
|
2019-04-15 09:30:45 +00:00
|
|
|
using TTLInfo = MergeTreeDataPartTTLInfo;
|
|
|
|
using TTLInfos = MergeTreeDataPartTTLInfos;
|
|
|
|
|
|
|
|
TTLInfos ttl_infos;
|
|
|
|
|
2017-09-21 21:51:17 +00:00
|
|
|
/// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex
|
2021-01-15 12:15:13 +00:00
|
|
|
void setState(State new_state) const;
|
|
|
|
State getState() const;
|
2017-09-05 19:03:51 +00:00
|
|
|
|
2021-09-06 14:24:03 +00:00
|
|
|
static constexpr std::string_view stateString(State state) { return magic_enum::enum_name(state); }
|
|
|
|
constexpr std::string_view stateString() const { return stateString(state); }
|
2017-09-11 22:40:51 +00:00
|
|
|
|
2021-09-06 14:24:03 +00:00
|
|
|
String getNameWithState() const { return fmt::format("{} (state {})", name, stateString()); }
|
2017-09-11 22:40:51 +00:00
|
|
|
|
|
|
|
/// Returns true if state of part is one of affordable_states
|
2017-09-21 21:51:17 +00:00
|
|
|
bool checkState(const std::initializer_list<State> & affordable_states) const
|
2017-09-11 22:40:51 +00:00
|
|
|
{
|
|
|
|
for (auto affordable_state : affordable_states)
|
|
|
|
{
|
|
|
|
if (state == affordable_state)
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Throws an exception if state of the part is not in affordable_states
|
2017-11-20 19:33:12 +00:00
|
|
|
void assertState(const std::initializer_list<State> & affordable_states) const;
|
2017-09-11 22:40:51 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Primary key (correspond to primary.idx file).
|
|
|
|
/// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple.
|
|
|
|
/// Note that marks (also correspond to primary key) is not always in RAM, but cached. See MarkCache.h.
|
|
|
|
using Index = Columns;
|
|
|
|
Index index;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2017-09-11 17:55:41 +00:00
|
|
|
MergeTreePartition partition;
|
2017-08-31 15:40:34 +00:00
|
|
|
|
2018-11-15 14:06:54 +00:00
|
|
|
/// Amount of rows between marks
|
|
|
|
/// As index always loaded into memory
|
2019-03-30 13:44:23 +00:00
|
|
|
MergeTreeIndexGranularity index_granularity;
|
2018-11-15 14:06:54 +00:00
|
|
|
|
2017-08-31 15:40:34 +00:00
|
|
|
/// Index that for each part stores min and max values of a set of columns. This allows quickly excluding
|
|
|
|
/// parts based on conditions on these columns imposed by a query.
|
|
|
|
/// Currently this index is built using only columns required by partition expression, but in principle it
|
|
|
|
/// can be built using any set of columns.
|
|
|
|
struct MinMaxIndex
|
|
|
|
{
|
2018-06-04 16:20:02 +00:00
|
|
|
/// A direct product of ranges for each key column. See Storages/MergeTree/KeyCondition.cpp for details.
|
2020-03-10 14:56:55 +00:00
|
|
|
std::vector<Range> hyperrectangle;
|
2017-08-31 15:40:34 +00:00
|
|
|
bool initialized = false;
|
|
|
|
|
|
|
|
public:
|
|
|
|
MinMaxIndex() = default;
|
|
|
|
|
|
|
|
/// For month-based partitioning.
|
2018-05-25 13:29:15 +00:00
|
|
|
MinMaxIndex(DayNum min_date, DayNum max_date)
|
2020-03-10 14:56:55 +00:00
|
|
|
: hyperrectangle(1, Range(min_date, true, max_date, true))
|
2017-08-31 15:40:34 +00:00
|
|
|
, initialized(true)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
|
|
|
|
|
2020-03-10 14:39:30 +00:00
|
|
|
void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path);
|
2022-02-01 10:36:51 +00:00
|
|
|
[[nodiscard]] WrittenFiles store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
|
|
|
|
[[nodiscard]] WrittenFiles store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
|
2017-08-31 15:40:34 +00:00
|
|
|
|
|
|
|
void update(const Block & block, const Names & column_names);
|
|
|
|
void merge(const MinMaxIndex & other);
|
|
|
|
};
|
|
|
|
|
2021-09-16 21:19:58 +00:00
|
|
|
using MinMaxIndexPtr = std::shared_ptr<MinMaxIndex>;
|
|
|
|
|
|
|
|
MinMaxIndexPtr minmax_idx;
|
2017-08-18 19:46:26 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Checksums checksums;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2019-04-15 09:30:45 +00:00
|
|
|
/// Columns with values, that all have been zeroed by expired ttl
|
2019-10-10 16:30:30 +00:00
|
|
|
NameSet expired_columns;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2020-08-26 15:29:46 +00:00
|
|
|
CompressionCodecPtr default_codec;
|
|
|
|
|
2022-01-28 17:47:37 +00:00
|
|
|
mutable VersionMetadata version;
|
2021-03-31 17:55:04 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
/// For data in RAM ('index')
|
|
|
|
UInt64 getIndexSizeInBytes() const;
|
|
|
|
UInt64 getIndexSizeInAllocatedBytes() const;
|
|
|
|
UInt64 getMarksCount() const;
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-23 13:32:02 +00:00
|
|
|
UInt64 getBytesOnDisk() const { return bytes_on_disk; }
|
|
|
|
void setBytesOnDisk(UInt64 bytes_on_disk_) { bytes_on_disk = bytes_on_disk_; }
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
size_t getFileSizeOrZero(const String & file_name) const;
|
2020-06-03 21:30:10 +00:00
|
|
|
|
|
|
|
/// Returns path to part dir relatively to disk mount point
|
2020-02-27 16:47:40 +00:00
|
|
|
String getFullRelativePath() const;
|
2020-06-03 21:30:10 +00:00
|
|
|
|
|
|
|
/// Returns full path to part dir
|
2019-10-10 16:30:30 +00:00
|
|
|
String getFullPath() const;
|
2020-05-29 15:02:12 +00:00
|
|
|
|
2020-06-03 21:30:10 +00:00
|
|
|
/// Moves a part to detached/ directory and adds prefix to its name
|
2018-05-21 13:49:54 +00:00
|
|
|
void renameToDetached(const String & prefix) const;
|
2020-06-03 21:30:10 +00:00
|
|
|
|
2020-06-16 03:37:55 +00:00
|
|
|
/// Makes checks and move part to new directory
|
|
|
|
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
|
2020-06-03 22:00:02 +00:00
|
|
|
virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const;
|
2020-06-16 03:37:55 +00:00
|
|
|
|
2021-12-27 16:27:06 +00:00
|
|
|
/// Cleanup shared locks made with old name after part renaming
|
2021-12-01 13:11:26 +00:00
|
|
|
virtual void cleanupOldName(const String & old_part_name) const;
|
|
|
|
|
2020-06-03 21:30:10 +00:00
|
|
|
/// Makes clone of a part in detached/ directory via hard links
|
2020-06-26 11:30:23 +00:00
|
|
|
virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2020-10-20 01:24:30 +00:00
|
|
|
/// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk
|
2020-10-15 13:55:13 +00:00
|
|
|
void makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const;
|
2019-06-07 19:16:42 +00:00
|
|
|
|
2020-05-15 10:26:44 +00:00
|
|
|
/// Checks that .bin and .mrk files exist.
|
|
|
|
///
|
|
|
|
/// NOTE: Doesn't take column renames into account, if some column renames
|
|
|
|
/// take place, you must take original name of column for this part from
|
|
|
|
/// storage and pass it to this method.
|
2020-09-14 11:22:17 +00:00
|
|
|
virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; }
|
2017-01-17 20:49:16 +00:00
|
|
|
|
2020-10-20 15:10:24 +00:00
|
|
|
/// Returns true if this part shall participate in merges according to
|
|
|
|
/// settings of given storage policy.
|
|
|
|
bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const;
|
2017-01-17 20:49:16 +00:00
|
|
|
|
2020-06-03 21:30:10 +00:00
|
|
|
/// Calculate the total size of the entire directory with all the files
|
2020-02-28 17:14:55 +00:00
|
|
|
static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from);
|
2021-10-08 13:13:56 +00:00
|
|
|
|
|
|
|
/// Calculate column and secondary indices sizes on disk.
|
|
|
|
void calculateColumnsAndSecondaryIndicesSizesOnDisk();
|
2017-08-16 19:22:49 +00:00
|
|
|
|
2021-12-01 14:24:26 +00:00
|
|
|
String getRelativePathForPrefix(const String & prefix, bool detached = false) const;
|
2020-06-03 09:51:23 +00:00
|
|
|
|
2021-02-10 14:12:49 +00:00
|
|
|
bool isProjectionPart() const { return parent_part != nullptr; }
|
|
|
|
|
|
|
|
const IMergeTreeDataPart * getParentPart() const { return parent_part; }
|
|
|
|
|
|
|
|
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & getProjectionParts() const { return projection_parts; }
|
|
|
|
|
|
|
|
void addProjectionPart(const String & projection_name, std::shared_ptr<IMergeTreeDataPart> && projection_part)
|
|
|
|
{
|
|
|
|
projection_parts.emplace(projection_name, std::move(projection_part));
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasProjection(const String & projection_name) const
|
|
|
|
{
|
|
|
|
return projection_parts.find(projection_name) != projection_parts.end();
|
|
|
|
}
|
|
|
|
|
|
|
|
void loadProjections(bool require_columns_checksums, bool check_consistency);
|
2020-08-26 15:29:46 +00:00
|
|
|
|
2021-07-13 09:52:24 +00:00
|
|
|
/// Return set of metadata file names without checksums. For example,
|
2020-08-26 15:29:46 +00:00
|
|
|
/// columns.txt or checksums.txt itself.
|
|
|
|
NameSet getFileNamesWithoutChecksums() const;
|
|
|
|
|
|
|
|
/// File with compression codec name which was used to compress part columns
|
|
|
|
/// by default. Some columns may have their own compression codecs, but
|
|
|
|
/// default will be stored in this file.
|
|
|
|
static inline constexpr auto DEFAULT_COMPRESSION_CODEC_FILE_NAME = "default_compression_codec.txt";
|
|
|
|
|
|
|
|
static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME = "delete-on-destroy.txt";
|
|
|
|
|
2020-10-15 16:17:16 +00:00
|
|
|
static inline constexpr auto UUID_FILE_NAME = "uuid.txt";
|
|
|
|
|
2021-12-08 15:29:00 +00:00
|
|
|
/// File that contains information about kinds of serialization of columns
|
|
|
|
/// and information that helps to choose kind of serialization later during merging
|
|
|
|
/// (number of rows, number of rows with default values, etc).
|
2021-12-08 18:59:36 +00:00
|
|
|
static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.json";
|
2021-03-12 16:33:41 +00:00
|
|
|
|
2021-12-30 13:15:28 +00:00
|
|
|
static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt";
|
|
|
|
|
2020-09-03 08:59:41 +00:00
|
|
|
/// Checks that all TTLs (table min/max, column ttls, so on) for part
|
|
|
|
/// calculated. Part without calculated TTL may exist if TTL was added after
|
|
|
|
/// part creation (using alter query with materialize_ttl setting).
|
|
|
|
bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const;
|
|
|
|
|
2021-11-23 13:57:24 +00:00
|
|
|
/// Return some uniq string for file.
|
|
|
|
/// Required for distinguish different copies of the same part on remote FS.
|
2020-11-03 08:58:26 +00:00
|
|
|
String getUniqueId() const;
|
|
|
|
|
2021-12-30 13:15:28 +00:00
|
|
|
void storeVersionMetadata() const;
|
|
|
|
void loadVersionMetadata() const;
|
2022-01-10 16:53:50 +00:00
|
|
|
|
2021-11-23 13:57:24 +00:00
|
|
|
/// Return hardlink count for part.
|
|
|
|
/// Required for keep data on remote FS when part has shadow copies.
|
2021-12-27 16:27:06 +00:00
|
|
|
UInt32 getNumberOfRefereneces() const;
|
2021-11-23 13:57:24 +00:00
|
|
|
|
2019-11-18 12:22:27 +00:00
|
|
|
protected:
|
2020-08-26 15:29:46 +00:00
|
|
|
|
2020-03-23 12:19:43 +00:00
|
|
|
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk
|
|
|
|
ColumnSize total_columns_size;
|
|
|
|
|
|
|
|
/// Size for each column, calculated once in calcuateColumnSizesOnDisk
|
|
|
|
ColumnSizeByName columns_sizes;
|
|
|
|
|
2021-10-08 13:13:56 +00:00
|
|
|
ColumnSize total_secondary_indices_size;
|
|
|
|
|
|
|
|
IndexSizeByName secondary_index_sizes;
|
|
|
|
|
2020-03-23 13:32:02 +00:00
|
|
|
/// Total size on disk, not only columns. May not contain size of
|
|
|
|
/// checksums.txt and columns.txt. 0 - if not counted;
|
|
|
|
UInt64 bytes_on_disk{0};
|
|
|
|
|
2020-05-21 09:00:44 +00:00
|
|
|
/// Columns description. Cannot be changed, after part initialization.
|
2020-01-16 16:15:01 +00:00
|
|
|
NamesAndTypesList columns;
|
2021-10-29 17:21:02 +00:00
|
|
|
|
2020-01-17 12:24:27 +00:00
|
|
|
const Type part_type;
|
2020-01-16 16:15:01 +00:00
|
|
|
|
2021-02-10 14:12:49 +00:00
|
|
|
/// Not null when it's a projection part.
|
|
|
|
const IMergeTreeDataPart * parent_part;
|
|
|
|
|
|
|
|
std::map<String, std::shared_ptr<IMergeTreeDataPart>> projection_parts;
|
|
|
|
|
2019-11-18 12:22:27 +00:00
|
|
|
void removeIfNeeded();
|
2020-01-16 16:15:01 +00:00
|
|
|
|
2020-06-03 18:59:18 +00:00
|
|
|
virtual void checkConsistency(bool require_part_metadata) const;
|
2020-01-15 19:16:56 +00:00
|
|
|
void checkConsistencyBase() const;
|
2019-11-18 12:22:27 +00:00
|
|
|
|
2020-03-23 12:19:43 +00:00
|
|
|
/// Fill each_columns_size and total_size with sizes from columns files on
|
|
|
|
/// disk using columns and checksums.
|
2020-06-29 20:36:18 +00:00
|
|
|
virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0;
|
2020-03-23 12:19:43 +00:00
|
|
|
|
2020-05-29 15:02:12 +00:00
|
|
|
String getRelativePathForDetachedPart(const String & prefix) const;
|
|
|
|
|
2021-06-09 12:36:47 +00:00
|
|
|
std::optional<bool> keepSharedDataInDecoupledStorage() const;
|
|
|
|
|
2017-08-16 19:22:49 +00:00
|
|
|
private:
|
2019-12-25 20:06:16 +00:00
|
|
|
/// In compact parts order of columns is necessary
|
2021-03-09 14:46:52 +00:00
|
|
|
NameToNumber column_name_to_position;
|
2019-11-18 12:22:27 +00:00
|
|
|
|
2021-12-08 15:29:00 +00:00
|
|
|
/// Map from name of column to its serialization info.
|
2021-11-02 03:03:52 +00:00
|
|
|
SerializationInfoByName serialization_infos;
|
|
|
|
|
2020-10-15 16:17:16 +00:00
|
|
|
/// Reads part unique identifier (if exists) from uuid.txt
|
|
|
|
void loadUUID();
|
|
|
|
|
2019-10-31 14:44:17 +00:00
|
|
|
/// Reads columns names and types from columns.txt
|
|
|
|
void loadColumns(bool require);
|
|
|
|
|
2020-08-08 00:47:03 +00:00
|
|
|
/// If checksums.txt exists, reads file's checksums (and sizes) from it
|
2019-10-31 14:44:17 +00:00
|
|
|
void loadChecksums(bool require);
|
|
|
|
|
|
|
|
/// Loads marks index granularity into memory
|
|
|
|
virtual void loadIndexGranularity();
|
|
|
|
|
|
|
|
/// Loads index file.
|
|
|
|
void loadIndex();
|
|
|
|
|
|
|
|
/// Load rows count for this part from disk (for the newer storage format version).
|
|
|
|
/// For the older format version calculates rows count from the size of a column with a fixed size.
|
|
|
|
void loadRowsCount();
|
|
|
|
|
2020-05-21 09:00:44 +00:00
|
|
|
/// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros
|
2019-10-31 14:44:17 +00:00
|
|
|
void loadTTLInfos();
|
|
|
|
|
|
|
|
void loadPartitionAndMinMaxIndex();
|
2020-08-28 09:07:20 +00:00
|
|
|
|
2021-10-08 13:13:56 +00:00
|
|
|
void calculateColumnsSizesOnDisk();
|
|
|
|
|
|
|
|
void calculateSecondaryIndicesSizesOnDisk();
|
|
|
|
|
2020-08-28 09:07:20 +00:00
|
|
|
/// Load default compression codec from file default_compression_codec.txt
|
|
|
|
/// if it not exists tries to deduce codec from compressed column without
|
|
|
|
/// any specifial compression.
|
|
|
|
void loadDefaultCompressionCodec();
|
|
|
|
|
|
|
|
/// Found column without specific compression and return codec
|
|
|
|
/// for this column with default parameters.
|
|
|
|
CompressionCodecPtr detectDefaultCompressionCodec() const;
|
2021-01-15 12:15:13 +00:00
|
|
|
|
|
|
|
mutable State state{State::Temporary};
|
2016-02-14 04:58:47 +00:00
|
|
|
};
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
using MergeTreeDataPartState = IMergeTreeDataPart::State;
|
2019-10-21 15:33:59 +00:00
|
|
|
using MergeTreeDataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
2020-05-05 15:06:16 +00:00
|
|
|
using MergeTreeMutableDataPartPtr = std::shared_ptr<IMergeTreeDataPart>;
|
2017-11-20 19:33:12 +00:00
|
|
|
|
2019-12-18 13:09:58 +00:00
|
|
|
bool isCompactPart(const MergeTreeDataPartPtr & data_part);
|
|
|
|
bool isWidePart(const MergeTreeDataPartPtr & data_part);
|
2020-04-20 01:38:38 +00:00
|
|
|
bool isInMemoryPart(const MergeTreeDataPartPtr & data_part);
|
2019-12-18 13:09:58 +00:00
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|