ClickHouse/src/IO/ReadBufferFromFileDescriptor.cpp

282 lines
8.8 KiB
C++
Raw Normal View History

2016-10-25 06:49:24 +00:00
#include <errno.h>
#include <time.h>
#include <optional>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Common/CurrentMetrics.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
2021-08-24 21:45:58 +00:00
#include <IO/Progress.h>
#include <sys/stat.h>
2016-10-25 06:49:24 +00:00
#ifdef HAS_RESERVED_IDENTIFIER
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
2016-10-25 06:49:24 +00:00
namespace ProfileEvents
{
extern const Event ReadBufferFromFileDescriptorRead;
extern const Event ReadBufferFromFileDescriptorReadFailed;
extern const Event ReadBufferFromFileDescriptorReadBytes;
extern const Event DiskReadElapsedMicroseconds;
extern const Event Seek;
2016-10-25 06:49:24 +00:00
}
namespace CurrentMetrics
{
extern const Metric Read;
2016-10-25 06:49:24 +00:00
}
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int CANNOT_SELECT;
extern const int CANNOT_FSTAT;
2021-07-26 00:34:36 +00:00
extern const int CANNOT_ADVISE;
2016-10-25 06:49:24 +00:00
}
std::string ReadBufferFromFileDescriptor::getFileName() const
{
return "(fd = " + toString(fd) + ")";
2016-10-25 06:49:24 +00:00
}
bool ReadBufferFromFileDescriptor::nextImpl()
{
/// If internal_buffer size is empty, then read() cannot be distinguished from EOF
assert(!internal_buffer.empty());
Do not try to read pass EOF (to workaround a bug in a kernel) For unaligned offset pread() may return EINVAL even if the offset pass EOF, although it should not, since otherwise there is no abiliity to rely on read() == 0 is EOF (with pread() loop). Here is a reproducer for the problem on 4.9.0-12-amd64: $ head -c27 /dev/urandom > /tmp/pread.issue $ xfs_io xfs_io> open -d /tmp/pread.issue xfs_io> pread 1000 4096 pread: Invalid argument And this is how it should work: xfs_io> pread 29 4096 read 0/4096 bytes at offset 29 Note, here I use interactive mode since we had old xfs_io that does not allow to execute multiple commands at once, and to avoid EMFILE issue Here is some history of a patches that affects this behaviour in the linux kernel: - the issue had been introduced in torvalds/linux@9fe55eea7e4b444bafc42fa0000cc2d1d2847275 v3.14 ("Fix race when checking i_size on direct i/o read") - an attempt to fix it had been made in torvalds/linux@74cedf9b6c603f2278a05bc91b140b32b434d0b5 v4.4 ("direct-io: Fix negative return from dio read beyond eof") - but this wasn't enough, since alignment check was earlier, so eventually fixed in torvalds/linux@41b21af388f94baf7433d4e7845703c7275251de v5.10 ("direct-io: defer alignment check until after the EOF check") Someone may ask why CI does not shows the issue, since: - it had 4.19 kernel when CI was in yandex - now it has 5.4 when CI is in AWS Since both of those kernels does not have the last patch. But, this bug requires the following conditions to met: - index_granularity_bytes=0 - min_merge_bytes_to_use_direct_io=1 Which was not covered by CI yet.
2022-01-03 16:21:50 +00:00
/// This is a workaround of a read pass EOF bug in linux kernel with pread()
if (file_size.has_value() && file_offset_of_buffer_end >= *file_size)
return false;
size_t bytes_read = 0;
while (!bytes_read)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead);
Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC);
ssize_t res = 0;
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::Read};
if (use_pread)
res = ::pread(fd, internal_buffer.begin(), internal_buffer.size(), file_offset_of_buffer_end);
else
res = ::read(fd, internal_buffer.begin(), internal_buffer.size());
}
if (!res)
break;
if (-1 == res && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
2019-08-07 12:52:47 +00:00
throwFromErrnoWithPath("Cannot read from file " + getFileName(), getFileName(),
ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
}
if (res > 0)
bytes_read += res;
2018-08-22 00:24:55 +00:00
/// It reports real time spent including the time spent while thread was preempted doing nothing.
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
2020-07-13 00:45:37 +00:00
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
/// (TaskStatsInfoGetter has about 500K RPS).
watch.stop();
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
if (profile_callback)
{
ProfileInfo info;
info.bytes_requested = internal_buffer.size();
info.bytes_read = res;
info.nanoseconds = watch.elapsed();
profile_callback(info);
}
}
2020-07-31 14:53:41 +00:00
file_offset_of_buffer_end += bytes_read;
if (bytes_read)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read);
2021-02-06 16:30:46 +00:00
working_buffer = internal_buffer;
working_buffer.resize(bytes_read);
buffer_is_dirty = false;
}
else
return false;
return true;
2016-10-25 06:49:24 +00:00
}
2021-07-26 00:34:36 +00:00
void ReadBufferFromFileDescriptor::prefetch()
{
2021-07-27 23:59:56 +00:00
#if defined(POSIX_FADV_WILLNEED)
2021-07-26 00:34:36 +00:00
/// For direct IO, loading data into page cache is pointless.
if (required_alignment)
return;
/// Ask OS to prefetch data into page cache.
if (0 != posix_fadvise(fd, file_offset_of_buffer_end, internal_buffer.size(), POSIX_FADV_WILLNEED))
throwFromErrno("Cannot posix_fadvise", ErrorCodes::CANNOT_ADVISE);
2021-07-27 23:59:56 +00:00
#endif
2021-07-26 00:34:36 +00:00
}
2016-10-25 06:49:24 +00:00
/// If 'offset' is small enough to stay in buffer after seek, then true seek in file does not happen.
off_t ReadBufferFromFileDescriptor::seek(off_t offset, int whence)
2016-10-25 06:49:24 +00:00
{
2020-07-31 14:53:41 +00:00
size_t new_pos;
if (whence == SEEK_SET)
2020-07-31 14:53:41 +00:00
{
assert(offset >= 0);
new_pos = offset;
2020-07-31 14:53:41 +00:00
}
else if (whence == SEEK_CUR)
2020-07-31 14:53:41 +00:00
{
new_pos = file_offset_of_buffer_end - (working_buffer.end() - pos) + offset;
}
else
2020-07-31 14:53:41 +00:00
{
throw Exception("ReadBufferFromFileDescriptor::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
2020-07-31 14:53:41 +00:00
}
/// Position is unchanged.
if (!buffer_is_dirty && (new_pos + (working_buffer.end() - pos) == file_offset_of_buffer_end))
return new_pos;
if (!buffer_is_dirty && file_offset_of_buffer_end - working_buffer.size() <= static_cast<size_t>(new_pos)
2021-08-04 00:07:04 +00:00
&& new_pos <= file_offset_of_buffer_end)
{
/// Position is still inside the buffer.
2021-08-04 00:07:04 +00:00
/// Probably it is at the end of the buffer - then we will load data on the following 'next' call.
2020-07-31 14:53:41 +00:00
pos = working_buffer.end() - file_offset_of_buffer_end + new_pos;
assert(pos >= working_buffer.begin());
2021-08-04 00:07:04 +00:00
assert(pos <= working_buffer.end());
2020-07-31 14:53:41 +00:00
return new_pos;
}
else
{
/// Position is out of the buffer, we need to do real seek.
off_t seek_pos = required_alignment > 1
2021-07-11 00:35:43 +00:00
? new_pos / required_alignment * required_alignment
: new_pos;
off_t offset_after_seek_pos = new_pos - seek_pos;
/// First put position at the end of the buffer so the next read will fetch new data to the buffer.
pos = working_buffer.end();
/// Mark buffer as dirty to disallow further seek optimizations, because fetching data to the buffer
/// is delayed to the next call of 'nextImpl', but it may be not called before next seek.
buffer_is_dirty = true;
/// In case of using 'pread' we just update the info about the next position in file.
/// In case of using 'read' we call 'lseek'.
2021-07-11 00:35:43 +00:00
/// We account both cases as seek event as it leads to non-contiguous reads from file.
ProfileEvents::increment(ProfileEvents::Seek);
if (!use_pread)
{
Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC);
off_t res = ::lseek(fd, seek_pos, SEEK_SET);
if (-1 == res)
throwFromErrnoWithPath("Cannot seek through file " + getFileName(), getFileName(),
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
/// Also note that seeking past the file size is not allowed.
if (res != seek_pos)
throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE,
"The 'lseek' syscall returned value ({}) that is not expected ({})", res, seek_pos);
watch.stop();
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
}
2021-07-11 02:43:54 +00:00
file_offset_of_buffer_end = seek_pos;
2021-07-11 00:35:43 +00:00
if (offset_after_seek_pos > 0)
ignore(offset_after_seek_pos);
return seek_pos;
}
2016-10-25 06:49:24 +00:00
}
2021-07-04 03:03:49 +00:00
void ReadBufferFromFileDescriptor::rewind()
{
if (!use_pread)
{
ProfileEvents::increment(ProfileEvents::Seek);
off_t res = ::lseek(fd, 0, SEEK_SET);
if (-1 == res)
throwFromErrnoWithPath("Cannot seek through file " + getFileName(), getFileName(),
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
}
/// In case of pread, the ProfileEvents::Seek is not accounted, but it's Ok.
2021-07-04 03:03:49 +00:00
/// Clearing the buffer with existing data. New data will be read on subsequent call to 'next'.
working_buffer.resize(0);
pos = working_buffer.begin();
2021-07-12 00:24:38 +00:00
file_offset_of_buffer_end = 0;
2022-02-05 23:57:49 +00:00
buffer_is_dirty = true;
2021-07-04 03:03:49 +00:00
}
2016-10-25 06:49:24 +00:00
/// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout.
bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds)
{
fd_set fds;
FD_ZERO(&fds);
FD_SET(fd, &fds);
timeval timeout = { time_t(timeout_microseconds / 1000000), suseconds_t(timeout_microseconds % 1000000) };
2016-10-25 06:49:24 +00:00
int res = select(1, &fds, nullptr, nullptr, &timeout);
2016-10-25 06:49:24 +00:00
if (-1 == res)
throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT);
2016-10-25 06:49:24 +00:00
return res > 0;
2016-10-25 06:49:24 +00:00
}
off_t ReadBufferFromFileDescriptor::size()
{
struct stat buf;
int res = fstat(fd, &buf);
if (-1 == res)
throwFromErrnoWithPath("Cannot execute fstat " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSTAT);
return buf.st_size;
}
void ReadBufferFromFileDescriptor::setProgressCallback(ContextPtr context)
{
auto file_progress_callback = context->getFileProgressCallback();
if (!file_progress_callback)
return;
setProfileCallback([file_progress_callback](const ProfileInfo & progress)
{
file_progress_callback(FileProgress(progress.bytes_read, 0));
});
}
2016-10-25 06:49:24 +00:00
}