Lock-less totalRows/totalBytes + more clear postponed init

This commit is contained in:
Alexander Kazakov 2020-10-06 16:45:17 +03:00
parent 9594e463b4
commit e54ff6e60a
2 changed files with 73 additions and 67 deletions

View File

@ -21,81 +21,64 @@ namespace ErrorCodes
class MemorySource : public SourceWithProgress
{
using InitializerFunc = std::function<void(BlocksList::iterator &, size_t &)>;
public:
/// We use range [first, last] which includes right border.
/// Blocks are stored in std::list which may be appended in another thread.
/// We don't use synchronisation here, because elements in range [first, last] won't be modified.
/// We use pointer to the beginning of the list and its current size.
/// We don't need synchronisation in this reader, because while we hold SharedLock on storage,
/// only new elements can be added to the back of the list, so our iterators remain valid
MemorySource(
Names column_names_,
BlocksList::iterator first_,
const Names & column_names_,
BlocksList::const_iterator first_,
size_t num_blocks_,
const StorageMemory & storage,
const StorageMetadataPtr & metadata_snapshot)
const StorageMetadataPtr & metadata_snapshot,
InitializerFunc initializer_func_ = [](size_t &, size_t &) {})
: SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID()))
, column_names(std::move(column_names_))
, column_names(column_names_)
, current_it(first_)
, num_blocks(num_blocks_)
, initializer_func(std::move(initializer_func_))
{
}
/// If called, will initialize the number of blocks at first read.
/// It allows to read data which was inserted into memory table AFTER Storage::read was called.
/// This hack is needed for global subqueries.
void delayInitialization(BlocksList * data_, std::mutex * mutex_)
{
data = data_;
mutex = mutex_;
}
String getName() const override { return "Memory"; }
protected:
Chunk generate() override
{
if (data)
if (!postponed_init_done)
{
std::lock_guard guard(*mutex);
current_it = data->begin();
num_blocks = data->size();
is_finished = num_blocks == 0;
data = nullptr;
mutex = nullptr;
initializer_func(current_it, num_blocks);
postponed_init_done = true;
}
if (is_finished)
{
if (current_block_idx == num_blocks)
return {};
}
else
{
const Block & src = *current_it;
Columns columns;
columns.reserve(column_names.size());
/// Add only required columns to `res`.
for (const auto & name : column_names)
columns.emplace_back(src.getByName(name).column);
const Block & src = *current_it;
Columns columns;
columns.reserve(column_names.size());
++current_block_idx;
/// Add only required columns to `res`.
for (const auto & name : column_names)
columns.push_back(src.getByName(name).column);
if (current_block_idx == num_blocks)
is_finished = true;
else
++current_it;
if (++current_block_idx < num_blocks)
++current_it;
return Chunk(std::move(columns), src.rows());
}
return Chunk(std::move(columns), src.rows());
}
private:
Names column_names;
BlocksList::iterator current_it;
size_t current_block_idx = 0;
size_t num_blocks;
bool is_finished = false;
BlocksList * data = nullptr;
std::mutex * mutex = nullptr;
private:
const Names column_names;
BlocksList::const_iterator current_it;
size_t num_blocks;
size_t current_block_idx = 0;
bool postponed_init_done = false;
InitializerFunc initializer_func;
};
@ -113,9 +96,18 @@ public:
void write(const Block & block) override
{
const auto size_bytes_diff = block.allocatedBytes();
const auto size_rows_diff = block.rows();
metadata_snapshot->check(block, true);
std::lock_guard lock(storage.mutex);
storage.data.push_back(block);
{
std::lock_guard lock(storage.mutex);
storage.data.push_back(block);
storage.total_size_bytes.fetch_add(size_bytes_diff, std::memory_order_relaxed);
storage.total_size_rows.fetch_add(size_rows_diff, std::memory_order_relaxed);
}
}
private:
StorageMemory & storage;
@ -144,8 +136,6 @@ Pipe StorageMemory::read(
{
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
std::lock_guard lock(mutex);
if (delay_read_for_global_subqueries)
{
/// Note: for global subquery we use single source.
@ -156,11 +146,21 @@ Pipe StorageMemory::read(
/// set for IN or hash table for JOIN, which can't be done concurrently.
/// Since no other manipulation with data is done, multiple sources shouldn't give any profit.
auto source = std::make_shared<MemorySource>(column_names, data.begin(), data.size(), *this, metadata_snapshot);
source->delayInitialization(&data, &mutex);
return Pipe(std::move(source));
return {std::make_shared<MemorySource>(
column_names, data.end(), 0, *this, metadata_snapshot,
/// This hack is needed for global subqueries.
/// It allows to set up this Source for read AFTER Storage::read() has been called and just before actual reading
[this](size_t & current_it, size_t & num_blocks)
{
std::lock_guard guard(mutex);
current_it = data.begin();
num_blocks = data.size();
}
)};
}
std::lock_guard lock(mutex);
size_t size = data.size();
if (num_streams > size)
@ -178,7 +178,7 @@ Pipe StorageMemory::read(
assert(num_blocks > 0);
pipes.emplace_back(std::make_shared<MemorySource>(column_names, it, num_blocks, *this, metadata_snapshot));
pipes.push_back(std::make_shared<MemorySource>(column_names, it, num_blocks, *this, metadata_snapshot));
while (offset < next_offset)
{
@ -201,31 +201,32 @@ void StorageMemory::drop()
{
std::lock_guard lock(mutex);
data.clear();
total_size_bytes.store(0, std::memory_order_relaxed);
total_size_rows.store(0, std::memory_order_relaxed);
}
void StorageMemory::truncate(
const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &)
{
std::lock_guard lock(mutex);
data.clear();
total_size_bytes.store(0, std::memory_order_relaxed);
total_size_rows.store(0, std::memory_order_relaxed);
}
std::optional<UInt64> StorageMemory::totalRows() const
{
UInt64 rows = 0;
std::lock_guard lock(mutex);
for (const auto & buffer : data)
rows += buffer.rows();
return rows;
/// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency
/// When run concurrently we are fine with any value: "before" or "after"
return total_size_rows.load(std::memory_order_relaxed);
}
std::optional<UInt64> StorageMemory::totalBytes() const
{
UInt64 bytes = 0;
std::lock_guard lock(mutex);
for (const auto & buffer : data)
bytes += buffer.allocatedBytes();
return bytes;
return total_size_bytes.load(std::memory_order_relaxed);
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <atomic>
#include <optional>
#include <mutex>
#include <ext/shared_ptr_helper.h>
@ -93,6 +95,9 @@ private:
bool delay_read_for_global_subqueries = false;
std::atomic<size_t> total_size_bytes = 0;
std::atomic<size_t> total_size_rows = 0;
protected:
StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_);
};