Merge pull request #26241 from evillique/column_size

Add column sizes for `Log` and `TinyLog` tables
This commit is contained in:
alexey-milovidov 2021-07-13 03:54:03 +03:00 committed by GitHub
commit f5f9e35331
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 84 additions and 4 deletions

View File

@ -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())

View File

@ -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{

View File

@ -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;

View File

@ -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 &)
{

View File

@ -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;

View File

@ -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));

View File

@ -0,0 +1,6 @@
27
33
105
27
33
105

View 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;