2017-09-11 17:55:41 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreePartition.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2019-10-10 16:30:30 +00:00
|
|
|
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
2017-09-11 17:55:41 +00:00
|
|
|
#include <IO/HashingWriteBuffer.h>
|
2017-11-24 13:55:31 +00:00
|
|
|
#include <Common/FieldVisitors.h>
|
2017-09-11 17:55:41 +00:00
|
|
|
#include <DataTypes/DataTypeDate.h>
|
2018-11-26 12:41:17 +00:00
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
2017-09-11 17:55:41 +00:00
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <Common/hex.h>
|
2019-01-25 15:17:12 +00:00
|
|
|
#include <Core/Block.h>
|
2017-09-11 17:55:41 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2020-02-25 18:10:48 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
2017-09-11 17:55:41 +00:00
|
|
|
|
2020-02-27 16:47:40 +00:00
|
|
|
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2020-02-27 16:47:40 +00:00
|
|
|
return disk->readFile(path, std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), disk->getFileSize(path)));
|
2017-09-11 17:55:41 +00:00
|
|
|
}
|
|
|
|
|
2019-01-25 15:17:12 +00:00
|
|
|
String MergeTreePartition::getID(const MergeTreeData & storage) const
|
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
return getID(storage.getPartitionKey().sample_block);
|
2019-01-25 15:17:12 +00:00
|
|
|
}
|
|
|
|
|
2017-09-13 16:22:04 +00:00
|
|
|
/// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system.
|
|
|
|
/// So if you want to change this method, be sure to guarantee compatibility with existing table data.
|
2019-01-25 15:17:12 +00:00
|
|
|
String MergeTreePartition::getID(const Block & partition_key_sample) const
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2019-01-25 15:17:12 +00:00
|
|
|
if (value.size() != partition_key_sample.columns())
|
2017-09-11 17:55:41 +00:00
|
|
|
throw Exception("Invalid partition key size: " + toString(value.size()), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
if (value.empty())
|
2017-09-13 16:22:04 +00:00
|
|
|
return "all"; /// It is tempting to use an empty string here. But that would break directory structure in ZK.
|
2017-09-11 17:55:41 +00:00
|
|
|
|
2017-09-13 16:22:04 +00:00
|
|
|
/// In case all partition fields are represented by integral types, try to produce a human-readable ID.
|
2017-09-11 17:55:41 +00:00
|
|
|
/// Otherwise use a hex-encoded hash.
|
|
|
|
bool are_all_integral = true;
|
|
|
|
for (const Field & field : value)
|
|
|
|
{
|
|
|
|
if (field.getType() != Field::Types::UInt64 && field.getType() != Field::Types::Int64)
|
|
|
|
{
|
|
|
|
are_all_integral = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
String result;
|
|
|
|
|
|
|
|
if (are_all_integral)
|
|
|
|
{
|
|
|
|
FieldVisitorToString to_string_visitor;
|
|
|
|
for (size_t i = 0; i < value.size(); ++i)
|
|
|
|
{
|
|
|
|
if (i > 0)
|
|
|
|
result += '-';
|
|
|
|
|
2019-01-25 15:17:12 +00:00
|
|
|
if (typeid_cast<const DataTypeDate *>(partition_key_sample.getByPosition(i).type.get()))
|
2018-05-25 13:29:15 +00:00
|
|
|
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>())));
|
2017-09-11 17:55:41 +00:00
|
|
|
else
|
|
|
|
result += applyVisitor(to_string_visitor, value[i]);
|
2017-09-13 16:22:04 +00:00
|
|
|
|
|
|
|
/// It is tempting to output DateTime as YYYYMMDDhhmmss, but that would make partition ID
|
|
|
|
/// timezone-dependent.
|
2017-09-11 17:55:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
SipHash hash;
|
|
|
|
FieldVisitorHash hashing_visitor(hash);
|
|
|
|
for (const Field & field : value)
|
|
|
|
applyVisitor(hashing_visitor, field);
|
|
|
|
|
|
|
|
char hash_data[16];
|
|
|
|
hash.get128(hash_data);
|
|
|
|
result.resize(32);
|
|
|
|
for (size_t i = 0; i < 16; ++i)
|
|
|
|
writeHexByteLowercase(hash_data[i], &result[2 * i]);
|
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
2018-10-09 18:32:44 +00:00
|
|
|
void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
const auto & partition_key_sample = storage.getPartitionKey().sample_block;
|
|
|
|
size_t key_size = partition_key_sample.columns();
|
2017-09-11 17:55:41 +00:00
|
|
|
|
|
|
|
if (key_size == 0)
|
|
|
|
{
|
|
|
|
writeCString("tuple()", out);
|
|
|
|
}
|
2018-10-09 18:32:44 +00:00
|
|
|
else if (key_size == 1)
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
const DataTypePtr & type = partition_key_sample.getByPosition(0).type;
|
2017-12-15 20:48:46 +00:00
|
|
|
auto column = type->createColumn();
|
2018-10-09 18:32:44 +00:00
|
|
|
column->insert(value[0]);
|
2018-12-13 13:41:47 +00:00
|
|
|
type->serializeAsText(*column, 0, out, format_settings);
|
2017-09-11 17:55:41 +00:00
|
|
|
}
|
2018-10-09 18:32:44 +00:00
|
|
|
else
|
|
|
|
{
|
2018-11-26 12:41:17 +00:00
|
|
|
DataTypes types;
|
|
|
|
Columns columns;
|
2018-10-09 18:32:44 +00:00
|
|
|
for (size_t i = 0; i < key_size; ++i)
|
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
const auto & type = partition_key_sample.getByPosition(i).type;
|
2018-11-26 12:41:17 +00:00
|
|
|
types.push_back(type);
|
2018-10-09 18:32:44 +00:00
|
|
|
auto column = type->createColumn();
|
|
|
|
column->insert(value[i]);
|
2018-11-26 12:41:17 +00:00
|
|
|
columns.push_back(std::move(column));
|
2018-10-09 18:32:44 +00:00
|
|
|
}
|
2017-09-11 17:55:41 +00:00
|
|
|
|
2018-11-26 12:41:17 +00:00
|
|
|
DataTypeTuple tuple_type(types);
|
|
|
|
auto tuple_column = ColumnTuple::create(columns);
|
|
|
|
tuple_type.serializeText(*tuple_column, 0, out, format_settings);
|
2018-10-09 18:32:44 +00:00
|
|
|
}
|
2017-09-11 17:55:41 +00:00
|
|
|
}
|
|
|
|
|
2020-02-27 16:47:40 +00:00
|
|
|
void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path)
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
if (!storage.hasPartitionKey())
|
2017-09-11 17:55:41 +00:00
|
|
|
return;
|
|
|
|
|
2020-05-20 12:16:55 +00:00
|
|
|
const auto & partition_key_sample = storage.getPartitionKey().sample_block;
|
2020-02-27 16:47:40 +00:00
|
|
|
auto partition_file_path = part_path + "partition.dat";
|
|
|
|
auto file = openForReading(disk, partition_file_path);
|
2020-05-20 12:16:55 +00:00
|
|
|
value.resize(partition_key_sample.columns());
|
|
|
|
for (size_t i = 0; i < partition_key_sample.columns(); ++i)
|
|
|
|
partition_key_sample.getByPosition(i).type->deserializeBinary(value[i], *file);
|
2017-09-11 17:55:41 +00:00
|
|
|
}
|
|
|
|
|
2020-02-27 16:47:40 +00:00
|
|
|
void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
2017-09-11 17:55:41 +00:00
|
|
|
{
|
2020-05-20 12:16:55 +00:00
|
|
|
store(storage.getPartitionKey().sample_block, disk, part_path, checksums);
|
2019-01-25 15:17:12 +00:00
|
|
|
}
|
|
|
|
|
2020-02-27 16:47:40 +00:00
|
|
|
void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
2019-01-25 15:17:12 +00:00
|
|
|
{
|
|
|
|
if (!partition_key_sample)
|
2017-09-11 17:55:41 +00:00
|
|
|
return;
|
|
|
|
|
2020-02-27 16:47:40 +00:00
|
|
|
auto out = disk->writeFile(part_path + "partition.dat");
|
|
|
|
HashingWriteBuffer out_hashing(*out);
|
2017-09-11 17:55:41 +00:00
|
|
|
for (size_t i = 0; i < value.size(); ++i)
|
2019-01-25 15:17:12 +00:00
|
|
|
partition_key_sample.getByPosition(i).type->serializeBinary(value[i], out_hashing);
|
2017-10-24 14:11:53 +00:00
|
|
|
out_hashing.next();
|
2017-09-11 17:55:41 +00:00
|
|
|
checksums.files["partition.dat"].file_size = out_hashing.count();
|
|
|
|
checksums.files["partition.dat"].file_hash = out_hashing.getHash();
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|