ClickHouse/src/Storages/MergeTree/IMergeTreeDataPart.cpp

1635 lines
58 KiB
C++
Raw Normal View History

2019-10-10 16:30:30 +00:00
#include "IMergeTreeDataPart.h"
#include <optional>
2021-12-08 02:40:59 +00:00
#include <boost/algorithm/string/join.hpp>
#include <string_view>
#include <Core/Defines.h>
#include <IO/HashingWriteBuffer.h>
2021-12-08 02:40:59 +00:00
#include <IO/HashingReadBuffer.h>
#include <IO/ReadBufferFromString.h>
2019-10-10 16:30:30 +00:00
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/localBackup.h>
2020-07-16 10:54:49 +00:00
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
#include <Storages/MergeTree/PartMetadataManagerWithCache.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/ZooKeeper/ZooKeeper.h>
2021-01-15 12:28:53 +00:00
#include <Common/CurrentMetrics.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/MemoryTrackerBlockerInThread.h>
2021-10-02 07:13:14 +00:00
#include <base/JSON.h>
#include <base/logger_useful.h>
2020-08-28 09:07:20 +00:00
#include <Compression/getCompressionCodecForFile.h>
#include <Parsers/parseQuery.h>
2020-08-28 09:07:20 +00:00
#include <Parsers/queryToString.h>
#include <Parsers/ExpressionElementParsers.h>
2020-11-27 11:00:33 +00:00
#include <DataTypes/NestedUtils.h>
2021-05-30 13:57:30 +00:00
#include <DataTypes/DataTypeAggregateFunction.h>
2019-10-10 16:30:30 +00:00
2021-01-15 12:28:53 +00:00
namespace CurrentMetrics
{
extern const Metric PartsTemporary;
extern const Metric PartsPreCommitted;
extern const Metric PartsCommitted;
2021-12-30 14:27:22 +00:00
extern const Metric PartsPreActive;
extern const Metric PartsActive;
2021-01-15 12:28:53 +00:00
extern const Metric PartsOutdated;
extern const Metric PartsDeleting;
extern const Metric PartsDeleteOnDestroy;
extern const Metric PartsWide;
extern const Metric PartsCompact;
extern const Metric PartsInMemory;
2021-01-15 12:28:53 +00:00
}
2019-10-10 16:30:30 +00:00
namespace DB
{
2019-10-10 16:30:30 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int DIRECTORY_ALREADY_EXISTS;
extern const int CANNOT_READ_ALL_DATA;
extern const int LOGICAL_ERROR;
2019-10-10 16:30:30 +00:00
extern const int FILE_DOESNT_EXIST;
extern const int NO_FILE_IN_DATA_PART;
extern const int EXPECTED_END_OF_FILE;
extern const int CORRUPTED_DATA;
extern const int NOT_FOUND_EXPECTED_DATA_PART;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
extern const int BAD_TTL_FILE;
2019-11-18 15:18:50 +00:00
extern const int NOT_IMPLEMENTED;
2019-10-10 16:30:30 +00:00
}
2022-01-07 10:37:08 +00:00
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager)
2019-10-10 16:30:30 +00:00
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
size_t minmax_idx_size = minmax_column_types.size();
2020-03-10 14:56:55 +00:00
hyperrectangle.reserve(minmax_idx_size);
2019-10-10 16:30:30 +00:00
for (size_t i = 0; i < minmax_idx_size; ++i)
{
String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx";
auto file = manager->read(file_name);
2021-03-09 14:46:52 +00:00
auto serialization = minmax_column_types[i]->getDefaultSerialization();
2019-10-10 16:30:30 +00:00
Field min_val;
serialization->deserializeBinary(min_val, *file);
2019-10-10 16:30:30 +00:00
Field max_val;
serialization->deserializeBinary(max_val, *file);
2019-10-10 16:30:30 +00:00
// NULL_LAST
if (min_val.isNull())
min_val = POSITIVE_INFINITY;
if (max_val.isNull())
max_val = POSITIVE_INFINITY;
2020-03-10 14:56:55 +00:00
hyperrectangle.emplace_back(min_val, true, max_val, true);
2019-10-10 16:30:30 +00:00
}
initialized = true;
}
IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store(
2022-04-12 18:59:49 +00:00
const MergeTreeData & data, const DataPartStorageBuilderPtr & data_part_storage_builder, Checksums & out_checksums) const
2019-10-10 16:30:30 +00:00
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
2022-04-12 18:59:49 +00:00
return store(minmax_column_names, minmax_column_types, data_part_storage_builder, out_checksums);
2019-10-10 16:30:30 +00:00
}
IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store(
const Names & column_names,
const DataTypes & data_types,
2022-04-12 18:59:49 +00:00
const DataPartStorageBuilderPtr & data_part_storage_builder,
Checksums & out_checksums) const
2019-10-10 16:30:30 +00:00
{
if (!initialized)
2022-04-12 18:59:49 +00:00
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Attempt to store uninitialized MinMax index for part {}. This is a bug.",
data_part_storage_builder->getFullPath());
2019-10-10 16:30:30 +00:00
WrittenFiles written_files;
2019-10-10 16:30:30 +00:00
for (size_t i = 0; i < column_names.size(); ++i)
{
String file_name = "minmax_" + escapeForFileName(column_names[i]) + ".idx";
2021-03-09 14:46:52 +00:00
auto serialization = data_types.at(i)->getDefaultSerialization();
2019-10-10 16:30:30 +00:00
2022-04-12 18:59:49 +00:00
auto out = data_part_storage_builder->writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE);
HashingWriteBuffer out_hashing(*out);
2021-03-09 14:46:52 +00:00
serialization->serializeBinary(hyperrectangle[i].left, out_hashing);
serialization->serializeBinary(hyperrectangle[i].right, out_hashing);
2019-10-10 16:30:30 +00:00
out_hashing.next();
out_checksums.files[file_name].file_size = out_hashing.count();
out_checksums.files[file_name].file_hash = out_hashing.getHash();
out->preFinalize();
written_files.emplace_back(std::move(out));
2019-10-10 16:30:30 +00:00
}
return written_files;
2019-10-10 16:30:30 +00:00
}
void IMergeTreeDataPart::MinMaxIndex::update(const Block & block, const Names & column_names)
{
if (!initialized)
2020-03-10 14:56:55 +00:00
hyperrectangle.reserve(column_names.size());
2019-10-10 16:30:30 +00:00
for (size_t i = 0; i < column_names.size(); ++i)
{
FieldRef min_value;
FieldRef max_value;
2019-10-10 16:30:30 +00:00
const ColumnWithTypeAndName & column = block.getByName(column_names[i]);
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(column.column.get()))
column_nullable->getExtremesNullLast(min_value, max_value);
else
column.column->getExtremes(min_value, max_value);
2019-10-10 16:30:30 +00:00
if (!initialized)
2020-03-10 14:56:55 +00:00
hyperrectangle.emplace_back(min_value, true, max_value, true);
2019-10-10 16:30:30 +00:00
else
{
hyperrectangle[i].left
= applyVisitor(FieldVisitorAccurateLess(), hyperrectangle[i].left, min_value) ? hyperrectangle[i].left : min_value;
hyperrectangle[i].right
= applyVisitor(FieldVisitorAccurateLess(), hyperrectangle[i].right, max_value) ? max_value : hyperrectangle[i].right;
2019-10-10 16:30:30 +00:00
}
}
initialized = true;
}
void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other)
{
if (!other.initialized)
return;
if (!initialized)
{
2020-03-10 14:56:55 +00:00
hyperrectangle = other.hyperrectangle;
2019-10-10 16:30:30 +00:00
initialized = true;
}
else
{
2020-03-10 14:56:55 +00:00
for (size_t i = 0; i < hyperrectangle.size(); ++i)
2019-10-10 16:30:30 +00:00
{
2020-03-10 14:56:55 +00:00
hyperrectangle[i].left = std::min(hyperrectangle[i].left, other.hyperrectangle[i].left);
hyperrectangle[i].right = std::max(hyperrectangle[i].right, other.hyperrectangle[i].right);
2019-10-10 16:30:30 +00:00
}
}
}
2021-12-08 02:40:59 +00:00
void IMergeTreeDataPart::MinMaxIndex::appendFiles(const MergeTreeData & data, Strings & files)
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
size_t minmax_idx_size = minmax_column_names.size();
for (size_t i = 0; i < minmax_idx_size; ++i)
{
String file_name = "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx";
files.push_back(file_name);
}
}
2019-10-10 16:30:30 +00:00
static void incrementStateMetric(IMergeTreeDataPart::State state)
2021-01-15 12:28:53 +00:00
{
switch (state)
{
case IMergeTreeDataPart::State::Temporary:
CurrentMetrics::add(CurrentMetrics::PartsTemporary);
return;
2021-12-30 14:27:22 +00:00
case IMergeTreeDataPart::State::PreActive:
CurrentMetrics::add(CurrentMetrics::PartsPreActive);
2021-01-15 12:28:53 +00:00
CurrentMetrics::add(CurrentMetrics::PartsPreCommitted);
return;
2021-12-30 14:27:22 +00:00
case IMergeTreeDataPart::State::Active:
CurrentMetrics::add(CurrentMetrics::PartsActive);
2021-01-15 12:28:53 +00:00
CurrentMetrics::add(CurrentMetrics::PartsCommitted);
return;
case IMergeTreeDataPart::State::Outdated:
CurrentMetrics::add(CurrentMetrics::PartsOutdated);
return;
case IMergeTreeDataPart::State::Deleting:
CurrentMetrics::add(CurrentMetrics::PartsDeleting);
return;
case IMergeTreeDataPart::State::DeleteOnDestroy:
CurrentMetrics::add(CurrentMetrics::PartsDeleteOnDestroy);
return;
}
}
static void decrementStateMetric(IMergeTreeDataPart::State state)
2021-01-15 12:28:53 +00:00
{
switch (state)
{
case IMergeTreeDataPart::State::Temporary:
CurrentMetrics::sub(CurrentMetrics::PartsTemporary);
return;
2021-12-30 14:27:22 +00:00
case IMergeTreeDataPart::State::PreActive:
CurrentMetrics::sub(CurrentMetrics::PartsPreActive);
2021-01-15 12:28:53 +00:00
CurrentMetrics::sub(CurrentMetrics::PartsPreCommitted);
return;
2021-12-30 14:27:22 +00:00
case IMergeTreeDataPart::State::Active:
CurrentMetrics::sub(CurrentMetrics::PartsActive);
2021-01-15 12:28:53 +00:00
CurrentMetrics::sub(CurrentMetrics::PartsCommitted);
return;
case IMergeTreeDataPart::State::Outdated:
CurrentMetrics::sub(CurrentMetrics::PartsOutdated);
return;
case IMergeTreeDataPart::State::Deleting:
CurrentMetrics::sub(CurrentMetrics::PartsDeleting);
return;
case IMergeTreeDataPart::State::DeleteOnDestroy:
CurrentMetrics::sub(CurrentMetrics::PartsDeleteOnDestroy);
return;
}
}
2021-01-15 12:28:53 +00:00
static void incrementTypeMetric(MergeTreeDataPartType type)
{
switch (type.getValue())
{
case MergeTreeDataPartType::WIDE:
CurrentMetrics::add(CurrentMetrics::PartsWide);
return;
case MergeTreeDataPartType::COMPACT:
CurrentMetrics::add(CurrentMetrics::PartsCompact);
return;
case MergeTreeDataPartType::IN_MEMORY:
CurrentMetrics::add(CurrentMetrics::PartsInMemory);
return;
case MergeTreeDataPartType::UNKNOWN:
return;
}
}
static void decrementTypeMetric(MergeTreeDataPartType type)
{
switch (type.getValue())
{
case MergeTreeDataPartType::WIDE:
CurrentMetrics::sub(CurrentMetrics::PartsWide);
return;
case MergeTreeDataPartType::COMPACT:
CurrentMetrics::sub(CurrentMetrics::PartsCompact);
return;
case MergeTreeDataPartType::IN_MEMORY:
CurrentMetrics::sub(CurrentMetrics::PartsInMemory);
return;
case MergeTreeDataPartType::UNKNOWN:
return;
}
2021-01-15 12:28:53 +00:00
}
2019-10-10 16:30:30 +00:00
IMergeTreeDataPart::IMergeTreeDataPart(
2021-05-14 21:45:13 +00:00
const MergeTreeData & storage_,
2021-03-12 16:33:41 +00:00
const String & name_,
2022-04-05 19:12:48 +00:00
const DataPartStoragePtr & data_part_storage_,
Type part_type_,
const IMergeTreeDataPart * parent_part_)
2019-10-10 16:30:30 +00:00
: storage(storage_)
, name(name_)
, info(MergeTreePartInfo::fromPartName(name_, storage.format_version))
2022-04-05 19:12:48 +00:00
, data_part_storage(parent_part_ ? parent_part_->data_part_storage : data_part_storage_)
2020-01-14 13:23:51 +00:00
, index_granularity_info(storage_, part_type_)
, part_type(part_type_)
, parent_part(parent_part_)
2021-12-28 11:29:01 +00:00
, use_metadata_cache(storage.use_metadata_cache)
2019-10-16 18:27:53 +00:00
{
if (parent_part)
2021-12-30 14:27:22 +00:00
state = State::Active;
incrementStateMetric(state);
incrementTypeMetric(part_type);
minmax_idx = std::make_shared<MinMaxIndex>();
initializePartMetadataManager();
2019-10-16 18:27:53 +00:00
}
2019-10-10 16:30:30 +00:00
IMergeTreeDataPart::IMergeTreeDataPart(
const MergeTreeData & storage_,
const String & name_,
const MergeTreePartInfo & info_,
2022-04-05 19:12:48 +00:00
const DataPartStoragePtr & data_part_storage_,
Type part_type_,
const IMergeTreeDataPart * parent_part_)
2019-10-10 16:30:30 +00:00
: storage(storage_)
, name(name_)
, info(info_)
2022-04-05 19:12:48 +00:00
, data_part_storage(parent_part_ ? parent_part_->data_part_storage : data_part_storage_)
2020-01-14 13:23:51 +00:00
, index_granularity_info(storage_, part_type_)
, part_type(part_type_)
, parent_part(parent_part_)
2021-12-28 11:29:01 +00:00
, use_metadata_cache(storage.use_metadata_cache)
2019-10-16 18:27:53 +00:00
{
if (parent_part)
2021-12-30 14:27:22 +00:00
state = State::Active;
incrementStateMetric(state);
incrementTypeMetric(part_type);
minmax_idx = std::make_shared<MinMaxIndex>();
2022-01-05 12:05:22 +00:00
initializePartMetadataManager();
2021-01-15 12:28:53 +00:00
}
IMergeTreeDataPart::~IMergeTreeDataPart()
{
decrementStateMetric(state);
decrementTypeMetric(part_type);
2019-10-16 18:27:53 +00:00
}
2019-10-10 16:30:30 +00:00
String IMergeTreeDataPart::getNewName(const MergeTreePartInfo & new_part_info) const
{
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
/// NOTE: getting min and max dates from the part name (instead of part data) because we want
/// the merged part name be determined only by source part names.
/// It is simpler this way when the real min and max dates for the block range can change
/// (e.g. after an ALTER DELETE command).
DayNum min_date;
DayNum max_date;
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
return new_part_info.getPartNameV0(min_date, max_date);
}
else
return new_part_info.getPartName();
}
std::optional<size_t> IMergeTreeDataPart::getColumnPosition(const String & column_name) const
2019-10-31 14:44:17 +00:00
{
auto it = column_name_to_position.find(column_name);
if (it == column_name_to_position.end())
return {};
return it->second;
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::setState(IMergeTreeDataPart::State new_state) const
{
decrementStateMetric(state);
state = new_state;
incrementStateMetric(state);
}
IMergeTreeDataPart::State IMergeTreeDataPart::getState() const
{
return state;
}
std::pair<DayNum, DayNum> IMergeTreeDataPart::getMinMaxDate() const
2019-10-10 16:30:30 +00:00
{
if (storage.minmax_idx_date_column_pos != -1 && minmax_idx->initialized)
{
const auto & hyperrectangle = minmax_idx->hyperrectangle[storage.minmax_idx_date_column_pos];
return {DayNum(hyperrectangle.left.get<UInt64>()), DayNum(hyperrectangle.right.get<UInt64>())};
}
2019-10-10 16:30:30 +00:00
else
return {};
2019-10-10 16:30:30 +00:00
}
std::pair<time_t, time_t> IMergeTreeDataPart::getMinMaxTime() const
2019-10-10 16:30:30 +00:00
{
if (storage.minmax_idx_time_column_pos != -1 && minmax_idx->initialized)
{
const auto & hyperrectangle = minmax_idx->hyperrectangle[storage.minmax_idx_time_column_pos];
2019-10-10 16:30:30 +00:00
/// The case of DateTime
if (hyperrectangle.left.getType() == Field::Types::UInt64)
{
assert(hyperrectangle.right.getType() == Field::Types::UInt64);
return {hyperrectangle.left.get<UInt64>(), hyperrectangle.right.get<UInt64>()};
}
/// The case of DateTime64
else if (hyperrectangle.left.getType() == Field::Types::Decimal64)
{
2021-03-23 23:03:14 +00:00
assert(hyperrectangle.right.getType() == Field::Types::Decimal64);
2019-10-10 16:30:30 +00:00
auto left = hyperrectangle.left.get<DecimalField<Decimal64>>();
auto right = hyperrectangle.right.get<DecimalField<Decimal64>>();
assert(left.getScale() == right.getScale());
return { left.getValue() / left.getScaleMultiplier(), right.getValue() / right.getScaleMultiplier() };
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part minmax index by time is neither DateTime or DateTime64");
}
2019-10-10 16:30:30 +00:00
else
return {};
2019-10-10 16:30:30 +00:00
}
2022-01-21 00:20:41 +00:00
void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns)
2019-11-18 12:22:27 +00:00
{
columns = new_columns;
2021-10-29 17:21:02 +00:00
column_name_to_position.clear();
column_name_to_position.reserve(new_columns.size());
size_t pos = 0;
2021-10-29 17:21:02 +00:00
2019-11-18 12:22:27 +00:00
for (const auto & column : columns)
2021-11-02 03:03:52 +00:00
column_name_to_position.emplace(column.name, pos++);
2022-01-21 00:20:41 +00:00
}
2021-10-29 17:21:02 +00:00
2022-01-21 00:20:41 +00:00
void IMergeTreeDataPart::setSerializationInfos(const SerializationInfoByName & new_infos)
{
serialization_infos = new_infos;
2019-11-18 12:22:27 +00:00
}
2021-11-02 03:03:52 +00:00
SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & column) const
2021-11-01 02:40:43 +00:00
{
2021-11-02 03:03:52 +00:00
auto it = serialization_infos.find(column.getNameInStorage());
return it == serialization_infos.end()
? IDataType::getSerialization(column)
: IDataType::getSerialization(column, *it->second);
2021-11-01 02:40:43 +00:00
}
2019-11-18 12:22:27 +00:00
void IMergeTreeDataPart::removeIfNeeded()
2019-10-10 16:30:30 +00:00
{
if (!is_temp && state != State::DeleteOnDestroy)
return;
2019-10-31 14:44:17 +00:00
try
{
2022-04-05 19:12:48 +00:00
auto path = data_part_storage->getFullRelativePath();
2019-10-31 14:44:17 +00:00
2022-04-05 19:12:48 +00:00
if (!data_part_storage->exists()) // path
return;
2019-10-10 16:30:30 +00:00
if (is_temp)
{
2022-04-19 19:34:41 +00:00
String file_name = fileName(data_part_storage->getRelativePath());
2019-10-10 16:30:30 +00:00
if (file_name.empty())
2022-04-19 19:34:41 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "relative_path {} of part {} is invalid or not set", data_part_storage->getRelativePath(), name);
2019-10-10 16:30:30 +00:00
if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj"))
2021-06-09 12:36:47 +00:00
{
LOG_ERROR(
storage.log,
"~DataPart() should remove part {} but its name doesn't start with \"tmp\" or end with \".tmp_proj\". Too "
"suspicious, keeping the part.",
path);
return;
2021-06-09 12:36:47 +00:00
}
}
2019-11-19 09:38:17 +00:00
2022-04-19 19:34:41 +00:00
remove();
if (state == State::DeleteOnDestroy)
2019-10-31 14:44:17 +00:00
{
LOG_TRACE(storage.log, "Removed part from old location {}", path);
2019-10-31 14:44:17 +00:00
}
}
catch (...)
{
/// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime).
/// If it's tmp_merge_<part_name> or tmp_fetch_<part_name>,
/// then all future attempts to execute part producing operation will fail with "directory already exists".
/// Seems like it's especially important for remote disks, because removal may fail due to network issues.
tryLogCurrentException(__PRETTY_FUNCTION__);
assert(!is_temp);
assert(state != State::DeleteOnDestroy);
assert(state != State::Temporary);
}
2019-10-10 16:30:30 +00:00
}
UInt64 IMergeTreeDataPart::getIndexSizeInBytes() const
{
UInt64 res = 0;
for (const ColumnPtr & column : index)
res += column->byteSize();
return res;
}
UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const
{
UInt64 res = 0;
for (const ColumnPtr & column : index)
res += column->allocatedBytes();
return res;
}
void IMergeTreeDataPart::assertState(const std::initializer_list<IMergeTreeDataPart::State> & affordable_states) const
{
if (!checkState(affordable_states))
{
String states_str;
for (auto affordable_state : affordable_states)
{
states_str += stateString(affordable_state);
states_str += ' ';
}
2019-10-10 16:30:30 +00:00
throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART);
}
}
void IMergeTreeDataPart::assertOnDisk() const
{
if (!isStoredOnDisk())
2019-11-05 11:53:22 +00:00
throw Exception("Data part '" + name + "' with type '"
+ getType().toString() + "' is not stored on disk", ErrorCodes::LOGICAL_ERROR);
2019-10-10 16:30:30 +00:00
}
UInt64 IMergeTreeDataPart::getMarksCount() const
{
return index_granularity.getMarksCount();
}
size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
{
auto checksum = checksums.files.find(file_name);
if (checksum == checksums.files.end())
return 0;
return checksum->second.file_size;
}
2022-03-28 17:21:47 +00:00
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageSnapshotPtr & storage_snapshot) const
{
2022-03-28 17:21:47 +00:00
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects().withSubcolumns();
auto storage_columns = storage_snapshot->getColumns(options);
MergeTreeData::AlterConversions alter_conversions;
if (!parent_part)
alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
2020-05-15 10:26:44 +00:00
std::optional<std::string> minimum_size_column;
UInt64 minimum_size = std::numeric_limits<UInt64>::max();
for (const auto & column : storage_columns)
{
2020-05-15 10:26:44 +00:00
auto column_name = column.name;
auto column_type = column.type;
if (alter_conversions.isColumnRenamed(column.name))
column_name = alter_conversions.getColumnOldName(column.name);
if (!hasColumnFiles(column))
continue;
const auto size = getColumnSize(column_name).data_compressed;
if (size < minimum_size)
{
minimum_size = size;
2020-05-15 10:26:44 +00:00
minimum_size_column = column_name;
}
}
if (!minimum_size_column)
2022-04-05 19:12:48 +00:00
throw Exception("Could not find a column of minimum size in MergeTree, part " + data_part_storage->getFullPath(), ErrorCodes::LOGICAL_ERROR);
return *minimum_size_column;
}
2022-04-05 19:12:48 +00:00
// String IMergeTreeDataPart::getFullPath() const
// {
// if (relative_path.empty())
// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
2019-10-10 16:30:30 +00:00
2022-04-05 19:12:48 +00:00
// return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / (parent_part ? parent_part->relative_path : "") / relative_path / "";
// }
2019-10-10 16:30:30 +00:00
2022-04-05 19:12:48 +00:00
// String IMergeTreeDataPart::getFullRelativePath() const
// {
// if (relative_path.empty())
// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
2022-04-05 19:12:48 +00:00
// return fs::path(storage.relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / "";
// }
2019-11-18 15:18:50 +00:00
void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency)
2019-10-31 14:44:17 +00:00
{
assertOnDisk();
/// Memory should not be limited during ATTACH TABLE query.
/// This is already true at the server startup but must be also ensured for manual table ATTACH.
/// Motivation: memory for index is shared between queries - not belong to the query itself.
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker(VariableContext::Global);
2019-10-31 14:44:17 +00:00
2022-03-31 02:10:05 +00:00
try
{
loadUUID();
loadColumns(require_columns_checksums);
loadChecksums(require_columns_checksums);
loadIndexGranularity();
calculateColumnsAndSecondaryIndicesSizesOnDisk();
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`.
loadPartitionAndMinMaxIndex();
if (!parent_part)
{
loadTTLInfos();
loadProjections(require_columns_checksums, check_consistency);
}
if (check_consistency)
checkConsistency(require_columns_checksums);
loadDefaultCompressionCodec();
}
catch (...)
{
// There could be conditions that data part to be loaded is broken, but some of meta infos are already written
// into meta data before exception, need to clean them all.
metadata_manager->deleteAll(/*include_projection*/ true);
metadata_manager->assertAllDeleted(/*include_projection*/ true);
throw;
}
2019-10-31 14:44:17 +00:00
}
2021-12-08 02:40:59 +00:00
void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, bool include_projection) const
{
if (isStoredOnDisk())
{
appendFilesOfUUID(files);
appendFilesOfColumns(files);
appendFilesOfChecksums(files);
appendFilesOfIndexGranularity(files);
2022-01-04 05:41:11 +00:00
appendFilesOfIndex(files);
2021-12-08 02:40:59 +00:00
appendFilesOfRowsCount(files);
appendFilesOfPartitionAndMinMaxIndex(files);
appendFilesOfTTLInfos(files);
appendFilesOfDefaultCompressionCodec(files);
}
if (!parent_part && include_projection)
{
for (const auto & [projection_name, projection_part] : projection_parts)
{
Strings projection_files;
projection_part->appendFilesOfColumnsChecksumsIndexes(projection_files, true);
for (const auto & projection_file : projection_files)
2022-04-07 17:44:49 +00:00
files.push_back(fs::path(projection_part->name + ".proj") / projection_file);
2021-12-08 02:40:59 +00:00
}
}
}
void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency)
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
for (const auto & projection : metadata_snapshot->projections)
{
2022-04-05 19:12:48 +00:00
String path = /*getFullRelativePath() + */ projection.name + ".proj";
if (data_part_storage->exists(path))
{
2022-04-05 19:12:48 +00:00
auto projection_part_storage = data_part_storage->getProjection(projection.name + ".proj");
auto part = storage.createPart(projection.name, {"all", 0, 0, 0}, projection_part_storage, this);
part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency);
projection_parts.emplace(projection.name, std::move(part));
}
}
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadIndexGranularity()
{
throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
2019-10-31 14:44:17 +00:00
}
2022-03-23 04:13:42 +00:00
/// Currently we don't cache mark files of part, because cache other meta files is enough to speed up loading.
2021-12-08 02:40:59 +00:00
void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) const
{
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadIndex()
{
/// It can be empty in case of mutations
if (!index_granularity.isInitialized())
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
2020-06-17 12:39:20 +00:00
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part)
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
2020-06-17 12:39:20 +00:00
const auto & primary_key = metadata_snapshot->getPrimaryKey();
2020-05-21 19:46:03 +00:00
size_t key_size = primary_key.column_names.size();
2019-10-31 14:44:17 +00:00
if (key_size)
{
MutableColumns loaded_index;
loaded_index.resize(key_size);
for (size_t i = 0; i < key_size; ++i)
{
2020-05-20 18:11:38 +00:00
loaded_index[i] = primary_key.data_types[i]->createColumn();
2019-10-31 14:44:17 +00:00
loaded_index[i]->reserve(index_granularity.getMarksCount());
}
String index_name = "primary.idx";
2022-04-05 19:12:48 +00:00
String index_path = fs::path(data_part_storage->getFullRelativePath()) / index_name;
auto index_file = metadata_manager->read(index_name);
2020-05-25 23:47:11 +00:00
size_t marks_count = index_granularity.getMarksCount();
2021-10-29 17:21:02 +00:00
Serializations key_serializations(key_size);
for (size_t j = 0; j < key_size; ++j)
2021-10-29 17:21:02 +00:00
key_serializations[j] = primary_key.data_types[j]->getDefaultSerialization();
2020-05-25 23:47:11 +00:00
for (size_t i = 0; i < marks_count; ++i) //-V756
2019-10-31 14:44:17 +00:00
for (size_t j = 0; j < key_size; ++j)
key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file);
2019-10-31 14:44:17 +00:00
for (size_t i = 0; i < key_size; ++i)
{
loaded_index[i]->protect();
2020-05-25 23:47:11 +00:00
if (loaded_index[i]->size() != marks_count)
2019-10-31 14:44:17 +00:00
throw Exception("Cannot read all data from index file " + index_path
2020-05-25 23:47:11 +00:00
+ "(expected size: " + toString(marks_count) + ", read: " + toString(loaded_index[i]->size()) + ")",
2019-10-31 14:44:17 +00:00
ErrorCodes::CANNOT_READ_ALL_DATA);
}
if (!index_file->eof())
2022-04-05 19:12:48 +00:00
throw Exception("Index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
2019-10-31 14:44:17 +00:00
index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end()));
}
}
2022-01-04 05:41:11 +00:00
void IMergeTreeDataPart::appendFilesOfIndex(Strings & files) const
2021-12-08 02:40:59 +00:00
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part)
metadata_snapshot = metadata_snapshot->projections.has(name) ? metadata_snapshot->projections.get(name).metadata : nullptr;
if (!metadata_snapshot)
return;
2021-12-29 04:31:54 +00:00
if (metadata_snapshot->hasPrimaryKey())
2021-12-08 02:40:59 +00:00
files.push_back("primary.idx");
}
NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const
{
if (!isStoredOnDisk())
return {};
NameSet result = {"checksums.txt", "columns.txt"};
2022-04-05 19:12:48 +00:00
//String default_codec_path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME;
2022-04-05 19:12:48 +00:00
if (data_part_storage->exists(DEFAULT_COMPRESSION_CODEC_FILE_NAME))
result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
return result;
}
2020-08-28 09:07:20 +00:00
void IMergeTreeDataPart::loadDefaultCompressionCodec()
{
2020-08-27 08:35:55 +00:00
/// In memory parts doesn't have any compression
if (!isStoredOnDisk())
{
default_codec = CompressionCodecFactory::instance().get("NONE", {});
2020-08-28 09:07:20 +00:00
return;
}
2022-04-05 19:12:48 +00:00
String path = fs::path(data_part_storage->getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME;
bool exists = metadata_manager->exists(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
if (!exists)
2020-08-28 09:07:20 +00:00
{
default_codec = detectDefaultCompressionCodec();
}
else
{
auto file_buf = metadata_manager->read(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
2020-08-28 09:07:20 +00:00
String codec_line;
readEscapedStringUntilEOL(codec_line, *file_buf);
2020-08-28 09:07:20 +00:00
ReadBufferFromString buf(codec_line);
2020-08-28 09:07:20 +00:00
if (!checkString("CODEC", buf))
{
2021-12-08 02:40:59 +00:00
LOG_WARNING(
storage.log,
"Cannot parse default codec for part {} from file {}, content '{}'. Default compression codec will be deduced "
"automatically, from data on disk",
name,
path,
codec_line);
2020-08-28 09:07:20 +00:00
default_codec = detectDefaultCompressionCodec();
}
2020-08-28 09:07:20 +00:00
try
{
ParserCodec codec_parser;
auto codec_ast = parseQuery(codec_parser, codec_line.data() + buf.getPosition(), codec_line.data() + codec_line.length(), "codec parser", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
default_codec = CompressionCodecFactory::instance().get(codec_ast, {});
}
catch (const DB::Exception & ex)
{
LOG_WARNING(storage.log, "Cannot parse default codec for part {} from file {}, content '{}', error '{}'. Default compression codec will be deduced automatically, from data on disk.", name, path, codec_line, ex.what());
default_codec = detectDefaultCompressionCodec();
}
}
2020-08-28 09:07:20 +00:00
}
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME);
}
2020-08-28 09:07:20 +00:00
CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const
{
/// In memory parts doesn't have any compression
if (!isStoredOnDisk())
return CompressionCodecFactory::instance().get("NONE", {});
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
const auto & storage_columns = metadata_snapshot->getColumns();
CompressionCodecPtr result = nullptr;
for (const auto & part_column : columns)
{
2020-08-31 13:39:27 +00:00
/// It was compressed with default codec and it's not empty
auto column_size = getColumnSize(part_column.name);
2020-08-31 13:39:27 +00:00
if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name))
2020-08-28 09:07:20 +00:00
{
String path_to_data_file;
2021-11-02 03:03:52 +00:00
getSerialization(part_column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
2020-12-23 11:53:49 +00:00
{
if (path_to_data_file.empty())
{
2022-04-05 19:12:48 +00:00
String candidate_path = /*fs::path(getFullRelativePath()) */ (ISerialization::getFileNameForStream(part_column, substream_path) + ".bin");
2021-01-15 09:10:03 +00:00
/// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file.
2022-04-05 19:12:48 +00:00
if (data_part_storage->exists(candidate_path) && data_part_storage->getFileSize(candidate_path) != 0)
path_to_data_file = candidate_path;
}
});
if (path_to_data_file.empty())
2020-12-23 11:53:49 +00:00
{
LOG_WARNING(storage.log, "Part's {} column {} has non zero data compressed size, but all data files don't exist or empty", name, backQuoteIfNeed(part_column.name));
2020-12-23 11:53:49 +00:00
continue;
}
2022-04-05 19:12:48 +00:00
result = getCompressionCodecForFile(data_part_storage, path_to_data_file);
2020-08-28 09:07:20 +00:00
break;
}
}
2020-08-28 09:07:20 +00:00
if (!result)
result = CompressionCodecFactory::instance().getDefaultCodec();
2020-08-28 09:07:20 +00:00
return result;
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
{
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part)
2019-10-31 14:44:17 +00:00
{
DayNum min_date;
DayNum max_date;
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
const auto & date_lut = DateLUT::instance();
partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date));
minmax_idx = std::make_shared<MinMaxIndex>(min_date, max_date);
2019-10-31 14:44:17 +00:00
}
else
{
2022-04-05 19:12:48 +00:00
//String path = getFullRelativePath();
if (!parent_part)
partition.load(storage, metadata_manager);
2019-10-31 14:44:17 +00:00
if (!isEmpty())
{
if (parent_part)
// projection parts don't have minmax_idx, and it's always initialized
minmax_idx->initialized = true;
else
minmax_idx->load(storage, metadata_manager);
}
if (parent_part)
return;
2019-10-31 14:44:17 +00:00
}
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
String calculated_partition_id = partition.getID(metadata_snapshot->getPartitionKey().sample_block);
2019-10-31 14:44:17 +00:00
if (calculated_partition_id != info.partition_id)
throw Exception(
2022-04-05 19:12:48 +00:00
"While loading part " + data_part_storage->getFullPath() + ": calculated partition ID: " + calculated_partition_id
2019-10-31 14:44:17 +00:00
+ " differs from partition ID in part name: " + info.partition_id,
ErrorCodes::CORRUPTED_DATA);
}
2021-12-08 02:40:59 +00:00
void IMergeTreeDataPart::appendFilesOfPartitionAndMinMaxIndex(Strings & files) const
{
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part)
return;
if (!parent_part)
partition.appendFiles(storage, files);
if (!isEmpty())
if (!parent_part)
minmax_idx->appendFiles(storage, files);
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadChecksums(bool require)
{
2022-04-05 19:12:48 +00:00
//const String path = fs::path(getFullRelativePath()) / "checksums.txt";
bool exists = metadata_manager->exists("checksums.txt");
if (exists)
{
auto buf = metadata_manager->read("checksums.txt");
if (checksums.read(*buf))
2019-10-31 14:44:17 +00:00
{
assertEOF(*buf);
2019-10-31 14:44:17 +00:00
bytes_on_disk = checksums.getTotalSizeOnDisk();
}
else
2022-04-05 19:12:48 +00:00
bytes_on_disk = data_part_storage->calculateTotalSizeOnDisk(); //calculateTotalSizeOnDisk(volume->getDisk(), getFullRelativePath());
2019-10-31 14:44:17 +00:00
}
else
{
if (require)
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No checksums.txt in part {}", name);
2019-10-31 14:44:17 +00:00
2020-07-16 10:54:49 +00:00
/// If the checksums file is not present, calculate the checksums and write them to disk.
/// Check the data while we are at it.
LOG_WARNING(storage.log, "Checksums for part {} not found. Will calculate them from data on disk.", name);
2020-07-16 10:54:49 +00:00
checksums = checkDataPart(shared_from_this(), false);
2022-04-05 19:12:48 +00:00
data_part_storage->writeChecksums(checksums);
2020-07-16 10:54:49 +00:00
bytes_on_disk = checksums.getTotalSizeOnDisk();
2019-10-31 14:44:17 +00:00
}
}
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfChecksums(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back("checksums.txt");
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadRowsCount()
{
2022-04-05 19:12:48 +00:00
//String path = fs::path(getFullRelativePath()) / "count.txt";
auto read_rows_count = [&]()
{
auto buf = metadata_manager->read("count.txt");
readIntText(rows_count, *buf);
assertEOF(*buf);
};
2019-10-31 14:44:17 +00:00
if (index_granularity.empty())
{
rows_count = 0;
}
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part)
2019-10-31 14:44:17 +00:00
{
bool exists = metadata_manager->exists("count.txt");
if (!exists)
throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
read_rows_count();
2020-07-25 14:42:20 +00:00
#ifndef NDEBUG
/// columns have to be loaded
for (const auto & column : getColumns())
{
2020-07-27 09:42:37 +00:00
/// Most trivial types
if (column.type->isValueRepresentedByNumber()
&& !column.type->haveSubtypes()
2021-11-02 20:30:28 +00:00
&& getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT)
2020-07-25 14:42:20 +00:00
{
auto size = getColumnSize(column.name);
2020-07-25 14:42:20 +00:00
if (size.data_uncompressed == 0)
continue;
size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory();
if (rows_in_column != rows_count)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Column {} has rows count {} according to size in memory "
"and size of single value, but data part {} has {} rows", backQuote(column.name), rows_in_column, name, rows_count);
}
2020-12-09 11:46:04 +00:00
size_t last_possibly_incomplete_mark_rows = index_granularity.getLastNonFinalMarkRows();
/// All this rows have to be written in column
size_t index_granularity_without_last_mark = index_granularity.getTotalRows() - last_possibly_incomplete_mark_rows;
/// We have more rows in column than in index granularity without last possibly incomplete mark
if (rows_in_column < index_granularity_without_last_mark)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Column {} has rows count {} according to size in memory "
2020-12-09 11:46:04 +00:00
"and size of single value, but index granularity in part {} without last mark has {} rows, which is more than in column",
backQuote(column.name), rows_in_column, name, index_granularity.getTotalRows());
}
/// In last mark we actually written less or equal rows than stored in last mark of index granularity
if (rows_in_column - index_granularity_without_last_mark > last_possibly_incomplete_mark_rows)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Column {} has rows count {} in last mark according to size in memory "
"and size of single value, but index granularity in part {} in last mark has {} rows which is less than in column",
backQuote(column.name), rows_in_column - index_granularity_without_last_mark, name, last_possibly_incomplete_mark_rows);
}
2020-07-25 14:42:20 +00:00
}
}
#endif
2019-10-31 14:44:17 +00:00
}
else
{
2022-04-05 19:12:48 +00:00
if (data_part_storage->exists("count.txt"))
{
read_rows_count();
return;
}
2019-10-31 14:44:17 +00:00
for (const NameAndTypePair & column : columns)
{
2021-11-02 03:03:52 +00:00
ColumnPtr column_col = column.type->createColumn(*getSerialization(column));
2019-10-31 14:44:17 +00:00
if (!column_col->isFixedAndContiguous() || column_col->lowCardinality())
continue;
size_t column_size = getColumnSize(column.name).data_uncompressed;
2019-10-31 14:44:17 +00:00
if (!column_size)
continue;
size_t sizeof_field = column_col->sizeOfValueIfFixed();
rows_count = column_size / sizeof_field;
if (column_size % sizeof_field != 0)
{
throw Exception(
"Uncompressed size of column " + column.name + "(" + toString(column_size)
+ ") is not divisible by the size of value (" + toString(sizeof_field) + ")",
ErrorCodes::LOGICAL_ERROR);
}
size_t last_mark_index_granularity = index_granularity.getLastNonFinalMarkRows();
size_t rows_approx = index_granularity.getTotalRows();
if (!(rows_count <= rows_approx && rows_approx < rows_count + last_mark_index_granularity))
throw Exception(
"Unexpected size of column " + column.name + ": " + toString(rows_count) + " rows, expected "
+ toString(rows_approx) + "+-" + toString(last_mark_index_granularity) + " rows according to the index",
ErrorCodes::LOGICAL_ERROR);
return;
}
throw Exception("Data part doesn't contain fixed size column (even Date column)", ErrorCodes::LOGICAL_ERROR);
}
}
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfRowsCount(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back("count.txt");
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadTTLInfos()
{
bool exists = metadata_manager->exists("ttl.txt");
if (exists)
{
auto in = metadata_manager->read("ttl.txt");
assertString("ttl format version: ", *in);
2019-10-31 14:44:17 +00:00
size_t format_version;
readText(format_version, *in);
assertChar('\n', *in);
2019-10-31 14:44:17 +00:00
if (format_version == 1)
{
try
{
ttl_infos.read(*in);
2019-10-31 14:44:17 +00:00
}
catch (const JSONException &)
{
throw Exception("Error while parsing file ttl.txt in part: " + name, ErrorCodes::BAD_TTL_FILE);
}
}
else
throw Exception("Unknown ttl format version: " + toString(format_version), ErrorCodes::BAD_TTL_FILE);
}
}
2021-12-08 02:40:59 +00:00
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfTTLInfos(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back("ttl.txt");
}
void IMergeTreeDataPart::loadUUID()
{
bool exists = metadata_manager->exists(UUID_FILE_NAME);
if (exists)
{
auto in = metadata_manager->read(UUID_FILE_NAME);
readText(uuid, *in);
if (uuid == UUIDHelpers::Nil)
throw Exception("Unexpected empty " + String(UUID_FILE_NAME) + " in part: " + name, ErrorCodes::LOGICAL_ERROR);
}
}
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfUUID(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back(UUID_FILE_NAME);
}
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart::loadColumns(bool require)
{
2022-04-05 19:12:48 +00:00
String path = fs::path(data_part_storage->getFullRelativePath()) / "columns.txt";
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part)
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
NamesAndTypesList loaded_columns;
bool exists = metadata_manager->exists("columns.txt");
if (!exists)
2019-10-31 14:44:17 +00:00
{
/// We can get list of columns only from columns.txt in compact parts.
if (require || part_type == Type::COMPACT)
2022-04-05 19:12:48 +00:00
throw Exception("No columns.txt in part " + name + ", expected path " + path + " on drive " + data_part_storage->getName(),
ErrorCodes::NO_FILE_IN_DATA_PART);
2019-10-31 14:44:17 +00:00
/// If there is no file with a list of columns, write it down.
for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical())
2022-04-05 19:12:48 +00:00
if (data_part_storage->exists(getFileNameForColumn(column) + ".bin"))
loaded_columns.push_back(column);
2019-10-31 14:44:17 +00:00
if (columns.empty())
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
2022-04-05 19:12:48 +00:00
data_part_storage->writeColumns(loaded_columns);
2019-12-09 21:21:17 +00:00
}
else
{
auto in = metadata_manager->read("columns.txt");
2021-12-08 02:40:59 +00:00
loaded_columns.readText(*in);
2021-11-27 09:40:46 +00:00
for (const auto & column : loaded_columns)
{
const auto * aggregate_function_data_type = typeid_cast<const DataTypeAggregateFunction *>(column.type.get());
if (aggregate_function_data_type && aggregate_function_data_type->isVersioned())
aggregate_function_data_type->setVersion(0, /* if_empty */true);
}
2019-10-31 14:44:17 +00:00
}
2021-10-29 17:21:02 +00:00
SerializationInfo::Settings settings =
{
2021-12-08 15:29:00 +00:00
.ratio_of_defaults_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization,
2021-10-29 17:21:02 +00:00
.choose_kind = false,
};
SerializationInfoByName infos(loaded_columns, settings);
2022-03-23 04:13:42 +00:00
exists = metadata_manager->exists(SERIALIZATION_FILE_NAME);
if (exists)
{
auto in = metadata_manager->read(SERIALIZATION_FILE_NAME);
infos.readJSON(*in);
}
2021-10-29 17:21:02 +00:00
2022-01-21 00:20:41 +00:00
setColumns(loaded_columns);
setSerializationInfos(infos);
2019-10-31 14:44:17 +00:00
}
2021-12-31 03:13:38 +00:00
void IMergeTreeDataPart::appendFilesOfColumns(Strings & files)
2021-12-08 02:40:59 +00:00
{
files.push_back("columns.txt");
2022-03-23 04:13:42 +00:00
files.push_back(SERIALIZATION_FILE_NAME);
2021-12-08 02:40:59 +00:00
}
bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const
{
2022-04-07 17:44:49 +00:00
return data_part_storage->shallParticipateInMerges(*storage_policy);
}
2022-04-05 19:12:48 +00:00
// UInt64 IMergeTreeDataPart::calculateTotalSizeOnDisk(const DataPartStoragePtr & data_part_storage_, const String & from)
// {
// if (data_part_storage_->isFile(from))
// return data_part_storage_->getFileSize(from);
// std::vector<std::string> files;
// disk_->listFiles(from, files);
// UInt64 res = 0;
// for (const auto & file : files)
// res += calculateTotalSizeOnDisk(data_part_storage_, fs::path(from) / file);
// return res;
// }
2019-10-10 16:30:30 +00:00
void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const
try
2019-10-10 16:30:30 +00:00
{
2019-10-31 14:44:17 +00:00
assertOnDisk();
2022-04-07 11:58:38 +00:00
if (parent_part)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Move is not supported for projection parts: moving form {} to {}",
data_part_storage->getFullPath(), new_relative_path);
2022-04-07 17:44:49 +00:00
String from = data_part_storage->getFullRelativePath();
2022-04-07 11:58:38 +00:00
String to = fs::path(storage.relative_data_path) / new_relative_path / "";
2022-04-12 18:59:49 +00:00
data_part_storage->rename(new_relative_path, storage.log, remove_new_dir_if_exists, storage.getSettings()->fsync_part_directory);
2022-04-07 11:58:38 +00:00
metadata_manager->move(from, to);
2021-02-26 09:48:57 +00:00
storage.lockSharedData(*this);
2019-10-10 16:30:30 +00:00
}
catch (...)
{
if (startsWith(new_relative_path, "detached/"))
{
// Don't throw when the destination is to the detached folder. It might be able to
// recover in some cases, such as fetching parts into multi-disks while some of the
// disks are broken.
tryLogCurrentException(__PRETTY_FUNCTION__);
}
else
throw;
}
2019-10-10 16:30:30 +00:00
2021-12-21 14:55:20 +00:00
void IMergeTreeDataPart::cleanupOldName(const String & old_part_name) const
{
2021-12-21 14:55:20 +00:00
if (name == old_part_name)
return;
2021-12-21 14:55:20 +00:00
storage.unlockSharedData(*this, old_part_name);
}
2021-06-09 12:36:47 +00:00
std::optional<bool> IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const
2021-06-08 19:11:22 +00:00
{
/// NOTE: It's needed for zero-copy replication
2021-06-09 12:36:47 +00:00
if (force_keep_shared_data)
return true;
2022-02-08 11:05:01 +00:00
return !storage.unlockSharedData(*this);
2021-06-08 19:11:22 +00:00
}
void IMergeTreeDataPart::initializePartMetadataManager()
{
#if USE_ROCKSDB
if (use_metadata_cache)
metadata_manager = std::make_shared<PartMetadataManagerWithCache>(this, storage.getContext()->getMergeTreeMetadataCache());
else
metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(this);
#else
metadata_manager = std::make_shared<PartMetadataManagerOrdinary>(this);
#endif
}
2021-06-09 12:36:47 +00:00
void IMergeTreeDataPart::remove() const
2019-10-31 14:44:17 +00:00
{
2021-06-09 12:36:47 +00:00
std::optional<bool> keep_shared_data = keepSharedDataInDecoupledStorage();
if (!keep_shared_data.has_value())
return;
2019-10-31 14:44:17 +00:00
if (!isStoredOnDisk())
return;
if (isProjectionPart())
2022-04-21 19:19:13 +00:00
LOG_WARNING(storage.log, "Projection part {} should be removed by its parent {}.", name, parent_part->name);
metadata_manager->deleteAll(false);
metadata_manager->assertAllDeleted(false);
2021-12-08 02:40:59 +00:00
2022-04-19 19:34:41 +00:00
std::list<IDataPartStorage::ProjectionChecksums> projection_checksums;
2019-10-31 14:44:17 +00:00
for (const auto & [p_name, projection_part] : projection_parts)
{
2022-04-19 19:34:41 +00:00
projection_part->metadata_manager->deleteAll(false);
projection_part->metadata_manager->assertAllDeleted(false);
2022-04-22 16:58:09 +00:00
projection_checksums.emplace_back(IDataPartStorage::ProjectionChecksums{.name = p_name, .checksums = projection_part->checksums});
}
2022-04-19 19:34:41 +00:00
data_part_storage->remove(*keep_shared_data, checksums, projection_checksums, storage.log);
2019-10-31 14:44:17 +00:00
}
String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool detached) const
2019-10-10 16:30:30 +00:00
{
String res;
/** If you need to detach a part, and directory into which we want to rename it already exists,
* we will rename to the directory with the name to which the suffix is added in the form of "_tryN".
* This is done only in the case of `to_detached`, because it is assumed that in this case the exact name does not matter.
* No more than 10 attempts are made so that there are not too many junk directories left.
*/
if (detached && parent_part)
2021-12-01 15:00:40 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot detach projection");
2019-10-10 16:30:30 +00:00
2022-04-19 19:34:41 +00:00
return data_part_storage->getRelativePathForPrefix(storage.log, prefix, detached);
2019-10-10 16:30:30 +00:00
}
2020-06-03 09:51:23 +00:00
String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const
2019-10-10 16:30:30 +00:00
{
2020-06-03 22:00:02 +00:00
/// Do not allow underscores in the prefix because they are used as separators.
assert(prefix.find_first_of('_') == String::npos);
assert(prefix.empty() || std::find(DetachedPartInfo::DETACH_REASONS.begin(),
DetachedPartInfo::DETACH_REASONS.end(),
prefix) != DetachedPartInfo::DETACH_REASONS.end());
return "detached/" + getRelativePathForPrefix(prefix, /* detached */ true);
2019-10-10 16:30:30 +00:00
}
void IMergeTreeDataPart::renameToDetached(const String & prefix) const
{
2020-06-03 22:00:02 +00:00
renameTo(getRelativePathForDetachedPart(prefix), true);
2019-10-10 16:30:30 +00:00
}
2020-06-26 11:30:23 +00:00
void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const
2019-10-10 16:30:30 +00:00
{
2022-04-19 19:34:41 +00:00
data_part_storage->freeze(storage.relative_data_path, getRelativePathForDetachedPart(prefix), {});
2019-10-10 16:30:30 +00:00
}
2022-04-22 16:58:09 +00:00
DataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const
2019-10-10 16:30:30 +00:00
{
assertOnDisk();
2022-04-19 19:34:41 +00:00
if (disk->getName() == data_part_storage->getName())
throw Exception("Can not clone data part " + name + " to same disk " + data_part_storage->getName(), ErrorCodes::LOGICAL_ERROR);
if (directory_name.empty())
throw Exception("Can not clone data part " + name + " to empty directory.", ErrorCodes::LOGICAL_ERROR);
2019-10-10 16:30:30 +00:00
String path_to_clone = fs::path(storage.relative_data_path) / directory_name / "";
2022-04-22 16:58:09 +00:00
return data_part_storage->clone(path_to_clone, data_part_storage->getRelativePath(), storage.log);
2019-10-10 16:30:30 +00:00
}
void IMergeTreeDataPart::checkConsistencyBase() const
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part)
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
else
{
// No need to check projections here because we already did consistent checking when loading projections if necessary.
}
2020-06-17 12:39:20 +00:00
const auto & pk = metadata_snapshot->getPrimaryKey();
const auto & partition_key = metadata_snapshot->getPartitionKey();
if (!checksums.empty())
{
2020-05-26 13:46:19 +00:00
if (!pk.column_names.empty() && !checksums.files.count("primary.idx"))
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
if (!checksums.files.count("count.txt"))
throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART);
if (metadata_snapshot->hasPartitionKey() && !checksums.files.count("partition.dat"))
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
if (!isEmpty() && !parent_part)
{
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
{
if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx"))
throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART);
}
}
}
2022-04-19 19:34:41 +00:00
data_part_storage->checkConsistency(checksums);
}
else
{
2022-04-19 19:34:41 +00:00
auto check_file_not_empty = [this](const String & file_path)
2020-02-27 17:57:49 +00:00
{
UInt64 file_size;
2022-04-19 19:34:41 +00:00
if (!data_part_storage->exists(file_path) || (file_size = data_part_storage->getFileSize(file_path)) == 0)
throw Exception(
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
"Part {} is broken: {} is empty",
data_part_storage->getFullPath(),
std::string(fs::path(data_part_storage->getFullPath()) / file_path));
return file_size;
};
/// Check that the primary key index is not empty.
2020-05-26 13:46:19 +00:00
if (!pk.column_names.empty())
2022-04-19 19:34:41 +00:00
check_file_not_empty("primary.idx");
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
2022-04-19 19:34:41 +00:00
check_file_not_empty("count.txt");
if (metadata_snapshot->hasPartitionKey())
2022-04-19 19:34:41 +00:00
check_file_not_empty("partition.dat");
if (!parent_part)
{
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
2022-04-19 19:34:41 +00:00
check_file_not_empty("minmax_" + escapeForFileName(col_name) + ".idx");
}
}
}
}
2020-06-03 18:59:18 +00:00
void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const
{
throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
}
void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk()
{
calculateColumnsSizesOnDisk();
calculateSecondaryIndicesSizesOnDisk();
}
2020-03-23 12:19:43 +00:00
void IMergeTreeDataPart::calculateColumnsSizesOnDisk()
{
if (getColumns().empty() || checksums.empty())
throw Exception("Cannot calculate columns sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR);
2020-06-29 20:36:18 +00:00
calculateEachColumnSizes(columns_sizes, total_columns_size);
2020-03-23 12:19:43 +00:00
}
void IMergeTreeDataPart::calculateSecondaryIndicesSizesOnDisk()
{
if (checksums.empty())
throw Exception("Cannot calculate secondary indexes sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR);
auto secondary_indices_descriptions = storage.getInMemoryMetadataPtr()->secondary_indices;
for (auto & index_description : secondary_indices_descriptions)
{
ColumnSize index_size;
auto index_ptr = MergeTreeIndexFactory::instance().get(index_description);
auto index_name = index_ptr->getFileName();
auto index_name_escaped = escapeForFileName(index_name);
auto index_file_name = index_name_escaped + index_ptr->getSerializedFileExtension();
auto index_marks_file_name = index_name_escaped + index_granularity_info.marks_file_extension;
2021-10-11 11:00:10 +00:00
/// If part does not contain index
auto bin_checksum = checksums.files.find(index_file_name);
if (bin_checksum != checksums.files.end())
{
index_size.data_compressed = bin_checksum->second.file_size;
index_size.data_uncompressed = bin_checksum->second.uncompressed_size;
}
auto mrk_checksum = checksums.files.find(index_marks_file_name);
if (mrk_checksum != checksums.files.end())
index_size.marks = mrk_checksum->second.file_size;
total_secondary_indices_size.add(index_size);
secondary_index_sizes[index_description.name] = index_size;
}
}
ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name) const
2020-03-23 12:19:43 +00:00
{
/// For some types of parts columns_size maybe not calculated
auto it = columns_sizes.find(column_name);
if (it != columns_sizes.end())
return it->second;
return ColumnSize{};
}
IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_index_name) const
{
auto it = secondary_index_sizes.find(secondary_index_name);
if (it != secondary_index_sizes.end())
return it->second;
return ColumnSize{};
}
2020-03-23 12:19:43 +00:00
void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const
{
2020-03-23 15:43:20 +00:00
for (const auto & [column_name, size] : columns_sizes)
column_to_size[column_name] = size.data_compressed;
2020-03-23 12:19:43 +00:00
}
bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const
{
if (!metadata_snapshot->hasAnyTTL())
return false;
if (metadata_snapshot->hasRowsTTL())
{
if (isEmpty()) /// All rows were finally deleted and we don't store TTL
return true;
else if (ttl_infos.table_ttl.min == 0)
return false;
}
for (const auto & [column, desc] : metadata_snapshot->getColumnTTLs())
{
/// Part has this column, but we don't calculated TTL for it
if (!ttl_infos.columns_ttl.count(column) && getColumns().contains(column))
return false;
}
for (const auto & move_desc : metadata_snapshot->getMoveTTLs())
{
/// Move TTL is not calculated
if (!ttl_infos.moves_ttl.count(move_desc.result_column))
return false;
}
2020-12-25 14:52:46 +00:00
for (const auto & group_by_desc : metadata_snapshot->getGroupByTTLs())
{
if (!ttl_infos.group_by_ttl.count(group_by_desc.result_column))
return false;
}
2021-01-13 14:04:27 +00:00
for (const auto & rows_where_desc : metadata_snapshot->getRowsWhereTTLs())
{
if (!ttl_infos.rows_where_ttl.count(rows_where_desc.result_column))
return false;
}
return true;
}
String IMergeTreeDataPart::getUniqueId() const
{
2022-04-19 19:34:41 +00:00
return data_part_storage->getUniqueId();
}
2022-02-02 16:44:29 +00:00
String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const
2021-03-31 15:20:30 +00:00
{
if (info.level != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to get block id for non zero level part {}", name);
SipHash hash;
if (token.empty())
{
checksums.computeTotalChecksumDataOnly(hash);
}
else
{
hash.update(token.data(), token.size());
}
2021-03-31 15:20:30 +00:00
union
{
char bytes[16];
UInt64 words[2];
} hash_value;
hash.get128(hash_value.bytes);
return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
}
2021-12-08 02:40:59 +00:00
IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_path) const
{
assert(use_metadata_cache);
2021-12-08 02:40:59 +00:00
String file_name = std::filesystem::path(file_path).filename();
const auto filenames_without_checksums = getFileNamesWithoutChecksums();
auto it = checksums.files.find(file_name);
if (filenames_without_checksums.count(file_name) == 0 && it != checksums.files.end())
{
return it->second.file_hash;
}
2022-04-19 19:34:41 +00:00
if (!data_part_storage->exists(file_path))
2021-12-08 02:40:59 +00:00
{
return {};
}
2022-04-19 19:34:41 +00:00
std::unique_ptr<ReadBufferFromFileBase> in_file = data_part_storage->readFile(file_path, {}, std::nullopt, std::nullopt);
2021-12-08 02:40:59 +00:00
HashingReadBuffer in_hash(*in_file);
String value;
readStringUntilEOF(value, in_hash);
return in_hash.getHash();
}
2022-01-07 10:37:08 +00:00
std::unordered_map<String, IMergeTreeDataPart::uint128> IMergeTreeDataPart::checkMetadata() const
2021-12-08 02:40:59 +00:00
{
2022-01-07 10:37:08 +00:00
return metadata_manager->check();
2021-12-08 02:40:59 +00:00
}
bool isCompactPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT);
}
bool isWidePart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::WIDE);
}
bool isInMemoryPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::IN_MEMORY);
}
2019-10-10 16:30:30 +00:00
}