mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Trying print less scary messages in case No such key for outdated parts
This commit is contained in:
parent
c8dcd34abe
commit
ad279940d6
@ -131,18 +131,18 @@ bool ReadBufferFromS3::nextImpl()
|
|||||||
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds());
|
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds());
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (Exception & e)
|
||||||
{
|
{
|
||||||
watch.stop();
|
watch.stop();
|
||||||
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds());
|
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Microseconds, watch.elapsedMicroseconds());
|
||||||
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1);
|
ProfileEvents::increment(ProfileEvents::ReadBufferFromS3RequestsErrors, 1);
|
||||||
|
|
||||||
if (const auto * s3_exception = dynamic_cast<const S3Exception *>(&e))
|
if (auto * s3_exception = dynamic_cast<S3Exception *>(&e))
|
||||||
{
|
{
|
||||||
/// It doesn't make sense to retry Access Denied or No Such Key
|
/// It doesn't make sense to retry Access Denied or No Such Key
|
||||||
if (!s3_exception->isRetryableError())
|
if (!s3_exception->isRetryableError())
|
||||||
{
|
{
|
||||||
tryLogCurrentException(log, fmt::format("while reading key: {}, from bucket: {}", key, bucket));
|
s3_exception->addMessage("while reading key: {}, from bucket: {}", key, bucket);
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -55,7 +55,7 @@ public:
|
|||||||
bool isRetryableError() const;
|
bool isRetryableError() const;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const Aws::S3::S3Errors code;
|
Aws::S3::S3Errors code;
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <IO/ReadBufferFromMemory.h>
|
#include <IO/ReadBufferFromMemory.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
|
#include <IO/S3Common.h>
|
||||||
#include <Interpreters/Aggregator.h>
|
#include <Interpreters/Aggregator.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/PartLog.h>
|
#include <Interpreters/PartLog.h>
|
||||||
@ -956,6 +957,7 @@ void MergeTreeData::loadDataPartsFromDisk(
|
|||||||
/// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal
|
/// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal
|
||||||
/// others' tasks when finish current disk part loading process.
|
/// others' tasks when finish current disk part loading process.
|
||||||
std::vector<std::vector<std::pair<String, DiskPtr>>> threads_parts(num_threads);
|
std::vector<std::vector<std::pair<String, DiskPtr>>> threads_parts(num_threads);
|
||||||
|
std::unordered_map<std::string, std::string> parts_broken_because_of_no_such_key;
|
||||||
std::set<size_t> remaining_thread_parts;
|
std::set<size_t> remaining_thread_parts;
|
||||||
std::queue<size_t> threads_queue;
|
std::queue<size_t> threads_queue;
|
||||||
for (size_t i = 0; i < num_threads; ++i)
|
for (size_t i = 0; i < num_threads; ++i)
|
||||||
@ -1038,6 +1040,31 @@ void MergeTreeData::loadDataPartsFromDisk(
|
|||||||
{
|
{
|
||||||
part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
part->loadColumnsChecksumsIndexes(require_part_metadata, true);
|
||||||
}
|
}
|
||||||
|
#if USE_AWS_S3
|
||||||
|
/// This code looks really strange. Why can it happen? When we remove something from S3 we can receive different kinds of errors during
|
||||||
|
/// interaction with [Zoo]Keeper, S3 and even local disk. In such way we can get into situation where part is partially removed (at least we
|
||||||
|
/// had an intention to remove it) but server was restarted and now we are trying to load this partially removed part. It can throw an exception
|
||||||
|
/// but if this part is actually covered by some other part it's Ok and we should react to it with <Error> message.
|
||||||
|
///
|
||||||
|
/// The only known case is related to zookeeper connection loss in zero-copy replication during part unlock from [Zoo]Keeper before removal.
|
||||||
|
catch (const S3Exception & e)
|
||||||
|
{
|
||||||
|
broken = true;
|
||||||
|
if (e.getS3ErrorCode() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||||
|
{
|
||||||
|
{
|
||||||
|
std::lock_guard loading_lock(mutex);
|
||||||
|
parts_broken_because_of_no_such_key[part->name] = e.displayText();
|
||||||
|
}
|
||||||
|
LOG_WARNING(log, "Part {} on path {} is broken because of NO_SUCH_KEY error in S3. It's Ok if we had [Zoo]Keeper connection failures during part removal."
|
||||||
|
" Will check that part is covered", part->name, part_path);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
tryLogCurrentException(log, fmt::format("while loading part {} on path {}", part->name, part_path));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
#endif
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
/// Don't count the part as broken if there is not enough memory to load it.
|
/// Don't count the part as broken if there is not enough memory to load it.
|
||||||
@ -1066,7 +1093,7 @@ void MergeTreeData::loadDataPartsFromDisk(
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
tryLogCurrentException(log, fmt::format("while calculating part size {} on path {}", part->name, part_path));
|
tryLogCurrentException(log, fmt::format("while calculating part {} on path {}", part->name, part_path));
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string part_size_str = "failed to calculate size";
|
std::string part_size_str = "failed to calculate size";
|
||||||
@ -1172,6 +1199,14 @@ void MergeTreeData::loadDataPartsFromDisk(
|
|||||||
if (has_lightweight_in_parts)
|
if (has_lightweight_in_parts)
|
||||||
has_lightweight_delete_parts.store(true);
|
has_lightweight_delete_parts.store(true);
|
||||||
|
|
||||||
|
for (const auto & [part_name, exception] : parts_broken_because_of_no_such_key)
|
||||||
|
{
|
||||||
|
if (getActiveContainingPart(part_name) == nullptr)
|
||||||
|
LOG_ERROR(log, "Part {} is broken because of NO_SUCH_KEY error and not covered by any part: {}", part_name, exception);
|
||||||
|
else
|
||||||
|
LOG_DEBUG(log, "Part {} was not completely removed (and NO_SUCH_KEY was thrown on part load), but covered by active part, it's Ok", part_name);
|
||||||
|
}
|
||||||
|
|
||||||
if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks)
|
if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks)
|
||||||
throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS,
|
throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS,
|
||||||
"Suspiciously many ({} parts, {} in total) broken parts to remove while maximum allowed broken parts count is {}. You can change the maximum value "
|
"Suspiciously many ({} parts, {} in total) broken parts to remove while maximum allowed broken parts count is {}. You can change the maximum value "
|
||||||
|
Loading…
Reference in New Issue
Block a user