mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge pull request #11540 from ClickHouse/fix-direct-io
Fix direct IO with PREWHERE
This commit is contained in:
commit
8e05558a5f
@ -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()
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
1
tests/queries/0_stateless/01304_direct_io.reference
Normal file
1
tests/queries/0_stateless/01304_direct_io.reference
Normal file
@ -0,0 +1 @@
|
|||||||
|
Loaded 1 queries.
|
17
tests/queries/0_stateless/01304_direct_io.sh
Executable file
17
tests/queries/0_stateless/01304_direct_io.sh
Executable 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;"
|
Loading…
Reference in New Issue
Block a user