Merge pull request #11540 from ClickHouse/fix-direct-io

Fix direct IO with PREWHERE
This commit is contained in:
alexey-milovidov 2020-06-09 23:53:09 +03:00 committed by GitHub
commit 8e05558a5f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 26 additions and 9 deletions

View File

@ -4,6 +4,7 @@
#include <IO/AIOContextPool.h> #include <IO/AIOContextPool.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/MemorySanitizer.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <sys/types.h> #include <sys/types.h>
@ -95,11 +96,8 @@ bool ReadBufferAIO::nextImpl()
if (profile_callback) if (profile_callback)
watch.emplace(clock_type); watch.emplace(clock_type);
if (!is_aio) if (!is_pending_read)
{
synchronousRead(); synchronousRead();
is_aio = true;
}
else else
receive(); receive();
@ -215,7 +213,9 @@ void ReadBufferAIO::synchronousRead()
void ReadBufferAIO::receive() void ReadBufferAIO::receive()
{ {
if (!waitForAIOCompletion()) if (!waitForAIOCompletion())
return; {
throw Exception("Trying to receive data from AIO, but nothing was queued. It's a bug", ErrorCodes::LOGICAL_ERROR);
}
finalize(); finalize();
} }
@ -224,8 +224,6 @@ void ReadBufferAIO::skip()
if (!waitForAIOCompletion()) if (!waitForAIOCompletion())
return; return;
is_aio = false;
/// @todo I presume this assignment is redundant since waitForAIOCompletion() performs a similar one /// @todo I presume this assignment is redundant since waitForAIOCompletion() performs a similar one
// bytes_read = future_bytes_read.get(); // bytes_read = future_bytes_read.get();
if ((bytes_read < 0) || (static_cast<size_t>(bytes_read) < region_left_padding)) if ((bytes_read < 0) || (static_cast<size_t>(bytes_read) < region_left_padding))
@ -274,6 +272,9 @@ void ReadBufferAIO::prepare()
region_aligned_size = region_aligned_end - region_aligned_begin; region_aligned_size = region_aligned_end - region_aligned_begin;
buffer_begin = fill_buffer.internalBuffer().begin(); buffer_begin = fill_buffer.internalBuffer().begin();
/// Unpoison because msan doesn't instrument linux AIO
__msan_unpoison(buffer_begin, fill_buffer.internalBuffer().size());
} }
void ReadBufferAIO::finalize() void ReadBufferAIO::finalize()

View File

@ -100,8 +100,6 @@ private:
bool is_eof = false; bool is_eof = false;
/// At least one read request was sent. /// At least one read request was sent.
bool is_started = false; bool is_started = false;
/// Is the operation asynchronous?
bool is_aio = false;
/// Did the asynchronous operation fail? /// Did the asynchronous operation fail?
bool aio_failed = false; bool aio_failed = false;

View File

@ -0,0 +1 @@
Loaded 1 queries.

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --multiquery --query "
DROP TABLE IF EXISTS bug;
CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date;
INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000);
OPTIMIZE TABLE bug FINAL;"
$CLICKHOUSE_BENCHMARK --database $CLICKHOUSE_DATABASE --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>$CLICKHOUSE_TMP/err
cat $CLICKHOUSE_TMP/err | grep Exception
cat $CLICKHOUSE_TMP/err | grep Loaded
$CLICKHOUSE_CLIENT --multiquery --query "
DROP TABLE bug;"