Skip to content

Commit

Permalink
Merge pull request #39179 from azat/fsync-profile-events
Browse files Browse the repository at this point in the history
Add profile events for fsync
  • Loading branch information
alexey-milovidov committed Aug 2, 2022
2 parents b33fe26 + 3e627e2 commit 3165288
Show file tree
Hide file tree
Showing 6 changed files with 105 additions and 0 deletions.
4 changes: 4 additions & 0 deletions src/Common/ProfileEvents.cpp
Expand Up @@ -25,6 +25,10 @@
M(WriteBufferFromFileDescriptorWrite, "Number of writes (write/pwrite) to a file descriptor. Does not include sockets.") \
M(WriteBufferFromFileDescriptorWriteFailed, "Number of times the write (write/pwrite) to a file descriptor have failed.") \
M(WriteBufferFromFileDescriptorWriteBytes, "Number of bytes written to file descriptors. If the file is compressed, this will show compressed data size.") \
M(FileSync, "Number of times the F_FULLFSYNC/fsync/fdatasync function was called for files.") \
M(DirectorySync, "Number of times the F_FULLFSYNC/fsync/fdatasync function was called for directories.") \
M(FileSyncElapsedMicroseconds, "Total time spent waiting for F_FULLFSYNC/fsync/fdatasync syscall for files.") \
M(DirectorySyncElapsedMicroseconds, "Total time spent waiting for F_FULLFSYNC/fsync/fdatasync syscall for directories.") \
M(ReadCompressedBytes, "Number of bytes (the number of bytes before decompression) read from compressed sources (files, network).") \
M(CompressedReadBufferBlocks, "Number of compressed blocks (the blocks of data that are compressed independent of each other) read from compressed sources (files, network).") \
M(CompressedReadBufferBytes, "Number of uncompressed bytes (the number of bytes after decompression) read from compressed sources (files, network).") \
Expand Down
6 changes: 6 additions & 0 deletions src/Dictionaries/SSDCacheDictionaryStorage.h
Expand Up @@ -34,6 +34,8 @@ namespace ProfileEvents
extern const Event AIOWriteBytes;
extern const Event AIORead;
extern const Event AIOReadBytes;
extern const Event FileSync;
extern const Event FileSyncElapsedMicroseconds;
}

namespace DB
Expand Down Expand Up @@ -544,13 +546,17 @@ class SSDCacheFileBuffer : private boost::noncopyable
file_path,
std::to_string(bytes_written));

ProfileEvents::increment(ProfileEvents::FileSync);

Stopwatch watch;
#if defined(OS_DARWIN)
if (::fsync(file.fd) < 0)
throwFromErrnoWithPath("Cannot fsync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC);
#else
if (::fdatasync(file.fd) < 0)
throwFromErrnoWithPath("Cannot fdatasync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC);
#endif
ProfileEvents::increment(ProfileEvents::FileSyncElapsedMicroseconds, watch.elapsedMicroseconds());

current_block_index += buffer_size_in_blocks;

Expand Down
14 changes: 14 additions & 0 deletions src/Disks/LocalDirectorySyncGuard.cpp
@@ -1,13 +1,21 @@
#include <Disks/LocalDirectorySyncGuard.h>
#include <Common/ProfileEvents.h>
#include <Common/Exception.h>
#include <Disks/IDisk.h>
#include <Common/Stopwatch.h>
#include <fcntl.h> // O_RDWR

/// OSX does not have O_DIRECTORY
#ifndef O_DIRECTORY
#define O_DIRECTORY O_RDWR
#endif

namespace ProfileEvents
{
extern const Event DirectorySync;
extern const Event DirectorySyncElapsedMicroseconds;
}

namespace DB
{

Expand All @@ -29,8 +37,12 @@ LocalDirectorySyncGuard::LocalDirectorySyncGuard(const String & full_path)

LocalDirectorySyncGuard::~LocalDirectorySyncGuard()
{
ProfileEvents::increment(ProfileEvents::DirectorySync);

try
{
Stopwatch watch;

#if defined(OS_DARWIN)
if (fcntl(fd, F_FULLFSYNC, 0))
throwFromErrno("Cannot fcntl(F_FULLFSYNC)", ErrorCodes::CANNOT_FSYNC);
Expand All @@ -40,6 +52,8 @@ LocalDirectorySyncGuard::~LocalDirectorySyncGuard()
#endif
if (-1 == ::close(fd))
throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE);

ProfileEvents::increment(ProfileEvents::DirectorySyncElapsedMicroseconds, watch.elapsedMicroseconds());
}
catch (...)
{
Expand Down
8 changes: 8 additions & 0 deletions src/IO/WriteBufferFromFileDescriptor.cpp
Expand Up @@ -18,6 +18,8 @@ namespace ProfileEvents
extern const Event WriteBufferFromFileDescriptorWriteFailed;
extern const Event WriteBufferFromFileDescriptorWriteBytes;
extern const Event DiskWriteElapsedMicroseconds;
extern const Event FileSync;
extern const Event FileSyncElapsedMicroseconds;
}

namespace CurrentMetrics
Expand Down Expand Up @@ -113,12 +115,18 @@ void WriteBufferFromFileDescriptor::sync()
/// If buffer has pending data - write it.
next();

ProfileEvents::increment(ProfileEvents::FileSync);

Stopwatch watch;

/// Request OS to sync data with storage medium.
#if defined(OS_DARWIN)
int res = ::fsync(fd);
#else
int res = ::fdatasync(fd);
#endif
ProfileEvents::increment(ProfileEvents::FileSyncElapsedMicroseconds, watch.elapsedMicroseconds());

if (-1 == res)
throwFromErrnoWithPath("Cannot fsync " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSYNC);
}
Expand Down
Empty file.
73 changes: 73 additions & 0 deletions tests/queries/0_stateless/02361_fsync_profile_events.sh
@@ -0,0 +1,73 @@
#!/usr/bin/env bash
# Tags: no-s3-storage
# Tag no-s3-storage: s3 does not have fsync

CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh

$CLICKHOUSE_CLIENT -nm -q "
drop table if exists data_fsync_pe;
create table data_fsync_pe (key Int) engine=MergeTree()
order by key
settings
min_rows_for_wide_part=2,
fsync_after_insert=1,
fsync_part_directory=1;
"

ret=1
# Retry in case of fsync/fdatasync was too fast
# (FileSyncElapsedMicroseconds/DirectorySyncElapsedMicroseconds was 0)
for i in {1..100}; do
query_id="insert-$i-$CLICKHOUSE_DATABASE"

$CLICKHOUSE_CLIENT --query_id "$query_id" -q "insert into data_fsync_pe values (1)"

read -r FileSync FileOpen DirectorySync FileSyncElapsedMicroseconds DirectorySyncElapsedMicroseconds <<<"$(
$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "
system flush logs;

select
ProfileEvents['FileSync'],
ProfileEvents['FileOpen'],
ProfileEvents['DirectorySync'],
ProfileEvents['FileSyncElapsedMicroseconds']>0,
ProfileEvents['DirectorySyncElapsedMicroseconds']>0
from system.query_log
where
event_date >= yesterday() and
current_database = currentDatabase() and
query_id = {query_id:String} and
type = 'QueryFinish';
")"

# Non retriable errors
if [[ $FileSync -ne 7 ]]; then
exit 2
fi
# Check that all files was synced
if [[ $FileSync -ne $FileOpen ]]; then
exit 3
fi
if [[ $DirectorySync -ne 2 ]]; then
exit 4
fi

# Retriable errors
if [[ $FileSyncElapsedMicroseconds -eq 0 ]]; then
continue
fi
if [[ $DirectorySyncElapsedMicroseconds -eq 0 ]]; then
continue
fi

# Everything is OK
ret=0
break
done

$CLICKHOUSE_CLIENT -q "drop table data_fsync_pe"

exit $ret

0 comments on commit 3165288

Please sign in to comment.