Implement totalBytes() for MergeTree/ReplicatedMergeTree

This commit is contained in:
Azat Khuzhin 2020-03-29 11:50:27 +03:00
parent 4a7ee947b2
commit 08d5872d90
4 changed files with 26 additions and 3 deletions

View File

@ -151,6 +151,11 @@ std::optional<UInt64> StorageMergeTree::totalRows() const
return getTotalActiveSizeInRows();
}
std::optional<UInt64> StorageMergeTree::totalBytes() const
{
return getTotalActiveSizeInBytes();
}
BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Context & context)
{
return std::make_shared<MergeTreeBlockOutputStream>(*this, context.getSettingsRef().max_partitions_per_insert_block);

View File

@ -46,6 +46,7 @@ public:
unsigned num_streams) override;
std::optional<UInt64> totalRows() const override;
std::optional<UInt64> totalBytes() const override;
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;

View File

@ -2995,9 +2995,9 @@ Pipes StorageReplicatedMergeTree::read(
}
std::optional<UInt64> StorageReplicatedMergeTree::totalRows() const
template <class Func>
void StorageReplicatedMergeTree::foreachCommittedParts(const Func & func) const
{
size_t res = 0;
auto max_added_blocks = getMaxAddedBlocks();
auto lock = lockParts();
for (auto & part : getDataPartsStateRange(DataPartState::Committed))
@ -3009,8 +3009,21 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalRows() const
if (blocks_iterator == max_added_blocks.end() || part->info.max_block > blocks_iterator->second)
continue;
res += part->rows_count;
func(part);
}
}
std::optional<UInt64> StorageReplicatedMergeTree::totalRows() const
{
UInt64 res = 0;
foreachCommittedParts([&res](auto & part) { res += part->rows_count; });
return res;
}
std::optional<UInt64> StorageReplicatedMergeTree::totalBytes() const
{
UInt64 res = 0;
foreachCommittedParts([&res](auto & part) { res += part->getBytesOnDisk(); });
return res;
}

View File

@ -96,6 +96,7 @@ public:
unsigned num_streams) override;
std::optional<UInt64> totalRows() const override;
std::optional<UInt64> totalBytes() const override;
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
@ -287,6 +288,9 @@ private:
/// True if replica was created for existing table with fixed granularity
bool other_replicas_fixed_granularity = false;
template <class Func>
void foreachCommittedParts(const Func & func) const;
/** Creates the minimum set of nodes in ZooKeeper.
*/
void createTableIfNotExists();