mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-10-08 17:40:49 +00:00
exception ptr added
This commit is contained in:
parent
2545f66fe5
commit
09805b2396
@ -385,10 +385,13 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
|
|||||||
throw std::runtime_error("Too many updates");
|
throw std::runtime_error("Too many updates");
|
||||||
|
|
||||||
// waitForCurrentUpdateFinish();
|
// waitForCurrentUpdateFinish();
|
||||||
while (!update_unit_ptr->is_done) {
|
while (!update_unit_ptr->is_done && !update_unit_ptr->current_exception) {
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||||
std::this_thread::yield();
|
std::this_thread::yield();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (update_unit_ptr->current_exception)
|
||||||
|
std::rethrow_exception(update_unit_ptr->current_exception);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -670,21 +673,20 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
|||||||
|
|
||||||
void CacheDictionary::updateThreadFunction()
|
void CacheDictionary::updateThreadFunction()
|
||||||
{
|
{
|
||||||
try
|
while (!finished)
|
||||||
{
|
{
|
||||||
while (!finished)
|
UpdateUnitPtr unit_ptr;
|
||||||
|
update_queue.pop(unit_ptr);
|
||||||
|
try
|
||||||
{
|
{
|
||||||
UpdateUnitPtr unit_ptr;
|
|
||||||
update_queue.pop(unit_ptr);
|
|
||||||
|
|
||||||
update(unit_ptr->requested_ids, unit_ptr->on_cell_updated, unit_ptr->on_id_not_found);
|
update(unit_ptr->requested_ids, unit_ptr->on_cell_updated, unit_ptr->on_id_not_found);
|
||||||
unit_ptr->is_done = true;
|
unit_ptr->is_done = true;
|
||||||
last_update.fetch_add(1);
|
last_update.fetch_add(1);
|
||||||
}
|
}
|
||||||
}
|
catch (...)
|
||||||
catch (...)
|
{
|
||||||
{
|
unit_ptr->current_exception = std::current_exception();
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -310,6 +310,7 @@ private:
|
|||||||
on_id_not_found(std::move(on_id_not_found_)) {}
|
on_id_not_found(std::move(on_id_not_found_)) {}
|
||||||
|
|
||||||
std::atomic<bool> is_done{false};
|
std::atomic<bool> is_done{false};
|
||||||
|
std::exception_ptr current_exception{nullptr};
|
||||||
std::vector<Key> requested_ids;
|
std::vector<Key> requested_ids;
|
||||||
std::function<void(const Key, const size_t)> on_cell_updated;
|
std::function<void(const Key, const size_t)> on_cell_updated;
|
||||||
std::function<void(const Key, const size_t)> on_id_not_found;
|
std::function<void(const Key, const size_t)> on_id_not_found;
|
||||||
|
@ -114,10 +114,13 @@ void CacheDictionary::getItemsNumberImpl(
|
|||||||
throw std::runtime_error("Too many updates");
|
throw std::runtime_error("Too many updates");
|
||||||
|
|
||||||
// waitForCurrentUpdateFinish();
|
// waitForCurrentUpdateFinish();
|
||||||
while (!update_unit_ptr->is_done) {
|
while (!update_unit_ptr->is_done && !update_unit_ptr->current_exception) {
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||||
std::this_thread::yield();
|
std::this_thread::yield();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (update_unit_ptr->current_exception)
|
||||||
|
std::rethrow_exception(update_unit_ptr->current_exception);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename DefaultGetter>
|
template <typename DefaultGetter>
|
||||||
@ -271,10 +274,13 @@ void CacheDictionary::getItemsString(
|
|||||||
throw std::runtime_error("Too many updates");
|
throw std::runtime_error("Too many updates");
|
||||||
|
|
||||||
// waitForCurrentUpdateFinish();
|
// waitForCurrentUpdateFinish();
|
||||||
while (!update_unit_ptr->is_done) {
|
while (!update_unit_ptr->is_done && !update_unit_ptr->current_exception) {
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||||
std::this_thread::yield();
|
std::this_thread::yield();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (update_unit_ptr->current_exception)
|
||||||
|
std::rethrow_exception(update_unit_ptr->current_exception);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -304,10 +310,13 @@ void CacheDictionary::getItemsString(
|
|||||||
throw std::runtime_error("Too many updates");
|
throw std::runtime_error("Too many updates");
|
||||||
|
|
||||||
// waitForCurrentUpdateFinish();begin
|
// waitForCurrentUpdateFinish();begin
|
||||||
while (!update_unit_ptr->is_done) {
|
while (!update_unit_ptr->is_done && !update_unit_ptr->current_exception) {
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||||
std::this_thread::yield();
|
std::this_thread::yield();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (update_unit_ptr->current_exception)
|
||||||
|
std::rethrow_exception(update_unit_ptr->current_exception);
|
||||||
}
|
}
|
||||||
|
|
||||||
out->getChars().reserve(total_length);
|
out->getChars().reserve(total_length);
|
||||||
|
Loading…
Reference in New Issue
Block a user