mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #26241 from evillique/column_size
Add column sizes for `Log` and `TinyLog` tables
This commit is contained in:
commit
f5f9e35331
@ -399,7 +399,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
view = nullptr;
|
||||
}
|
||||
|
||||
if (try_move_to_prewhere && storage && query.where() && !query.prewhere())
|
||||
if (try_move_to_prewhere && storage && storage->supportsPrewhere() && query.where() && !query.prewhere())
|
||||
{
|
||||
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
|
||||
if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty())
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Processors/Pipe.h>
|
||||
|
||||
#include <cassert>
|
||||
#include <chrono>
|
||||
|
||||
|
||||
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
||||
@ -719,6 +720,34 @@ CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr contex
|
||||
}
|
||||
|
||||
|
||||
IStorage::ColumnSizeByName StorageLog::getColumnSizes() const
|
||||
{
|
||||
std::shared_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC));
|
||||
if (!lock)
|
||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
|
||||
ColumnSizeByName column_sizes;
|
||||
FileChecker::Map file_sizes = file_checker.getFileSizes();
|
||||
|
||||
for (const auto & column : getInMemoryMetadata().getColumns().getAllPhysical())
|
||||
{
|
||||
ISerialization::StreamCallback stream_callback = [&, this] (const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
|
||||
ColumnSize & size = column_sizes[column.name];
|
||||
auto it = files.find(stream_name);
|
||||
if (it != files.end())
|
||||
size.data_compressed += file_sizes[fileName(it->second.data_file_path)];
|
||||
};
|
||||
|
||||
ISerialization::SubstreamPath substream_path;
|
||||
auto serialization = column.type->getDefaultSerialization();
|
||||
serialization->enumerateStreams(stream_callback, substream_path);
|
||||
}
|
||||
|
||||
return column_sizes;
|
||||
}
|
||||
|
||||
void registerStorageLog(StorageFactory & factory)
|
||||
{
|
||||
StorageFactory::StorageFeatures features{
|
||||
|
@ -45,6 +45,7 @@ public:
|
||||
bool storesDataOnDisk() const override { return true; }
|
||||
Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; }
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
ColumnSizeByName getColumnSizes() const override;
|
||||
|
||||
protected:
|
||||
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
||||
@ -87,7 +88,7 @@ private:
|
||||
DiskPtr disk;
|
||||
String table_path;
|
||||
|
||||
std::shared_timed_mutex rwlock;
|
||||
mutable std::shared_timed_mutex rwlock;
|
||||
|
||||
Files files;
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#include <map>
|
||||
#include <cassert>
|
||||
#include <chrono>
|
||||
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
@ -523,6 +524,34 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, ContextPtr co
|
||||
return file_checker.check();
|
||||
}
|
||||
|
||||
IStorage::ColumnSizeByName StorageTinyLog::getColumnSizes() const
|
||||
{
|
||||
std::shared_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC));
|
||||
if (!lock)
|
||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
|
||||
ColumnSizeByName column_sizes;
|
||||
FileChecker::Map file_sizes = file_checker.getFileSizes();
|
||||
|
||||
for (const auto & column : getInMemoryMetadata().getColumns().getAllPhysical())
|
||||
{
|
||||
ISerialization::StreamCallback stream_callback = [&, this] (const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
|
||||
ColumnSize & size = column_sizes[column.name];
|
||||
auto it = files.find(stream_name);
|
||||
if (it != files.end())
|
||||
size.data_compressed += file_sizes[fileName(it->second.data_file_path)];
|
||||
};
|
||||
|
||||
ISerialization::SubstreamPath substream_path;
|
||||
auto serialization = column.type->getDefaultSerialization();
|
||||
serialization->enumerateStreams(stream_callback, substream_path);
|
||||
}
|
||||
|
||||
return column_sizes;
|
||||
}
|
||||
|
||||
void StorageTinyLog::truncate(
|
||||
const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &)
|
||||
{
|
||||
|
@ -45,6 +45,7 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override;
|
||||
|
||||
ColumnSizeByName getColumnSizes() const override;
|
||||
protected:
|
||||
StorageTinyLog(
|
||||
DiskPtr disk_,
|
||||
@ -71,7 +72,7 @@ private:
|
||||
Files files;
|
||||
|
||||
FileChecker file_checker;
|
||||
std::shared_timed_mutex rwlock;
|
||||
mutable std::shared_timed_mutex rwlock;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
|
@ -98,7 +98,7 @@ protected:
|
||||
Names cols_required_for_sorting_key;
|
||||
Names cols_required_for_primary_key;
|
||||
Names cols_required_for_sampling;
|
||||
MergeTreeData::ColumnSizeByName column_sizes;
|
||||
IStorage::ColumnSizeByName column_sizes;
|
||||
|
||||
{
|
||||
StoragePtr storage = storages.at(std::make_pair(database_name, table_name));
|
||||
|
@ -0,0 +1,6 @@
|
||||
27
|
||||
33
|
||||
105
|
||||
27
|
||||
33
|
||||
105
|
14
tests/queries/0_stateless/01943_log_column_sizes.sql
Normal file
14
tests/queries/0_stateless/01943_log_column_sizes.sql
Normal file
@ -0,0 +1,14 @@
|
||||
DROP TABLE IF EXISTS test_log;
|
||||
DROP TABLE IF EXISTS test_tiny_log;
|
||||
|
||||
CREATE TABLE test_log (x UInt8, s String, a Array(Nullable(String))) ENGINE = Log;
|
||||
CREATE TABLE test_tiny_log (x UInt8, s String, a Array(Nullable(String))) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO test_log VALUES (64, 'Value1', ['Value2', 'Value3', NULL]);
|
||||
INSERT INTO test_tiny_log VALUES (64, 'Value1', ['Value2', 'Value3', NULL]);
|
||||
|
||||
SELECT data_compressed_bytes FROM system.columns WHERE table = 'test_log' AND database = currentDatabase();
|
||||
SELECT data_compressed_bytes FROM system.columns WHERE table = 'test_tiny_log' AND database = currentDatabase();
|
||||
|
||||
DROP TABLE test_log;
|
||||
DROP TABLE test_tiny_log;
|
Loading…
Reference in New Issue
Block a user