mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #48990 from incfly/partition
Add partition to the system part_log and merge table.
This commit is contained in:
commit
7ad8ea78bb
@ -107,6 +107,7 @@ NamesAndTypesList PartLogElement::getNamesAndTypes()
|
||||
{"table_uuid", std::make_shared<DataTypeUUID>()},
|
||||
{"part_name", std::make_shared<DataTypeString>()},
|
||||
{"partition_id", std::make_shared<DataTypeString>()},
|
||||
{"partition", std::make_shared<DataTypeString>()},
|
||||
{"part_type", std::make_shared<DataTypeString>()},
|
||||
{"disk_name", std::make_shared<DataTypeString>()},
|
||||
{"path_on_disk", std::make_shared<DataTypeString>()},
|
||||
@ -157,6 +158,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const
|
||||
columns[i++]->insert(table_uuid);
|
||||
columns[i++]->insert(part_name);
|
||||
columns[i++]->insert(partition_id);
|
||||
columns[i++]->insert(partition);
|
||||
columns[i++]->insert(part_type.toString());
|
||||
columns[i++]->insert(disk_name);
|
||||
columns[i++]->insert(path_on_disk);
|
||||
@ -229,6 +231,10 @@ bool PartLog::addNewParts(
|
||||
elem.table_name = table_id.table_name;
|
||||
elem.table_uuid = table_id.uuid;
|
||||
elem.partition_id = part->info.partition_id;
|
||||
{
|
||||
WriteBufferFromString out(elem.partition);
|
||||
part->partition.serializeText(part->storage, out, {});
|
||||
}
|
||||
elem.part_name = part->name;
|
||||
elem.disk_name = part->getDataPartStorage().getDiskName();
|
||||
elem.path_on_disk = part->getDataPartStorage().getFullPath();
|
||||
|
@ -63,6 +63,7 @@ struct PartLogElement
|
||||
UUID table_uuid{UUIDHelpers::Nil};
|
||||
String part_name;
|
||||
String partition_id;
|
||||
String partition;
|
||||
String disk_name;
|
||||
String path_on_disk;
|
||||
|
||||
|
@ -1,20 +1,16 @@
|
||||
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
|
||||
#include <Storages/MergeTree/FutureMergedMutatedPart.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <base/getThreadId.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
MergeListElement::MergeListElement(
|
||||
const StorageID & table_id_,
|
||||
FutureMergedMutatedPartPtr future_part,
|
||||
const ContextPtr & context)
|
||||
MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context)
|
||||
: table_id{table_id_}
|
||||
, partition_id{future_part->part_info.partition_id}
|
||||
, result_part_name{future_part->name}
|
||||
@ -40,6 +36,10 @@ MergeListElement::MergeListElement(
|
||||
{
|
||||
source_data_version = future_part->parts[0]->info.getDataVersion();
|
||||
is_mutation = (result_part_info.getDataVersion() != source_data_version);
|
||||
|
||||
WriteBufferFromString out(partition);
|
||||
const auto & part = future_part->parts[0];
|
||||
part->partition.serializeText(part->storage, out, {});
|
||||
}
|
||||
|
||||
thread_group = ThreadGroup::createForBackgroundProcess(context);
|
||||
@ -53,6 +53,7 @@ MergeInfo MergeListElement::getInfo() const
|
||||
res.result_part_name = result_part_name;
|
||||
res.result_part_path = result_part_path;
|
||||
res.partition_id = partition_id;
|
||||
res.partition = partition;
|
||||
res.is_mutation = is_mutation;
|
||||
res.elapsed = watch.elapsedSeconds();
|
||||
res.progress = progress.load(std::memory_order_relaxed);
|
||||
|
@ -35,6 +35,7 @@ struct MergeInfo
|
||||
Array source_part_names;
|
||||
Array source_part_paths;
|
||||
std::string partition_id;
|
||||
std::string partition;
|
||||
bool is_mutation;
|
||||
Float64 elapsed;
|
||||
Float64 progress;
|
||||
@ -67,6 +68,7 @@ struct MergeListElement : boost::noncopyable
|
||||
{
|
||||
const StorageID table_id;
|
||||
std::string partition_id;
|
||||
std::string partition;
|
||||
|
||||
const std::string result_part_name;
|
||||
const std::string result_part_path;
|
||||
|
@ -371,6 +371,12 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe
|
||||
const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block;
|
||||
size_t key_size = partition_key_sample.columns();
|
||||
|
||||
// In some cases we create empty parts and then value is empty.
|
||||
if (value.empty())
|
||||
{
|
||||
writeCString("tuple()", out);
|
||||
return;
|
||||
}
|
||||
if (key_size == 0)
|
||||
{
|
||||
writeCString("tuple()", out);
|
||||
|
@ -20,6 +20,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes()
|
||||
{"source_part_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"result_part_path", std::make_shared<DataTypeString>()},
|
||||
{"partition_id", std::make_shared<DataTypeString>()},
|
||||
{"partition", std::make_shared<DataTypeString>()},
|
||||
{"is_mutation", std::make_shared<DataTypeUInt8>()},
|
||||
{"total_size_bytes_compressed", std::make_shared<DataTypeUInt64>()},
|
||||
{"total_size_bytes_uncompressed", std::make_shared<DataTypeUInt64>()},
|
||||
@ -58,6 +59,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr cont
|
||||
res_columns[i++]->insert(merge.source_part_paths);
|
||||
res_columns[i++]->insert(merge.result_part_path);
|
||||
res_columns[i++]->insert(merge.partition_id);
|
||||
res_columns[i++]->insert(merge.partition);
|
||||
res_columns[i++]->insert(merge.is_mutation);
|
||||
res_columns[i++]->insert(merge.total_size_bytes_compressed);
|
||||
res_columns[i++]->insert(merge.total_size_bytes_uncompressed);
|
||||
|
@ -56,6 +56,23 @@ def test_config_with_standard_part_log(start_cluster):
|
||||
assert node2.query("SELECT * FROM system.part_log") != ""
|
||||
|
||||
|
||||
def test_part_log_contains_partition(start_cluster):
|
||||
node2.query(
|
||||
"CREATE TABLE test_partition_table (date Date, word String, value UInt64) ENGINE=MergeTree() "
|
||||
+ "PARTITION BY toYYYYMM(date) Order by value"
|
||||
)
|
||||
node2.query(
|
||||
"INSERT INTO test_partition_table VALUES "
|
||||
+ "('2023-06-20', 'a', 10), ('2023-06-21', 'b', 11),"
|
||||
+ "('2023-05-20', 'cc', 14),('2023-05-21', 'd1', 15);"
|
||||
)
|
||||
node2.query("SYSTEM FLUSH LOGS")
|
||||
resp = node2.query(
|
||||
"SELECT partition from system.part_log where table = 'test_partition_table'"
|
||||
)
|
||||
assert resp == "202306\n202305\n"
|
||||
|
||||
|
||||
def test_config_with_non_standard_part_log(start_cluster):
|
||||
node3.query(
|
||||
"CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() Order by value"
|
||||
|
@ -364,6 +364,7 @@ CREATE TABLE system.merges
|
||||
`source_part_paths` Array(String),
|
||||
`result_part_path` String,
|
||||
`partition_id` String,
|
||||
`partition` String,
|
||||
`is_mutation` UInt8,
|
||||
`total_size_bytes_compressed` UInt64,
|
||||
`total_size_bytes_uncompressed` UInt64,
|
||||
|
Loading…
Reference in New Issue
Block a user