Reduce lock contention for multiple layers of the Buffer engine

Otherwise you can see something like this for the following query:

    ```sql
    WITH
        arrayMap(x -> demangle(addressToSymbol(x)), s.trace) AS trace_array,
        arrayStringConcat(trace_array, '\n') AS trace_string
    SELECT
        p.thread_id,
        p.query_id,
        p.query,
        trace_string
    FROM
    (
        SELECT
            query_id,
            query,
            arrayJoin(thread_ids) AS thread_id
        FROM system.processes
    ) AS p
    INNER JOIN system.stack_trace AS s ON p.thread_id = s.thread_id
    ORDER BY p.query_id ASC
    SETTINGS enable_global_with_statement = 0, allow_introspection_functions = 1
    FORMAT PrettyCompactNoEscapes
    ```

Lots of the following:

    ```sql
    INSERT INTO buffer (...) VALUES

    __lll_lock_wait
    pthread_mutex_lock
    std::__1::mutex::lock()
    DB::StorageBuffer::reschedule()
    DB::BufferBlockOutputStream::write(DB::Block const&)
    ```

That will wait one of this:

    ```
    INSERT INTO buffer (...) VALUES

    ...
    DB::PushingToViewsBlockOutputStream::write(DB::Block const&)
    DB::AddingDefaultBlockOutputStream::write(DB::Block const&)
    DB::SquashingBlockOutputStream::finalize()
    DB::SquashingBlockOutputStream::writeSuffix()
    DB::PushingToViewsBlockOutputStream::writeSuffix()
    DB::StorageBuffer::writeBlockToDestination(DB::Block const&, std::__1::shared_ptr<DB::IStorage>)
    DB::StorageBuffer::flushBuffer(DB::StorageBuffer::Buffer&, bool, bool, bool)
    ```

P.S. we cannot simply unlock the buffer during flushing, see comments in
the code
This commit is contained in:
Azat Khuzhin 2021-01-21 21:11:39 +03:00
parent c68f7cd5b1
commit b0a80af888

View File

@ -867,10 +867,22 @@ void StorageBuffer::reschedule()
for (auto & buffer : buffers) for (auto & buffer : buffers)
{ {
std::lock_guard lock(buffer.mutex); /// try_to_lock here to avoid waiting for other layers flushing to be finished,
/// since the buffer table may:
/// - push to Distributed table, that may take too much time,
/// - push to table with materialized views attached,
/// this is also may take some time.
///
/// try_to_lock is also ok for background flush, since if there is
/// INSERT contended, then the reschedule will be done after
/// INSERT will be done.
std::unique_lock lock(buffer.mutex, std::try_to_lock);
if (lock.owns_lock())
{
min_first_write_time = buffer.first_write_time; min_first_write_time = buffer.first_write_time;
rows += buffer.data.rows(); rows += buffer.data.rows();
} }
}
/// will be rescheduled via INSERT /// will be rescheduled via INSERT
if (!rows) if (!rows)