mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Add column modification time into system.parts_columns
This can be useful to obtain at least some time for the part after mutations, since mutations will change the modification time of all parts. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
parent
e18b95b19d
commit
22e44ced8a
@ -80,6 +80,11 @@ DataPartStorageIteratorPtr DataPartStorageOnDiskFull::iterate() const
|
||||
volume->getDisk()->iterateDirectory(fs::path(root_path) / part_dir));
|
||||
}
|
||||
|
||||
Poco::Timestamp DataPartStorageOnDiskFull::getFileLastModified(const String & file_name) const
|
||||
{
|
||||
return volume->getDisk()->getLastModified(fs::path(root_path) / part_dir / file_name);
|
||||
}
|
||||
|
||||
size_t DataPartStorageOnDiskFull::getFileSize(const String & file_name) const
|
||||
{
|
||||
return volume->getDisk()->getFileSize(fs::path(root_path) / part_dir / file_name);
|
||||
|
@ -20,6 +20,7 @@ public:
|
||||
bool isDirectory(const std::string & name) const override;
|
||||
|
||||
DataPartStorageIteratorPtr iterate() const override;
|
||||
Poco::Timestamp getFileLastModified(const String & file_name) const override;
|
||||
size_t getFileSize(const std::string & file_name) const override;
|
||||
UInt32 getRefCount(const std::string & file_name) const override;
|
||||
std::string getRemotePath(const std::string & file_name) const override;
|
||||
|
@ -108,6 +108,7 @@ public:
|
||||
virtual DataPartStorageIteratorPtr iterate() const = 0;
|
||||
|
||||
/// Get metadata for a file inside path dir.
|
||||
virtual Poco::Timestamp getFileLastModified(const std::string & file_name) const = 0;
|
||||
virtual size_t getFileSize(const std::string & file_name) const = 0;
|
||||
virtual UInt32 getRefCount(const std::string & file_name) const = 0;
|
||||
|
||||
|
@ -116,6 +116,8 @@ public:
|
||||
/// Otherwise return information about column size on disk.
|
||||
ColumnSize getColumnSize(const String & column_name) const;
|
||||
|
||||
virtual std::optional<time_t> getColumnModificationTime(const String & column_name) const = 0;
|
||||
|
||||
/// NOTE: Returns zeros if secondary indexes are not found in checksums.
|
||||
/// Otherwise return information about secondary index size on disk.
|
||||
IndexSize getSecondaryIndexSize(const String & secondary_index_name) const;
|
||||
|
@ -144,6 +144,11 @@ bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) co
|
||||
return (bin_checksum != checksums.files.end() && mrk_checksum != checksums.files.end());
|
||||
}
|
||||
|
||||
std::optional<time_t> MergeTreeDataPartCompact::getColumnModificationTime(const String & /* column_name */) const
|
||||
{
|
||||
return getDataPartStorage().getFileLastModified(DATA_FILE_NAME_WITH_EXTENSION).epochTime();
|
||||
}
|
||||
|
||||
void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) const
|
||||
{
|
||||
checkConsistencyBase();
|
||||
|
@ -55,6 +55,8 @@ public:
|
||||
|
||||
bool hasColumnFiles(const NameAndTypePair & column) const override;
|
||||
|
||||
std::optional<time_t> getColumnModificationTime(const String & column_name) const override;
|
||||
|
||||
String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; }
|
||||
|
||||
~MergeTreeDataPartCompact() override;
|
||||
|
@ -43,6 +43,7 @@ public:
|
||||
String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; }
|
||||
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override;
|
||||
DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
std::optional<time_t> getColumnModificationTime(const String & /* column_name */) const override { return {}; }
|
||||
|
||||
MutableDataPartStoragePtr flushToDisk(const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
|
@ -260,6 +260,18 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const
|
||||
return res;
|
||||
}
|
||||
|
||||
std::optional<time_t> MergeTreeDataPartWide::getColumnModificationTime(const String & column_name) const
|
||||
{
|
||||
try
|
||||
{
|
||||
return getDataPartStorage().getFileLastModified(column_name + DATA_FILE_EXTENSION).epochTime();
|
||||
}
|
||||
catch (const fs::filesystem_error &)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
}
|
||||
|
||||
String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const
|
||||
{
|
||||
String filename;
|
||||
|
@ -54,6 +54,8 @@ public:
|
||||
|
||||
bool hasColumnFiles(const NameAndTypePair & column) const override;
|
||||
|
||||
std::optional<time_t> getColumnModificationTime(const String & column_name) const override;
|
||||
|
||||
protected:
|
||||
static void loadIndexGranularityImpl(
|
||||
MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_,
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNested.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
@ -62,6 +63,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_
|
||||
{"column_data_compressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_data_uncompressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_marks_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_modification_time", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>())},
|
||||
|
||||
{"serialization_kind", std::make_shared<DataTypeString>()},
|
||||
{"subcolumns.names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"subcolumns.types", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
@ -235,6 +238,13 @@ void StorageSystemPartsColumns::processNextStorage(
|
||||
columns[res_index++]->insert(column_size.data_uncompressed);
|
||||
if (columns_mask[src_index++])
|
||||
columns[res_index++]->insert(column_size.marks);
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (auto column_modification_time = part->getColumnModificationTime(column.name))
|
||||
columns[res_index++]->insert(UInt64(column_modification_time.value()));
|
||||
else
|
||||
columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
auto serialization = part->getSerialization(column.name);
|
||||
if (columns_mask[src_index++])
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
@ -66,7 +67,8 @@ StorageSystemProjectionPartsColumns::StorageSystemProjectionPartsColumns(const S
|
||||
{"column_bytes_on_disk", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_data_compressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_data_uncompressed_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_marks_bytes", std::make_shared<DataTypeUInt64>()}
|
||||
{"column_marks_bytes", std::make_shared<DataTypeUInt64>()},
|
||||
{"column_modification_time", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>())},
|
||||
}
|
||||
)
|
||||
{
|
||||
@ -247,6 +249,13 @@ void StorageSystemProjectionPartsColumns::processNextStorage(
|
||||
columns[res_index++]->insert(column_size.data_uncompressed);
|
||||
if (columns_mask[src_index++])
|
||||
columns[res_index++]->insert(column_size.marks);
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (auto column_modification_time = part->getColumnModificationTime(column.name))
|
||||
columns[res_index++]->insert(UInt64(column_modification_time.value()));
|
||||
else
|
||||
columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
if (has_state_column)
|
||||
columns[res_index++]->insert(part->stateString());
|
||||
|
@ -565,6 +565,7 @@ CREATE TABLE system.parts_columns
|
||||
`column_data_compressed_bytes` UInt64,
|
||||
`column_data_uncompressed_bytes` UInt64,
|
||||
`column_marks_bytes` UInt64,
|
||||
`column_modification_time` Nullable(DateTime),
|
||||
`serialization_kind` String,
|
||||
`subcolumns.names` Array(String),
|
||||
`subcolumns.types` Array(String),
|
||||
@ -750,6 +751,7 @@ CREATE TABLE system.projection_parts_columns
|
||||
`column_data_compressed_bytes` UInt64,
|
||||
`column_data_uncompressed_bytes` UInt64,
|
||||
`column_marks_bytes` UInt64,
|
||||
`column_modification_time` Nullable(DateTime),
|
||||
`bytes` UInt64,
|
||||
`marks_size` UInt64,
|
||||
`part_name` String
|
||||
|
@ -0,0 +1,6 @@
|
||||
Wide key 1 1
|
||||
Wide key 1 1
|
||||
Wide value 1 0
|
||||
Compact key 1 1
|
||||
Compact key 1 1
|
||||
Compact value 1 1
|
@ -0,0 +1,30 @@
|
||||
-- Tags: no-s3-storage
|
||||
-- Tag: no-s3-storage because S3 updates metadata for the virtual link file on metadata disk (see CreateHardlinkOperation::execute() for details)
|
||||
|
||||
set mutations_sync=1;
|
||||
|
||||
{# modification time of the part folder and column files not always equal, this is how much seconds of difference is allowed #}
|
||||
{% set mtime_diff_in_seconds = 5 %}
|
||||
|
||||
{% for id, settings, file_per_column in [
|
||||
("wide", "min_bytes_for_wide_part=0, min_rows_for_wide_part=0", true),
|
||||
("compact", "min_bytes_for_wide_part=1000, min_rows_for_wide_part=100", false)
|
||||
]
|
||||
%}
|
||||
|
||||
drop table if exists data_{{ id }};
|
||||
create table data_{{ id }} (key Int) engine=MergeTree() order by tuple() settings {{ settings }};
|
||||
insert into data_{{ id }} values (1);
|
||||
select sleep(3) format Null;
|
||||
select part_type, column, now()-modification_time < 10, modification_time - column_modification_time < {{ mtime_diff_in_seconds }} from system.parts_columns where database = currentDatabase() and table = 'data_{{ id }}';
|
||||
alter table data_{{ id }} add column value Int default 0;
|
||||
alter table data_{{ id }} materialize column value;
|
||||
select part_type, column, now()-modification_time < 10,
|
||||
{% if file_per_column %}
|
||||
modification_time - column_modification_time >= 3
|
||||
{% else %}
|
||||
modification_time - column_modification_time < {{ mtime_diff_in_seconds }}
|
||||
{% endif %}
|
||||
from system.parts_columns where active and database = currentDatabase() and table = 'data_{{ id }}' order by column;
|
||||
|
||||
{% endfor %}
|
Loading…
Reference in New Issue
Block a user