mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
after review
This commit is contained in:
parent
01ef99b704
commit
7af66db1b2
@ -756,7 +756,7 @@ void CacheDictionary::updateThreadFunction()
|
||||
|
||||
UpdateUnitPtr current_unit_ptr;
|
||||
|
||||
while (update_queue.tryPop(current_unit_ptr))
|
||||
while (update_request.size() && update_queue.tryPop(current_unit_ptr))
|
||||
update_request.emplace_back(std::move(current_unit_ptr));
|
||||
|
||||
BunchUpdateUnit bunch_update_unit(update_request);
|
||||
@ -793,7 +793,7 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr
|
||||
std::unique_lock<std::mutex> lock(update_mutex);
|
||||
const auto sleeping_result = is_update_finished.wait_for(
|
||||
lock,
|
||||
std::chrono::minutes(each_update_finish_timeout_seconds),
|
||||
std::chrono::seconds(each_update_finish_timeout_seconds),
|
||||
[&] {return update_unit_ptr->is_done || update_unit_ptr->current_exception; });
|
||||
|
||||
if (!sleeping_result)
|
||||
@ -806,8 +806,10 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr
|
||||
void CacheDictionary::tryPushToUpdateQueueOrThrow(UpdateUnitPtr & update_unit_ptr) const
|
||||
{
|
||||
if (!update_queue.tryPush(update_unit_ptr, update_queue_push_timeout_milliseconds))
|
||||
throw DB::Exception("Cannot push to internal update queue. Current queue size is " +
|
||||
std::to_string(update_queue.size()), ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL);
|
||||
throw DB::Exception(
|
||||
"Cannot push to internal update queue in dictionary " + getFullName() + ". Timelimit of " +
|
||||
std::to_string(update_queue_push_timeout_milliseconds) + " ms. exceeded. Current queue size is " +
|
||||
std::to_string(update_queue.size()), ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL);
|
||||
}
|
||||
|
||||
void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const
|
||||
|
Loading…
Reference in New Issue
Block a user