ClickHouse/src/Storages/MergeTree/IMergeTreeDataPart.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

671 lines
28 KiB
C++
Raw Normal View History

2016-02-14 04:58:47 +00:00
#pragma once
#include <IO/WriteSettings.h>
#include <Core/Block.h>
2021-10-02 07:13:14 +00:00
#include <base/types.h>
#include <Core/NamesAndTypes.h>
2019-10-10 16:30:30 +00:00
#include <Storages/IStorage.h>
2022-07-21 19:50:19 +00:00
#include <Storages/LightweightDeleteDescription.h>
2023-05-25 22:54:54 +00:00
#include <Storages/MergeTree/AlterConversions.h>
2022-08-12 11:03:57 +00:00
#include <Storages/MergeTree/IDataPartStorage.h>
#include <Storages/MergeTree/MergeTreeDataPartState.h>
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
2019-06-19 10:07:56 +00:00
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
#include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/MergeTreePartition.h>
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <Storages/MergeTree/MergeTreeIOSettings.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
2022-07-21 14:47:19 +00:00
#include <Storages/ColumnsDescription.h>
2021-11-08 18:56:09 +00:00
#include <Interpreters/TransactionVersionMetadata.h>
#include <DataTypes/Serializations/SerializationInfo.h>
#include <Storages/MergeTree/IPartMetadataManager.h>
2022-09-05 04:31:19 +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;
class IReservation;
using ReservationPtr = std::unique_ptr<IReservation>;
2019-10-21 15:33:59 +00:00
class IMergeTreeReader;
2019-10-21 17:23:06 +00:00
class IMergeTreeDataPartWriter;
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
enum class DataPartRemovalState
{
NOT_ATTEMPTED,
VISIBLE_TO_TRANSACTIONS,
NON_UNIQUE_OWNERSHIP,
NOT_REACHED_REMOVAL_TIME,
2022-12-15 16:05:15 +00:00
HAS_SKIPPED_MUTATION_PARENT,
REMOVED,
};
2020-02-19 14:07:36 +00:00
/// Description of the data part.
class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPart>, public DataPartStorageHolder
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";
2016-02-14 04:58:47 +00:00
using Checksums = MergeTreeDataPartChecksums;
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
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>;
using IndexSizeByName = std::unordered_map<std::string, ColumnSize>;
using Type = MergeTreeDataPartType;
using uint128 = IPartMetadataManager::uint128;
IMergeTreeDataPart(
const MergeTreeData & storage_,
const String & name_,
const MergeTreePartInfo & info_,
2022-10-22 22:51:59 +00:00
const MutableDataPartStoragePtr & data_part_storage_,
Type part_type_,
const IMergeTreeDataPart * parent_part_);
2019-10-10 16:30:30 +00:00
virtual MergeTreeReaderPtr getReader(
const NamesAndTypesList & columns_,
const StorageMetadataPtr & metadata_snapshot,
2019-10-10 16:30:30 +00:00
const MarkRanges & mark_ranges,
UncompressedCache * uncompressed_cache,
MarkCache * mark_cache,
2023-05-25 22:54:54 +00:00
const AlterConversionsPtr & alter_conversions,
const MergeTreeReaderSettings & reader_settings_,
2022-04-21 19:19:13 +00:00
const ValueSizeMap & avg_value_size_hints_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_) 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_,
const MergeTreeWriterSettings & writer_settings,
2022-10-22 22:51:59 +00:00
const MergeTreeIndexGranularity & computed_index_granularity) = 0;
2019-10-10 16:30:30 +00:00
virtual bool isStoredOnDisk() const = 0;
virtual bool isStoredOnRemoteDisk() const = 0;
virtual bool isStoredOnRemoteDiskWithZeroCopySupport() const = 0;
/// 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.
ColumnSize getColumnSize(const String & column_name) const;
virtual std::optional<time_t> getColumnModificationTime(const String & column_name) const = 0;
/// 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; }
/// 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;
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; }
MergeTreeDataPartFormat getFormat() const { return {part_type, getDataPartStorage().getType()}; }
2019-10-10 16:30:30 +00:00
String getTypeName() const { return getType().toString(); }
2019-10-10 16:30:30 +00:00
/// We could have separate method like setMetadata, but it's much more convenient to set it up with columns
void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos, int32_t metadata_version_);
/// Version of metadata for part (columns, pk and so on)
int32_t getMetadataVersion() const { return metadata_version; }
2020-01-16 16:15:01 +00:00
const NamesAndTypesList & getColumns() const { return columns; }
2022-07-27 15:35:09 +00:00
const ColumnsDescription & getColumnsDescription() const { return columns_description; }
2023-01-25 22:19:05 +00:00
const ColumnsDescription & getColumnsDescriptionWithCollectedNested() const { return columns_description_with_collected_nested; }
2022-01-21 00:20:41 +00:00
2022-07-21 14:47:19 +00:00
NameAndTypePair getColumn(const String & name) const;
std::optional<NameAndTypePair> tryGetColumn(const String & column_name) const;
2021-10-29 17:21:02 +00:00
const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; }
2022-01-21 00:20:41 +00:00
2022-07-27 14:05:16 +00:00
SerializationPtr getSerialization(const String & column_name) const;
SerializationPtr tryGetSerialization(const String & column_name) 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;
void remove();
2019-10-16 18:27:53 +00:00
/// Initialize columns (from columns.txt if exists, or create from column files if not).
/// Load various metadata into memory: checksums from checksums.txt, index if required, etc.
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency);
2021-12-08 02:40:59 +00:00
void appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection = false) const;
2022-09-05 05:26:58 +00:00
String getMarksFileExtension() const { return index_granularity_info.mark_type.getFileExtension(); }
2019-10-28 11:00:29 +00:00
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;
/// 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.
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.
2022-07-21 14:47:19 +00:00
String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) 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); }
/// 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;
/// 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-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.
/// If token is not empty, block id is calculated based on it instead of block data
String getZeroLevelPartBlockID(std::string_view token) const;
2021-04-01 08:07:56 +00:00
2023-07-08 12:58:38 +00:00
void setName(const String & new_name);
2018-10-17 03:13:00 +00:00
const MergeTreeData & storage;
2016-02-14 04:58:47 +00:00
2023-07-08 12:58:38 +00:00
private:
String mutable_name;
public:
const String & name; // const ref to private mutable_name
MergeTreePartInfo info;
/// Part unique identifier.
/// The intention is to use it for identifying cases where the same part is
/// processed by multiple shards.
UUID uuid = UUIDHelpers::Nil;
2020-01-14 13:23:51 +00:00
MergeTreeIndexGranularityInfo index_granularity_info;
size_t rows_count = 0;
2019-10-10 16:30:30 +00:00
2016-02-14 04:58:47 +00:00
time_t modification_time = 0;
/// When the part is removed from the working set. Changes once.
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.
/// FIXME Why do we need this flag? What's difference from Temporary and DeleteOnDestroy state? Can we get rid of this?
2016-02-14 04:58:47 +00:00
bool is_temp = false;
2023-05-12 11:00:56 +00:00
/// This type and the field remove_tmp_policy is used as a hint
/// to help avoid communication with keeper when temporary part is deleting.
/// The common procedure is to ask the keeper with unlock request to release a references to the blobs.
/// And then follow the keeper answer decide remove or preserve the blobs in that part from s3.
/// However in some special cases Clickhouse can make a decision without asking keeper.
2023-05-10 21:27:48 +00:00
enum class BlobsRemovalPolicyForTemporaryParts
{
2023-05-12 11:00:56 +00:00
/// decision about removing blobs is determined by keeper, the common case
2023-05-10 21:27:48 +00:00
ASK_KEEPER,
2023-05-12 11:00:56 +00:00
/// is set when Clickhouse is sure that the blobs in the part are belong only to it, other replicas have not seen them yet
2023-05-10 21:27:48 +00:00
REMOVE_BLOBS,
2023-05-12 11:00:56 +00:00
/// is set when Clickhouse is sure that the blobs belong to other replica and current replica has not locked them on s3 yet
2023-05-10 21:27:48 +00:00
PRESERVE_BLOBS,
};
BlobsRemovalPolicyForTemporaryParts remove_tmp_policy = BlobsRemovalPolicyForTemporaryParts::ASK_KEEPER;
/// 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;
/// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table.
mutable std::atomic<bool> is_frozen {false};
/// Indicates that the part was marked Outdated by PartCheckThread because the part was not committed to ZooKeeper
mutable bool is_unexpected_local_part = false;
/// Indicates that the part was detached and marked Outdated because it's broken
mutable std::atomic_bool was_removed_as_broken = false;
/// 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;
2023-06-27 18:18:38 +00:00
/// Some old parts don't have metadata version, so we set it to the current table's version when loading the part
bool old_part_with_no_metadata_version_on_disk = false;
using TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos;
2021-12-08 02:40:59 +00:00
mutable TTLInfos ttl_infos;
/// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex
2022-08-12 11:03:57 +00:00
void setState(MergeTreeDataPartState new_state) const;
MergeTreeDataPartState getState() const;
2022-08-12 11:03:57 +00:00
static constexpr std::string_view stateString(MergeTreeDataPartState state) { return magic_enum::enum_name(state); }
constexpr std::string_view stateString() const { return stateString(state); }
String getNameWithState() const { return fmt::format("{} (state {})", name, stateString()); }
/// Returns true if state of part is one of affordable_states
2022-08-12 11:03:57 +00:00
bool checkState(const std::initializer_list<MergeTreeDataPartState> & affordable_states) const
{
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
2022-08-12 11:03:57 +00:00
void assertState(const std::initializer_list<MergeTreeDataPartState> & affordable_states) const;
/// 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;
2016-02-14 04:58:47 +00:00
Index index;
MergeTreePartition partition;
2018-11-15 14:06:54 +00:00
/// Amount of rows between marks
/// As index always loaded into memory
MergeTreeIndexGranularity index_granularity;
2018-11-15 14:06:54 +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
{
/// 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;
bool initialized = false;
public:
MinMaxIndex() = default;
/// For month-based partitioning.
MinMaxIndex(DayNum min_date, DayNum max_date)
2020-03-10 14:56:55 +00:00
: hyperrectangle(1, Range(min_date, true, max_date, true))
, initialized(true)
{
}
void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager);
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
[[nodiscard]] WrittenFiles store(const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & checksums) const;
[[nodiscard]] WrittenFiles store(const Names & column_names, const DataTypes & data_types, IDataPartStorage & part_storage, Checksums & checksums) const;
void update(const Block & block, const Names & column_names);
void merge(const MinMaxIndex & other);
2021-12-08 02:40:59 +00:00
static void appendFiles(const MergeTreeData & data, Strings & files);
};
using MinMaxIndexPtr = std::shared_ptr<MinMaxIndex>;
MinMaxIndexPtr minmax_idx;
2016-02-14 04:58:47 +00:00
Checksums checksums;
/// 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
CompressionCodecPtr default_codec;
mutable VersionMetadata version;
2021-03-31 17:55:04 +00:00
/// Version of part metadata (columns, pk and so on). Managed properly only for replicated merge tree.
int32_t metadata_version;
2019-10-10 16:30:30 +00:00
/// For data in RAM ('index')
UInt64 getIndexSizeInBytes() const;
UInt64 getIndexSizeInAllocatedBytes() const;
UInt64 getMarksCount() const;
2023-06-27 16:45:34 +00:00
UInt64 getIndexSizeFromFile() const;
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
/// Moves a part to detached/ directory and adds prefix to its name
void renameToDetached(const String & prefix);
2020-06-03 21:30:10 +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
virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists);
2020-06-03 21:30:10 +00:00
/// Makes clone of a part in detached/ directory via hard links
2022-12-22 13:31:42 +00:00
virtual DataPartStoragePtr 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
2022-10-22 22:51:59 +00:00
MutableDataPartStoragePtr 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.
virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; }
/// Returns true if this part shall participate in merges according to
/// settings of given storage policy.
bool shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const;
/// Calculate column and secondary indices sizes on disk.
void calculateColumnsAndSecondaryIndicesSizesOnDisk();
2017-08-16 19:22:49 +00:00
2022-09-30 18:09:18 +00:00
std::optional<String> getRelativePathForPrefix(const String & prefix, bool detached = false, bool broken = false) const;
2020-06-03 09:51:23 +00:00
bool isProjectionPart() const { return parent_part != nullptr; }
const IMergeTreeDataPart * getParentPart() const { return parent_part; }
2023-07-08 12:58:38 +00:00
String getParentPartName() const { return parent_part_name; }
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & getProjectionParts() const { return projection_parts; }
MergeTreeDataPartBuilder getProjectionPartBuilder(const String & projection_name, bool is_temp_projection = false);
void addProjectionPart(const String & projection_name, std::shared_ptr<IMergeTreeDataPart> && projection_part);
bool hasProjection(const String & projection_name) const { return projection_parts.contains(projection_name); }
void loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false);
2021-07-13 09:52:24 +00:00
/// Return set of metadata file names without checksums. For example,
/// 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";
/// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed.
static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED = "delete-on-destroy.txt";
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).
static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.json";
2021-03-12 16:33:41 +00:00
/// Version used for transactions.
2021-12-30 13:15:28 +00:00
static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt";
static inline constexpr auto METADATA_VERSION_FILE_NAME = "metadata_version.txt";
/// One of part files which is used to check how many references (I'd like
/// to say hardlinks, but it will confuse even more) we have for the part
/// for zero copy replication. Sadly it's very complex.
///
/// NOTE: it's not a random "metadata" file for part like 'columns.txt'. If
/// two relative parts (for example all_1_1_0 and all_1_1_0_100) has equal
/// checksums.txt it means that one part was obtained by FREEZE operation or
/// it was mutation without any change for source part. In this case we
/// really don't need to remove data from remote FS and need only decrement
/// reference counter locally.
static inline constexpr auto FILE_FOR_REFERENCES_CHECK = "checksums.txt";
/// 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;
2022-06-30 20:51:27 +00:00
/// Return some uniq string for file.
/// Required for distinguish different copies of the same part on remote FS.
String getUniqueId() const;
2022-02-17 21:26:37 +00:00
/// Ensures that creation_tid was correctly set after part creation.
2022-02-14 19:50:08 +00:00
void assertHasVersionMetadata(MergeTreeTransaction * txn) const;
2022-02-17 21:26:37 +00:00
/// [Re]writes file with transactional metadata on disk
2022-08-25 13:10:14 +00:00
void storeVersionMetadata(bool force = false) const;
2022-02-17 21:26:37 +00:00
/// Appends the corresponding CSN to file on disk (without fsync)
void appendCSNToVersionMetadata(VersionMetadata::WhichCSN which_csn) const;
2022-03-08 19:11:47 +00:00
/// Appends removal TID to file on disk (with fsync)
void appendRemovalTIDToVersionMetadata(bool clear = false) const;
2022-02-17 21:26:37 +00:00
/// Loads transactional metadata from disk
2021-12-30 13:15:28 +00:00
void loadVersionMetadata() const;
2022-02-17 21:26:37 +00:00
/// Returns true if part was created or removed by a transaction
bool wasInvolvedInTransaction() const;
/// Moar hardening: this method is supposed to be used for debug assertions
bool assertHasValidVersionMetadata() const;
/// 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;
2022-01-07 10:37:08 +00:00
/// Get checksums of metadata file in part directory
2022-06-03 18:49:12 +00:00
IMergeTreeDataPart::uint128 getActualChecksumByFile(const String & file_name) const;
2022-01-07 10:37:08 +00:00
/// Check metadata in cache is consistent with actual metadata on disk(if use_metadata_cache is true)
std::unordered_map<String, uint128> checkMetadata() const;
/// True if the part supports lightweight delete mutate.
bool supportLightweightDeleteMutate() const;
/// True if here is lightweight deleted mask file in part.
2022-07-25 14:15:15 +00:00
bool hasLightweightDelete() const { return columns.contains(LightweightDeleteDescription::FILTER_COLUMN.name); }
2022-10-23 22:29:24 +00:00
void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings);
/// "delete-on-destroy.txt" is deprecated. It is no longer being created, only is removed.
/// TODO: remove this method after some time.
2022-10-23 22:29:24 +00:00
void removeDeleteOnDestroyMarker();
/// It may look like a stupid joke. but these two methods are absolutely unrelated.
/// This one is about removing file with metadata about part version (for transactions)
2022-10-23 22:29:24 +00:00
void removeVersionMetadata();
/// This one is about removing file with version of part's metadata (columns, pk and so on)
void removeMetadataVersion();
2022-10-23 22:29:24 +00:00
mutable std::atomic<DataPartRemovalState> removal_state = DataPartRemovalState::NOT_ATTEMPTED;
mutable std::atomic<time_t> last_removal_attemp_time = 0;
2019-11-18 12:22:27 +00:00
protected:
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;
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
/// Not null when it's a projection part.
const IMergeTreeDataPart * parent_part;
2023-07-08 12:58:38 +00:00
String parent_part_name;
std::map<String, std::shared_ptr<IMergeTreeDataPart>> projection_parts;
2022-03-23 04:13:42 +00:00
/// Disabled when USE_ROCKSDB is OFF or use_metadata_cache is set to false in merge tree settings
bool use_metadata_cache = false;
mutable PartMetadataManagerPtr metadata_manager;
2021-12-08 02:40:59 +00:00
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;
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
2022-09-30 18:09:18 +00:00
std::optional<String> getRelativePathForDetachedPart(const String & prefix, bool broken) const;
2020-05-29 15:02:12 +00:00
2022-04-19 12:01:30 +00:00
/// Checks that part can be actually removed from disk.
/// In ordinary scenario always returns true, but in case of
/// zero-copy replication part can be hold by some other replicas.
///
/// If method return false than only metadata of part from
/// local storage can be removed, leaving data in remove FS untouched.
///
/// If method return true, than files can be actually removed from remote
/// storage storage, excluding files in the second returned argument.
/// They can be hardlinks to some newer parts.
2022-04-18 23:09:09 +00:00
std::pair<bool, NameSet> canRemovePart() const;
2021-06-09 12:36:47 +00:00
void initializePartMetadataManager();
2022-09-13 03:18:25 +00:00
void initializeIndexGranularityInfo();
2022-01-07 10:37:08 +00:00
2017-08-16 19:22:49 +00:00
private:
/// 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;
2022-07-28 13:10:43 +00:00
/// Serializations for every columns and subcolumns by their names.
2022-07-27 14:05:16 +00:00
SerializationByName serializations;
2022-07-28 13:10:43 +00:00
/// Columns description for more convenient access
2022-07-21 14:47:19 +00:00
/// to columns by name and getting subcolumns.
ColumnsDescription columns_description;
2023-01-25 22:19:05 +00:00
/// The same as above but after call of Nested::collect().
/// It is used while reading from wide parts.
ColumnsDescription columns_description_with_collected_nested;
/// Reads part unique identifier (if exists) from uuid.txt
void loadUUID();
2021-12-31 03:13:38 +00:00
static void appendFilesOfUUID(Strings & files);
2021-12-08 02:40:59 +00:00
2019-10-31 14:44:17 +00:00
/// Reads columns names and types from columns.txt
void loadColumns(bool require);
2021-12-31 03:13:38 +00:00
static void appendFilesOfColumns(Strings & files);
2021-12-08 02:40:59 +00:00
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);
2021-12-31 03:13:38 +00:00
static void appendFilesOfChecksums(Strings & files);
2021-12-08 02:40:59 +00:00
2019-10-31 14:44:17 +00:00
/// Loads marks index granularity into memory
virtual void loadIndexGranularity();
2021-12-08 02:40:59 +00:00
virtual void appendFilesOfIndexGranularity(Strings & files) const;
2019-10-31 14:44:17 +00:00
/// Loads index file.
void loadIndex();
2022-01-04 05:41:11 +00:00
void appendFilesOfIndex(Strings & files) const;
2021-12-08 02:40:59 +00:00
2019-10-31 14:44:17 +00:00
/// 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();
2021-12-31 03:13:38 +00:00
static void appendFilesOfRowsCount(Strings & files);
2021-12-08 02:40:59 +00:00
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();
2021-12-31 03:13:38 +00:00
static void appendFilesOfTTLInfos(Strings & files);
2021-12-08 02:40:59 +00:00
2019-10-31 14:44:17 +00:00
void loadPartitionAndMinMaxIndex();
2020-08-28 09:07:20 +00:00
void calculateColumnsSizesOnDisk();
void calculateSecondaryIndicesSizesOnDisk();
2021-12-08 02:40:59 +00:00
void appendFilesOfPartitionAndMinMaxIndex(Strings & files) const;
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();
2022-10-23 22:29:24 +00:00
void writeColumns(const NamesAndTypesList & columns_, const WriteSettings & settings);
void writeVersionMetadata(const VersionMetadata & version_, bool fsync_part_dir) const;
template <typename Writer>
void writeMetadata(const String & filename, const WriteSettings & settings, Writer && writer);
2021-12-31 03:13:38 +00:00
static void appendFilesOfDefaultCompressionCodec(Strings & files);
2021-12-08 02:40:59 +00:00
static void appendFilesOfMetadataVersion(Strings & files);
2020-08-28 09:07:20 +00:00
/// Found column without specific compression and return codec
/// for this column with default parameters.
CompressionCodecPtr detectDefaultCompressionCodec() const;
void incrementStateMetric(MergeTreeDataPartState state) const;
void decrementStateMetric(MergeTreeDataPartState state) const;
2022-08-12 11:03:57 +00:00
mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary};
2022-03-09 20:38:18 +00:00
/// This ugly flag is needed for debug assertions only
mutable bool part_is_probably_removed_from_disk = false;
2016-02-14 04:58:47 +00:00
};
2019-10-21 15:33:59 +00:00
using MergeTreeDataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
using MergeTreeMutableDataPartPtr = std::shared_ptr<IMergeTreeDataPart>;
bool isCompactPart(const MergeTreeDataPartPtr & data_part);
bool isWidePart(const MergeTreeDataPartPtr & data_part);
bool isInMemoryPart(const MergeTreeDataPartPtr & data_part);
inline String getIndexExtension(bool is_compressed_primary_key) { return is_compressed_primary_key ? ".cidx" : ".idx"; }
std::optional<String> getIndexExtensionFromFilesystem(const IDataPartStorage & data_part_storage);
bool isCompressedFromIndexExtension(const String & index_extension);
2022-09-05 04:31:19 +00:00
using MergeTreeDataPartsVector = std::vector<MergeTreeDataPartPtr>;
Strings getPartsNames(const MergeTreeDataPartsVector & parts);
2016-02-14 04:58:47 +00:00
}