2016-02-14 04:58:47 +00:00
|
|
|
#pragma once
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
|
|
|
|
2017-08-19 18:11:20 +00:00
|
|
|
#include <Core/Row.h>
|
2017-08-21 15:35:29 +00:00
|
|
|
#include <Core/Block.h>
|
2018-06-04 16:20:02 +00:00
|
|
|
#include <Core/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>
|
2017-07-13 16:49:09 +00:00
|
|
|
#include <Columns/IColumn.h>
|
2018-06-04 16:20:02 +00:00
|
|
|
|
|
|
|
#include <Poco/Path.h>
|
|
|
|
|
2017-07-28 17:34:02 +00:00
|
|
|
#include <shared_mutex>
|
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
|
|
|
|
|
|
|
class IMergeTreeReader;
|
2019-10-21 17:23:06 +00:00
|
|
|
class IMergeTreeDataPartWriter;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
}
|
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:
|
|
|
|
|
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>;
|
2019-12-25 20:06:16 +00:00
|
|
|
using NameToPosition = std::unordered_map<std::string, size_t>;
|
2019-12-03 00:23:11 +00:00
|
|
|
|
2020-02-10 20:27:06 +00:00
|
|
|
using Type = MergeTreeDataPartType;
|
|
|
|
|
|
|
|
|
|
|
|
IMergeTreeDataPart(
|
|
|
|
const MergeTreeData & storage_,
|
|
|
|
const String & name_,
|
|
|
|
const MergeTreePartInfo & info_,
|
|
|
|
const DiskPtr & disk,
|
|
|
|
const std::optional<String> & relative_path,
|
|
|
|
Type part_type_);
|
|
|
|
|
|
|
|
IMergeTreeDataPart(
|
|
|
|
MergeTreeData & storage_,
|
|
|
|
const String & name_,
|
|
|
|
const DiskPtr & disk,
|
|
|
|
const std::optional<String> & relative_path,
|
|
|
|
Type part_type_);
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
virtual MergeTreeReaderPtr getReader(
|
|
|
|
const NamesAndTypesList & columns_,
|
|
|
|
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,
|
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;
|
|
|
|
|
|
|
|
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.
|
2020-03-23 12:19:43 +00:00
|
|
|
ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const;
|
2018-03-26 14:18:04 +00:00
|
|
|
|
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
|
|
|
|
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-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
|
|
|
|
2020-01-16 16:15:01 +00:00
|
|
|
void setColumns(const NamesAndTypesList & new_columns);
|
|
|
|
|
|
|
|
const NamesAndTypesList & getColumns() const { return columns; }
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void assertOnDisk() const;
|
|
|
|
|
2019-12-03 00:23:11 +00:00
|
|
|
void remove() const;
|
2019-10-16 18:27:53 +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.
|
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.
|
|
|
|
String getColumnNameWithMinumumCompressedSize() const;
|
|
|
|
|
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
|
|
|
|
2017-08-21 15:35:29 +00:00
|
|
|
/// If the partition key includes date column (a common case), these functions will return min and max values for this column.
|
2018-05-25 13:29:15 +00:00
|
|
|
DayNum getMinDate() const;
|
|
|
|
DayNum getMaxDate() const;
|
2017-05-16 15:40:32 +00:00
|
|
|
|
2018-12-10 10:20:19 +00:00
|
|
|
/// otherwise, if the partition key includes dateTime column (also a common case), these functions will return min and max values for this column.
|
|
|
|
time_t getMinTime() const;
|
|
|
|
time_t getMaxTime() const;
|
|
|
|
|
2018-05-23 19:34:37 +00:00
|
|
|
bool isEmpty() const { return rows_count == 0; }
|
|
|
|
|
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;
|
|
|
|
|
2019-12-19 13:10:57 +00:00
|
|
|
DiskPtr disk;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
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
|
|
|
|
2019-12-18 16:41:11 +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.
|
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
|
|
|
|
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:
|
2019-09-04 18:26:18 +00:00
|
|
|
* Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set
|
|
|
|
* Precommitted -> Outdated: we could not to add a part to active set and doing a rollback (for example it is duplicated part)
|
|
|
|
* Precommitted -> Commited: we successfully committed a part to active dataset
|
|
|
|
* Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION
|
|
|
|
* Outdated -> Deleting: a cleaner selected this part for deletion
|
|
|
|
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
|
|
|
|
* Committed -> 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
|
|
|
|
PreCommitted, /// the part is in data_parts, but not used for SELECTs
|
|
|
|
Committed, /// active data part, used by current and upcoming SELECTs
|
|
|
|
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
|
2017-09-11 22:40:51 +00:00
|
|
|
mutable State state{State::Temporary};
|
2017-09-05 19:03:51 +00:00
|
|
|
|
2017-09-11 22:40:51 +00:00
|
|
|
/// Returns name of state
|
|
|
|
static String stateToString(State state);
|
|
|
|
String stateString() const;
|
|
|
|
|
|
|
|
String getNameWithState() const
|
|
|
|
{
|
|
|
|
return name + " (state " + stateString() + ")";
|
|
|
|
}
|
|
|
|
|
|
|
|
/// 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)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-03-10 14:39:30 +00:00
|
|
|
void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path);
|
|
|
|
void store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
|
2020-02-27 16:47:40 +00:00
|
|
|
void 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);
|
|
|
|
};
|
|
|
|
|
2017-08-18 19:46:26 +00:00
|
|
|
MinMaxIndex minmax_idx;
|
|
|
|
|
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
|
|
|
|
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-02-27 16:47:40 +00:00
|
|
|
String getFullRelativePath() const;
|
2019-10-10 16:30:30 +00:00
|
|
|
String getFullPath() const;
|
|
|
|
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const;
|
2018-05-21 13:49:54 +00:00
|
|
|
void renameToDetached(const String & prefix) const;
|
|
|
|
void makeCloneInDetached(const String & prefix) const;
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2019-06-07 19:16:42 +00:00
|
|
|
/// Makes full clone of part in detached/ on another disk
|
2019-11-27 09:39:44 +00:00
|
|
|
void makeCloneOnDiskDetached(const ReservationPtr & reservation) const;
|
2019-06-07 19:16:42 +00:00
|
|
|
|
2017-03-24 13:52:50 +00:00
|
|
|
/// Checks that .bin and .mrk files exist
|
2020-01-13 14:53:32 +00:00
|
|
|
virtual bool hasColumnFiles(const String & /* column */, const IDataType & /* type */) const{ return false; }
|
2017-01-17 20:49:16 +00:00
|
|
|
|
2020-02-28 17:14:55 +00:00
|
|
|
static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from);
|
2020-03-23 12:19:43 +00:00
|
|
|
void calculateColumnsSizesOnDisk();
|
2017-08-16 19:22:49 +00:00
|
|
|
|
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;
|
|
|
|
|
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-03-23 12:19:43 +00:00
|
|
|
/// Columns description. Cannot be changed, after part initialiation.
|
2020-01-16 16:15:01 +00:00
|
|
|
NamesAndTypesList columns;
|
2020-01-17 12:24:27 +00:00
|
|
|
const Type part_type;
|
2020-01-16 16:15:01 +00:00
|
|
|
|
2019-11-18 12:22:27 +00:00
|
|
|
void removeIfNeeded();
|
2020-01-16 16:15:01 +00:00
|
|
|
|
|
|
|
virtual void checkConsistency(bool require_part_metadata) const = 0;
|
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.
|
|
|
|
virtual void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0;
|
|
|
|
|
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
|
|
|
|
NameToPosition column_name_to_position;
|
2019-11-18 12:22:27 +00:00
|
|
|
|
2019-10-31 14:44:17 +00:00
|
|
|
/// Reads columns names and types from columns.txt
|
|
|
|
void loadColumns(bool require);
|
|
|
|
|
|
|
|
/// If checksums.txt exists, reads files' checksums (and sizes) from it
|
|
|
|
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();
|
|
|
|
|
|
|
|
/// Loads ttl infos in json format from file ttl.txt. If file doesn`t exists assigns ttl infos with all zeros
|
|
|
|
void loadTTLInfos();
|
|
|
|
|
|
|
|
void loadPartitionAndMinMaxIndex();
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
String getRelativePathForDetachedPart(const String & prefix) const;
|
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>;
|
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);
|
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|