mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Query CHECK TABLE takes care about progress and cancellation
This commit is contained in:
parent
67a87a4b95
commit
a882ef295f
@ -84,10 +84,20 @@ size_t FileChecker::getTotalSize() const
|
||||
|
||||
CheckResults FileChecker::check() const
|
||||
{
|
||||
if (map.empty())
|
||||
return {};
|
||||
|
||||
CheckResults results;
|
||||
auto callback = [&results](const CheckResult & result, size_t) -> bool
|
||||
{
|
||||
results.push_back(result);
|
||||
return true;
|
||||
};
|
||||
check(callback);
|
||||
return results;
|
||||
}
|
||||
|
||||
void FileChecker::check(CheckDataCallback check_callback) const
|
||||
{
|
||||
if (map.empty())
|
||||
return;
|
||||
|
||||
for (const auto & name_size : map)
|
||||
{
|
||||
@ -101,14 +111,12 @@ CheckResults FileChecker::check() const
|
||||
String failure_message = exists
|
||||
? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second))
|
||||
: ("File " + path + " doesn't exist");
|
||||
results.emplace_back(name, false, failure_message);
|
||||
check_callback(CheckResult(name, false, failure_message), map.size());
|
||||
break;
|
||||
}
|
||||
|
||||
results.emplace_back(name, true, "");
|
||||
check_callback(CheckResult(name, true, ""), map.size());
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
void FileChecker::repair()
|
||||
|
@ -29,6 +29,7 @@ public:
|
||||
|
||||
/// Check the files whose parameters are specified in sizes.json
|
||||
CheckResults check() const;
|
||||
void check(CheckDataCallback check_callback) const;
|
||||
|
||||
/// Truncate files that have excessive size to the expected size.
|
||||
/// Throw exception if the file size is less than expected.
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <algorithm>
|
||||
|
||||
|
||||
@ -17,43 +18,19 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
NamesAndTypes getBlockStructure()
|
||||
Block getBlockFromCheckResult(const CheckResults & check_results, bool check_query_single_value_result)
|
||||
{
|
||||
return {
|
||||
if (check_query_single_value_result)
|
||||
{
|
||||
bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; });
|
||||
return Block{{ColumnUInt8::create(1, static_cast<UInt8>(result)), std::make_shared<DataTypeUInt8>(), "result"}};
|
||||
}
|
||||
|
||||
NamesAndTypes block_structure = NamesAndTypes{
|
||||
{"part_path", std::make_shared<DataTypeString>()},
|
||||
{"is_passed", std::make_shared<DataTypeUInt8>()},
|
||||
{"message", std::make_shared<DataTypeString>()},
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCheckQuery::execute()
|
||||
{
|
||||
const auto & check = query_ptr->as<ASTCheckQuery &>();
|
||||
auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary);
|
||||
|
||||
getContext()->checkAccess(AccessType::SHOW_TABLES, table_id);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
auto check_results = table->checkData(query_ptr, getContext());
|
||||
|
||||
Block block;
|
||||
if (getContext()->getSettingsRef().check_query_single_value_result)
|
||||
{
|
||||
bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; });
|
||||
auto column = ColumnUInt8::create();
|
||||
column->insertValue(static_cast<UInt64>(result));
|
||||
block = Block{{std::move(column), std::make_shared<DataTypeUInt8>(), "result"}};
|
||||
}
|
||||
else
|
||||
{
|
||||
auto block_structure = getBlockStructure();
|
||||
auto path_column = block_structure[0].type->createColumn();
|
||||
auto is_passed_column = block_structure[1].type->createColumn();
|
||||
auto message_column = block_structure[2].type->createColumn();
|
||||
@ -65,15 +42,134 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
message_column->insert(check_result.failure_message);
|
||||
}
|
||||
|
||||
block = Block({
|
||||
return Block({
|
||||
{std::move(path_column), block_structure[0].type, block_structure[0].name},
|
||||
{std::move(is_passed_column), block_structure[1].type, block_structure[1].name},
|
||||
{std::move(message_column), block_structure[2].type, block_structure[2].name}});
|
||||
{std::move(message_column), block_structure[2].type, block_structure[2].name},
|
||||
});
|
||||
}
|
||||
|
||||
class TableCheckResultSource : public ISource
|
||||
{
|
||||
|
||||
public:
|
||||
explicit TableCheckResultSource(const ASTPtr & query_ptr_, StoragePtr table_, bool check_query_single_value_result_, ContextPtr context_)
|
||||
: ISource(getBlockFromCheckResult({}, check_query_single_value_result_).cloneEmpty())
|
||||
, query_ptr(query_ptr_)
|
||||
, table(table_)
|
||||
, context(context_)
|
||||
, check_query_single_value_result(check_query_single_value_result_)
|
||||
{
|
||||
worker_result = std::async(std::launch::async, [this]{ worker(); });
|
||||
}
|
||||
|
||||
BlockIO res;
|
||||
res.pipeline = QueryPipeline(std::make_shared<SourceFromSingleChunk>(std::move(block)));
|
||||
String getName() const override { return "TableCheckResultSource"; }
|
||||
|
||||
protected:
|
||||
|
||||
std::optional<Chunk> tryGenerate() override
|
||||
{
|
||||
|
||||
if (is_check_completed)
|
||||
return {};
|
||||
|
||||
auto status = worker_result.wait_for(std::chrono::milliseconds(100));
|
||||
is_check_completed = (status == std::future_status::ready);
|
||||
|
||||
if (is_check_completed)
|
||||
{
|
||||
worker_result.get();
|
||||
auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result);
|
||||
check_results.clear();
|
||||
return Chunk(result_block.getColumns(), result_block.rows());
|
||||
}
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
progress(progress_rows, 0);
|
||||
progress_rows = 0;
|
||||
|
||||
if (check_query_single_value_result || check_results.empty())
|
||||
{
|
||||
return Chunk();
|
||||
}
|
||||
|
||||
auto result_block = getBlockFromCheckResult(check_results, check_query_single_value_result);
|
||||
check_results.clear();
|
||||
return Chunk(result_block.getColumns(), result_block.rows());
|
||||
}
|
||||
|
||||
private:
|
||||
void worker()
|
||||
{
|
||||
table->checkData(query_ptr, context,
|
||||
[this](const CheckResult & check_result, size_t new_total_rows)
|
||||
{
|
||||
if (isCancelled())
|
||||
return false;
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
if (new_total_rows > total_rows)
|
||||
{
|
||||
addTotalRowsApprox(new_total_rows - total_rows);
|
||||
total_rows = new_total_rows;
|
||||
}
|
||||
progress_rows++;
|
||||
|
||||
if (!check_result.success)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"),
|
||||
"Check query for table {} failed, path {}, reason: {}",
|
||||
table->getStorageID().getNameForLogs(),
|
||||
check_result.fs_path,
|
||||
check_result.failure_message);
|
||||
}
|
||||
|
||||
check_results.push_back(check_result);
|
||||
|
||||
bool should_continue = check_result.success || !check_query_single_value_result;
|
||||
return should_continue;
|
||||
});
|
||||
}
|
||||
|
||||
ASTPtr query_ptr;
|
||||
StoragePtr table;
|
||||
ContextPtr context;
|
||||
bool check_query_single_value_result;
|
||||
|
||||
std::future<void> worker_result;
|
||||
|
||||
std::mutex mutex;
|
||||
CheckResults check_results;
|
||||
size_t progress_rows = 0;
|
||||
size_t total_rows = 0;
|
||||
|
||||
bool is_check_completed = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCheckQuery::execute()
|
||||
{
|
||||
const auto & check = query_ptr->as<ASTCheckQuery &>();
|
||||
const auto & context = getContext();
|
||||
auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary);
|
||||
|
||||
context->checkAccess(AccessType::SHOW_TABLES, table_id);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
|
||||
BlockIO res;
|
||||
{
|
||||
bool check_query_single_value_result = context->getSettingsRef().check_query_single_value_result;
|
||||
auto result_source = std::make_shared<TableCheckResultSource>(query_ptr, table, check_query_single_value_result, context);
|
||||
res.pipeline = QueryPipeline(result_source);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -22,6 +22,11 @@ struct CheckResult
|
||||
{}
|
||||
};
|
||||
|
||||
/// Process single result of checkData
|
||||
/// Second argument is an estimated number of check results
|
||||
/// Return true to continue checking, false to stop
|
||||
using CheckDataCallback = std::function<bool(const CheckResult &, size_t)>;
|
||||
|
||||
using CheckResults = std::vector<CheckResult>;
|
||||
|
||||
}
|
||||
|
@ -273,6 +273,14 @@ bool IStorage::isStaticStorage() const
|
||||
return false;
|
||||
}
|
||||
|
||||
CheckResults IStorage::checkData(const ASTPtr & query, ContextPtr context)
|
||||
{
|
||||
CheckResults results;
|
||||
auto callback = [&](const CheckResult & result, size_t) { results.push_back(result); return true;};
|
||||
checkData(query, context, callback);
|
||||
return results;
|
||||
}
|
||||
|
||||
void IStorage::adjustCreateQueryForBackup(ASTPtr &) const
|
||||
{
|
||||
}
|
||||
|
@ -596,7 +596,9 @@ public:
|
||||
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; }
|
||||
|
||||
/// Checks validity of the data
|
||||
virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); }
|
||||
virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */);
|
||||
|
||||
virtual void checkData(const ASTPtr & /* query */, ContextPtr /* context */, CheckDataCallback /* callback */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); }
|
||||
|
||||
/// Checks that table could be dropped right now
|
||||
/// Otherwise - throws an exception with detailed information.
|
||||
|
@ -866,13 +866,13 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada
|
||||
return std::make_shared<LogSink>(*this, metadata_snapshot, std::move(lock));
|
||||
}
|
||||
|
||||
CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context)
|
||||
void StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback)
|
||||
{
|
||||
ReadLock lock{rwlock, getLockTimeout(local_context)};
|
||||
if (!lock)
|
||||
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
||||
|
||||
return file_checker.check();
|
||||
file_checker.check(check_callback);
|
||||
}
|
||||
|
||||
|
||||
|
@ -59,7 +59,7 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override;
|
||||
void checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback) override;
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
|
||||
|
||||
|
@ -2197,9 +2197,8 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type)
|
||||
background_moves_assignee.trigger();
|
||||
}
|
||||
|
||||
CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
|
||||
void StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback)
|
||||
{
|
||||
CheckResults results;
|
||||
DataPartsVector data_parts;
|
||||
if (const auto & check_query = query->as<ASTCheckQuery &>(); check_query.partition)
|
||||
{
|
||||
@ -2224,12 +2223,16 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
|
||||
part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings());
|
||||
|
||||
part->checkMetadata();
|
||||
results.emplace_back(part->name, true, "Checksums recounted and written to disk.");
|
||||
bool should_continue = check_callback(CheckResult(part->name, true, "Checksums recounted and written to disk."), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
|
||||
bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -2238,15 +2241,18 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
|
||||
{
|
||||
checkDataPart(part, true);
|
||||
part->checkMetadata();
|
||||
results.emplace_back(part->name, true, "");
|
||||
bool should_continue = check_callback(CheckResult(part->name, true, ""), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
results.emplace_back(part->name, false, ex.message());
|
||||
bool should_continue = check_callback(CheckResult(part->name, false, ex.message()), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
|
||||
|
@ -108,7 +108,7 @@ public:
|
||||
|
||||
void onActionLockRemove(StorageActionBlockType action_type) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
||||
void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override;
|
||||
|
||||
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
||||
|
||||
|
@ -149,7 +149,8 @@ public:
|
||||
return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
|
||||
}
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); }
|
||||
void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override { getNested()->checkData(query, context, check_callback); }
|
||||
|
||||
void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); }
|
||||
bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); }
|
||||
Strings getDataPaths() const override { return getNested()->getDataPaths(); }
|
||||
|
@ -8481,9 +8481,8 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t
|
||||
part_check_thread.enqueuePart(part_name, delay_to_check_seconds);
|
||||
}
|
||||
|
||||
CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
|
||||
void StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context, CheckDataCallback check_callback)
|
||||
{
|
||||
CheckResults results;
|
||||
DataPartsVector data_parts;
|
||||
if (const auto & check_query = query->as<ASTCheckQuery &>(); check_query.partition)
|
||||
{
|
||||
@ -8500,17 +8499,19 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context
|
||||
{
|
||||
try
|
||||
{
|
||||
results.push_back(part_check_thread.checkPartAndFix(part->name));
|
||||
bool should_continue = check_callback(part_check_thread.checkPartAndFix(part->name), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
|
||||
bool should_continue = check_callback(CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"), data_parts.size());
|
||||
if (!should_continue)
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
|
||||
|
@ -230,7 +230,7 @@ public:
|
||||
/// Add a part to the queue of parts whose data you want to check in the background thread.
|
||||
void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0);
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
||||
void checkData(const ASTPtr & query, ContextPtr context, CheckDataCallback check_callback) override;
|
||||
|
||||
/// Checks ability to use granularity
|
||||
bool canUseAdaptiveGranularity() const override;
|
||||
|
@ -403,14 +403,13 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage
|
||||
return std::make_shared<StripeLogSink>(*this, metadata_snapshot, std::move(lock));
|
||||
}
|
||||
|
||||
|
||||
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context)
|
||||
void StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context, CheckDataCallback check_callback)
|
||||
{
|
||||
ReadLock lock{rwlock, getLockTimeout(local_context)};
|
||||
if (!lock)
|
||||
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
||||
|
||||
return file_checker.check();
|
||||
file_checker.check(check_callback);
|
||||
}
|
||||
|
||||
|
||||
|
@ -53,7 +53,7 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override;
|
||||
void checkData(const ASTPtr & query, ContextPtr ocal_context, CheckDataCallback check_callback) override;
|
||||
|
||||
bool storesDataOnDisk() const override { return true; }
|
||||
Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; }
|
||||
|
Loading…
Reference in New Issue
Block a user