ClickHouse/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp
Azat Khuzhin 604cec475a Fix possible distributed sends stuck due to "No such file or directory"
In case of restoring from current_batch.txt it is possible that the some
file from the batch will not be exist, and the fix submitted in #49884
was not complete, since it will fail later in markAsSend() (due to it
tries to obtain file size there):

    2023.12.04 05:43:12.676658 [ 5006 ] {} <Error> dist.DirectoryMonitor.work4: std::exception. Code: 1001, type: std::__1::__fs::filesystem::filesystem_error, e.what() = filesystem error: in file_size: No such file or directory ["/work4/clickhouse/data/dist/shard8_all_replicas//150426396.bin"], Stack trace (when copying this message, always include the lines below):

    0. ./.build/./contrib/llvm-project/libcxx/include/exception:134: std::runtime_error::runtime_error(String const&) @ 0x00000000177e83f4 in /usr/lib/debug/usr/bin/clickhouse.debug
    1. ./.build/./contrib/llvm-project/libcxx/include/string:1499: std::system_error::system_error(std::error_code, String const&) @ 0x00000000177f0fd5 in /usr/lib/debug/usr/bin/clickhouse.debug
    2. ./.build/./contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:42: std::__fs::filesystem::filesystem_error::filesystem_error[abi:v15000](String const&, std::__fs::filesystem::path const&, std::error_code) @ 0x000000000b844ca1 in /usr/lib/debug/usr/bin/clickhouse.debug
    3. ./.build/./contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:90: void std::__fs::filesystem::__throw_filesystem_error[abi:v15000]<String&, std::__fs::filesystem::path const&, std::error_code const&>(String&, std::__fs::filesystem::path const&, std::error_code const&) @ 0x000000001778f953 in /usr/lib/debug/usr/bin/clickhouse.debug
    4. ./.build/./contrib/llvm-project/libcxx/src/filesystem/filesystem_common.h:0: std::__fs::filesystem::detail::(anonymous namespace)::ErrorHandler<unsigned long>::report(std::error_code const&) const @ 0x0000000017793ef7 in /usr/lib/debug/usr/bin/clickhouse.debug
    5. ./.build/./contrib/llvm-project/libcxx/src/filesystem/operations.cpp:0: std::__fs::filesystem::__file_size(std::__fs::filesystem::path const&, std::error_code*) @ 0x0000000017793e26 in /usr/lib/debug/usr/bin/clickhouse.debug
    6. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:707: DB::DistributedAsyncInsertDirectoryQueue::markAsSend(String const&) @ 0x0000000011cd92c5 in /usr/lib/debug/usr/bin/clickhouse.debug
    7. ./.build/./contrib/llvm-project/libcxx/include/__iterator/wrap_iter.h💯 DB::DistributedAsyncInsertBatch::send() @ 0x0000000011cdd81c in /usr/lib/debug/usr/bin/clickhouse.debug
    8. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:0: DB::DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() @ 0x0000000011cd5054 in /usr/lib/debug/usr/bin/clickhouse.debug
    9. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:417: DB::DistributedAsyncInsertDirectoryQueue::processFiles() @ 0x0000000011cd3440 in /usr/lib/debug/usr/bin/clickhouse.debug
    10. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:0: DB::DistributedAsyncInsertDirectoryQueue::run() @ 0x0000000011cd3878 in /usr/lib/debug/usr/bin/clickhouse.debug
    11. ./.build/./contrib/llvm-project/libcxx/include/__functional/function.h:0: DB::BackgroundSchedulePoolTaskInfo::execute() @ 0x00000000103dbc34 in /usr/lib/debug/usr/bin/clickhouse.debug
    12. ./.build/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::BackgroundSchedulePool::threadFunction() @ 0x00000000103de1b6 in /usr/lib/debug/usr/bin/clickhouse.debug
    13. ./.build/./src/Core/BackgroundSchedulePool.cpp:0: void std::__function::__policy_invoker<void ()>::__call_impl<std::__function::__default_alloc_func<ThreadFromGlobalPoolImpl<false>::ThreadFromGlobalPoolImpl<DB::BackgroundSchedulePool::BackgroundSchedulePool(unsigned long, StrongTypedef<unsigned long, CurrentMetrics::MetricTag>, StrongTypedef<unsigned long, CurrentMetrics::MetricTag>, char const*)::$_0>(DB::BackgroundSchedulePool::BackgroundSchedulePool(unsigned long, StrongTypedef<unsigned long, CurrentMetrics::MetricTag>, StrongTypedef<unsigned long, CurrentMetrics::MetricTag>, char const*)::$_0&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x00000000103de7d1 in /usr/lib/debug/usr/bin/clickhouse.debug
    14. ./.build/./base/base/../base/wide_integer_impl.h:809: ThreadPoolImpl<std::thread>::worker(std::__list_iterator<std::thread, void*>) @ 0x000000000b8c5502 in /usr/lib/debug/usr/bin/clickhouse.debug
    15. ./.build/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:302: void* std::__thread_proxy[abi:v15000]<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>>(void*) @ 0x000000000b8c936e in /usr/lib/debug/usr/bin/clickhouse.debug
    16. ? @ 0x00007f1be8b30fd4 in ?
    17. ? @ 0x00007f1be8bb15bc in ?

And instead of ignoring errors, DistributedAsyncInsertBatch::valid() had
been added, that should be called when the files had been read from the
current_batch.txt, if it is not valid (some files from the batch did not
exist), then there is no sense in trying to send the same batch, so just
this file will be ignored, and files will be processed in a regular
order.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-12-04 16:52:53 +01:00

315 lines
10 KiB
C++

#include <Storages/Distributed/DistributedAsyncInsertBatch.h>
#include <Storages/Distributed/DistributedAsyncInsertHelpers.h>
#include <Storages/Distributed/DistributedAsyncInsertHeader.h>
#include <Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h>
#include <Storages/StorageDistributed.h>
#include <QueryPipeline/RemoteInserter.h>
#include <Common/CurrentMetrics.h>
#include <base/defines.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromFile.h>
namespace CurrentMetrics
{
extern const Metric DistributedSend;
}
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int DISTRIBUTED_BROKEN_BATCH_INFO;
extern const int DISTRIBUTED_BROKEN_BATCH_FILES;
extern const int TOO_MANY_PARTS;
extern const int TOO_MANY_BYTES;
extern const int TOO_MANY_ROWS_OR_BYTES;
extern const int TOO_MANY_PARTITIONS;
extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES;
extern const int ARGUMENT_OUT_OF_BOUND;
}
/// Can the batch be split and send files from batch one-by-one instead?
bool isSplittableErrorCode(int code, bool remote)
{
return code == ErrorCodes::MEMORY_LIMIT_EXCEEDED
/// FunctionRange::max_elements and similar
|| code == ErrorCodes::ARGUMENT_OUT_OF_BOUND
|| code == ErrorCodes::TOO_MANY_PARTS
|| code == ErrorCodes::TOO_MANY_BYTES
|| code == ErrorCodes::TOO_MANY_ROWS_OR_BYTES
|| code == ErrorCodes::TOO_MANY_PARTITIONS
|| code == ErrorCodes::DISTRIBUTED_TOO_MANY_PENDING_BYTES
|| code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO
|| isDistributedSendBroken(code, remote)
;
}
DistributedAsyncInsertBatch::DistributedAsyncInsertBatch(DistributedAsyncInsertDirectoryQueue & parent_)
: parent(parent_)
, split_batch_on_failure(parent.split_batch_on_failure)
, fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert)
, dir_fsync(parent.dir_fsync)
{}
bool DistributedAsyncInsertBatch::isEnoughSize() const
{
return (!parent.min_batched_block_size_rows && !parent.min_batched_block_size_bytes)
|| (parent.min_batched_block_size_rows && total_rows >= parent.min_batched_block_size_rows)
|| (parent.min_batched_block_size_bytes && total_bytes >= parent.min_batched_block_size_bytes);
}
void DistributedAsyncInsertBatch::send()
{
if (files.empty())
return;
CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend};
Stopwatch watch;
if (!recovered)
{
/// For deduplication in Replicated tables to work, in case of error
/// we must try to re-send exactly the same batches.
/// So we save contents of the current batch into the current_batch_file_path file
/// and truncate it afterwards if all went well.
serialize();
}
bool batch_broken = false;
bool batch_marked_as_broken = false;
try
{
try
{
sendBatch();
}
catch (const Exception & e)
{
if (split_batch_on_failure && files.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException()))
{
tryLogCurrentException(parent.log, "Trying to split batch due to");
sendSeparateFiles();
}
else
throw;
}
}
catch (Exception & e)
{
if (isDistributedSendBroken(e.code(), e.isRemoteException()))
{
tryLogCurrentException(parent.log, "Failed to send batch due to");
batch_broken = true;
if (!e.isRemoteException() && e.code() == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES)
batch_marked_as_broken = true;
}
else
{
e.addMessage(fmt::format("While sending a batch of {} files, files: {}", files.size(), fmt::join(files, "\n")));
throw;
}
}
if (!batch_broken)
{
LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", files.size(), watch.elapsedMilliseconds());
auto dir_sync_guard = parent.getDirectorySyncGuard(parent.relative_path);
for (const auto & file : files)
parent.markAsSend(file);
}
else if (!batch_marked_as_broken)
{
LOG_ERROR(parent.log, "Marking a batch of {} files as broken, files: {}", files.size(), fmt::join(files, "\n"));
for (const auto & file : files)
parent.markAsBroken(file);
}
files.clear();
total_rows = 0;
total_bytes = 0;
recovered = false;
std::filesystem::resize_file(parent.current_batch_file_path, 0);
}
void DistributedAsyncInsertBatch::serialize()
{
/// Temporary file is required for atomicity.
String tmp_file{parent.current_batch_file_path + ".tmp"};
auto dir_sync_guard = parent.getDirectorySyncGuard(parent.relative_path);
if (std::filesystem::exists(tmp_file))
LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file));
{
WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT};
writeText(out);
out.finalize();
if (fsync)
out.sync();
}
std::filesystem::rename(tmp_file, parent.current_batch_file_path);
}
void DistributedAsyncInsertBatch::deserialize()
{
ReadBufferFromFile in{parent.current_batch_file_path};
readText(in);
}
bool DistributedAsyncInsertBatch::valid()
{
chassert(!files.empty());
bool res = true;
for (const auto & file : files)
{
if (!fs::exists(file))
{
LOG_WARNING(parent.log, "File {} does not exists, likely due abnormal shutdown", file);
res = false;
}
}
return res;
}
void DistributedAsyncInsertBatch::writeText(WriteBuffer & out)
{
for (const auto & file : files)
{
UInt64 file_index = parse<UInt64>(std::filesystem::path(file).stem());
out << file_index << '\n';
}
}
void DistributedAsyncInsertBatch::readText(ReadBuffer & in)
{
while (!in.eof())
{
UInt64 idx;
in >> idx >> "\n";
files.push_back(std::filesystem::absolute(fmt::format("{}/{}.bin", parent.path, idx)).string());
}
recovered = true;
}
void DistributedAsyncInsertBatch::sendBatch()
{
std::unique_ptr<RemoteInserter> remote;
bool compression_expected = false;
IConnectionPool::Entry connection;
/// Since the batch is sent as a whole (in case of failure, the whole batch
/// will be repeated), we need to mark the whole batch as failed in case of
/// error).
std::vector<OpenTelemetry::TracingContextHolderPtr> tracing_contexts;
UInt64 batch_start_time = clock_gettime_ns();
try
{
for (const auto & file : files)
{
ReadBufferFromFile in(file);
const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log);
tracing_contexts.emplace_back(distributed_header.createTracingContextHolder(
__PRETTY_FUNCTION__,
parent.storage.getContext()->getOpenTelemetrySpanLog()));
tracing_contexts.back()->root_span.addAttribute("clickhouse.distributed_batch_start_time", batch_start_time);
if (!remote)
{
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings);
connection = parent.pool->get(timeouts);
compression_expected = connection->getCompression() == Protocol::Compression::Enable;
LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).",
files.size(),
connection->getDescription(),
formatReadableQuantity(total_rows),
formatReadableSizeWithBinarySuffix(total_bytes));
remote = std::make_unique<RemoteInserter>(*connection, timeouts,
distributed_header.insert_query,
distributed_header.insert_settings,
distributed_header.client_info);
}
writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log);
}
if (remote)
remote->onFinish();
}
catch (...)
{
try
{
for (auto & tracing_context : tracing_contexts)
tracing_context->root_span.addAttribute(std::current_exception());
}
catch (...)
{
tryLogCurrentException(parent.log, "Cannot append exception to tracing context");
}
throw;
}
}
void DistributedAsyncInsertBatch::sendSeparateFiles()
{
size_t broken_files = 0;
for (const auto & file : files)
{
OpenTelemetry::TracingContextHolderPtr trace_context;
try
{
ReadBufferFromFile in(file);
const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log);
// This function is called in a separated thread, so we set up the trace context from the file
trace_context = distributed_header.createTracingContextHolder(
__PRETTY_FUNCTION__,
parent.storage.getContext()->getOpenTelemetrySpanLog());
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings);
auto connection = parent.pool->get(timeouts);
bool compression_expected = connection->getCompression() == Protocol::Compression::Enable;
RemoteInserter remote(*connection, timeouts,
distributed_header.insert_query,
distributed_header.insert_settings,
distributed_header.client_info);
writeRemoteConvert(distributed_header, remote, compression_expected, in, parent.log);
remote.onFinish();
}
catch (Exception & e)
{
trace_context->root_span.addAttribute(std::current_exception());
if (isDistributedSendBroken(e.code(), e.isRemoteException()))
{
parent.markAsBroken(file);
++broken_files;
}
}
}
if (broken_files)
throw Exception(ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES,
"Failed to send {} files", broken_files);
}
}