Pass Context parameter for IStorage.totalRows and IStorage.totalBytes

This commit is contained in:
hchen9 2020-09-30 16:47:42 -07:00
parent fda6bde428
commit a5ac39b564
10 changed files with 20 additions and 20 deletions

View File

@ -1125,7 +1125,7 @@ void InterpreterSelectQuery::executeFetchColumns(
{
const auto & desc = query_analyzer->aggregates()[0];
const auto & func = desc.function;
std::optional<UInt64> num_rows = storage->totalRows();
std::optional<UInt64> num_rows = storage->totalRows(context);
if (num_rows)
{
AggregateFunctionCount & agg_count = static_cast<AggregateFunctionCount &>(*func);

View File

@ -455,7 +455,7 @@ public:
/// - For total_rows column in system.tables
///
/// Does takes underlying Storage (if any) into account.
virtual std::optional<UInt64> totalRows() const { return {}; }
virtual std::optional<UInt64> totalRows(const Context &) const { return {}; }
/// If it is possible to quickly determine exact number of bytes for the table on storage:
/// - memory (approximated, resident)
@ -470,7 +470,7 @@ public:
/// Memory part should be estimated as a resident memory size.
/// In particular, alloctedBytes() is preferable over bytes()
/// when considering in-memory blocks.
virtual std::optional<UInt64> totalBytes() const { return {}; }
virtual std::optional<UInt64> totalBytes(const Context &) const { return {}; }
/// Number of rows INSERTed since server start.
///

View File

@ -793,13 +793,13 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S
}
}
std::optional<UInt64> StorageBuffer::totalRows() const
std::optional<UInt64> StorageBuffer::totalRows(const Context & context) const
{
std::optional<UInt64> underlying_rows;
auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id, global_context);
if (underlying)
underlying_rows = underlying->totalRows();
underlying_rows = underlying->totalRows(context);
if (!underlying_rows)
return underlying_rows;
@ -812,7 +812,7 @@ std::optional<UInt64> StorageBuffer::totalRows() const
return rows + *underlying_rows;
}
std::optional<UInt64> StorageBuffer::totalBytes() const
std::optional<UInt64> StorageBuffer::totalBytes(const Context & /*context*/) const
{
UInt64 bytes = 0;
for (const auto & buffer : buffers)

View File

@ -94,8 +94,8 @@ public:
/// The structure of the subordinate table is not checked and does not change.
void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
std::optional<UInt64> totalRows() const override;
std::optional<UInt64> totalBytes() const override;
std::optional<UInt64> totalRows(const Context & context) const override;
std::optional<UInt64> totalBytes(const Context & context) const override;
std::optional<UInt64> lifetimeRows() const override { return writes.rows; }
std::optional<UInt64> lifetimeBytes() const override { return writes.bytes; }

View File

@ -210,7 +210,7 @@ void StorageMemory::truncate(
data.clear();
}
std::optional<UInt64> StorageMemory::totalRows() const
std::optional<UInt64> StorageMemory::totalRows(const Context &) const
{
UInt64 rows = 0;
std::lock_guard lock(mutex);
@ -219,7 +219,7 @@ std::optional<UInt64> StorageMemory::totalRows() const
return rows;
}
std::optional<UInt64> StorageMemory::totalBytes() const
std::optional<UInt64> StorageMemory::totalBytes(const Context &) const
{
UInt64 bytes = 0;
std::lock_guard lock(mutex);

View File

@ -45,8 +45,8 @@ public:
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
std::optional<UInt64> totalRows() const override;
std::optional<UInt64> totalBytes() const override;
std::optional<UInt64> totalRows(const Context &) const override;
std::optional<UInt64> totalBytes(const Context &) const override;
/** Delays initialization of StorageMemory::read() until the first read is actually happen.
* Usually, fore code like this:

View File

@ -184,12 +184,12 @@ Pipe StorageMergeTree::read(
context, max_block_size, num_streams);
}
std::optional<UInt64> StorageMergeTree::totalRows() const
std::optional<UInt64> StorageMergeTree::totalRows(const Context &) const
{
return getTotalActiveSizeInRows();
}
std::optional<UInt64> StorageMergeTree::totalBytes() const
std::optional<UInt64> StorageMergeTree::totalBytes(const Context &) const
{
return getTotalActiveSizeInBytes();
}

View File

@ -46,8 +46,8 @@ public:
size_t max_block_size,
unsigned num_streams) override;
std::optional<UInt64> totalRows() const override;
std::optional<UInt64> totalBytes() const override;
std::optional<UInt64> totalRows(const Context &) const override;
std::optional<UInt64> totalBytes(const Context &) const override;
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override;

View File

@ -46,11 +46,11 @@ public:
void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
std::optional<UInt64> totalRows() const override
std::optional<UInt64> totalRows(const Context &) const override
{
return {0};
}
std::optional<UInt64> totalBytes() const override
std::optional<UInt64> totalBytes(const Context &) const override
{
return {0};
}

View File

@ -430,7 +430,7 @@ protected:
if (columns_mask[src_index++])
{
assert(table != nullptr);
auto total_rows = table->totalRows();
auto total_rows = table->totalRows(context);
if (total_rows)
res_columns[res_index++]->insert(*total_rows);
else
@ -440,7 +440,7 @@ protected:
if (columns_mask[src_index++])
{
assert(table != nullptr);
auto total_bytes = table->totalBytes();
auto total_bytes = table->totalBytes(context);
if (total_bytes)
res_columns[res_index++]->insert(*total_bytes);
else