mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +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;
|
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
|
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
|
||||||
if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty())
|
if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty())
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
|
|
||||||
#include <cassert>
|
#include <cassert>
|
||||||
|
#include <chrono>
|
||||||
|
|
||||||
|
|
||||||
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
#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)
|
void registerStorageLog(StorageFactory & factory)
|
||||||
{
|
{
|
||||||
StorageFactory::StorageFeatures features{
|
StorageFactory::StorageFeatures features{
|
||||||
|
@ -45,6 +45,7 @@ public:
|
|||||||
bool storesDataOnDisk() const override { return true; }
|
bool storesDataOnDisk() const override { return true; }
|
||||||
Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; }
|
Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; }
|
||||||
bool supportsSubcolumns() const override { return true; }
|
bool supportsSubcolumns() const override { return true; }
|
||||||
|
ColumnSizeByName getColumnSizes() const override;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
||||||
@ -87,7 +88,7 @@ private:
|
|||||||
DiskPtr disk;
|
DiskPtr disk;
|
||||||
String table_path;
|
String table_path;
|
||||||
|
|
||||||
std::shared_timed_mutex rwlock;
|
mutable std::shared_timed_mutex rwlock;
|
||||||
|
|
||||||
Files files;
|
Files files;
|
||||||
|
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
|
|
||||||
#include <map>
|
#include <map>
|
||||||
#include <cassert>
|
#include <cassert>
|
||||||
|
#include <chrono>
|
||||||
|
|
||||||
#include <Poco/Util/XMLConfiguration.h>
|
#include <Poco/Util/XMLConfiguration.h>
|
||||||
|
|
||||||
@ -523,6 +524,34 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, ContextPtr co
|
|||||||
return file_checker.check();
|
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(
|
void StorageTinyLog::truncate(
|
||||||
const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &)
|
const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &)
|
||||||
{
|
{
|
||||||
|
@ -45,6 +45,7 @@ public:
|
|||||||
|
|
||||||
void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override;
|
void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override;
|
||||||
|
|
||||||
|
ColumnSizeByName getColumnSizes() const override;
|
||||||
protected:
|
protected:
|
||||||
StorageTinyLog(
|
StorageTinyLog(
|
||||||
DiskPtr disk_,
|
DiskPtr disk_,
|
||||||
@ -71,7 +72,7 @@ private:
|
|||||||
Files files;
|
Files files;
|
||||||
|
|
||||||
FileChecker file_checker;
|
FileChecker file_checker;
|
||||||
std::shared_timed_mutex rwlock;
|
mutable std::shared_timed_mutex rwlock;
|
||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
|
@ -98,7 +98,7 @@ protected:
|
|||||||
Names cols_required_for_sorting_key;
|
Names cols_required_for_sorting_key;
|
||||||
Names cols_required_for_primary_key;
|
Names cols_required_for_primary_key;
|
||||||
Names cols_required_for_sampling;
|
Names cols_required_for_sampling;
|
||||||
MergeTreeData::ColumnSizeByName column_sizes;
|
IStorage::ColumnSizeByName column_sizes;
|
||||||
|
|
||||||
{
|
{
|
||||||
StoragePtr storage = storages.at(std::make_pair(database_name, table_name));
|
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