提交 149be6c0 编写于 作者: N Nikita Mikhaylov

better updates

上级 1f1fabba
......@@ -255,8 +255,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \
M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(SettingUInt64, query_profiler_real_time_period_ns, 0, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
M(SettingUInt64, query_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
\
\
/** Limits during query execution are part of the settings. \
......
......@@ -412,11 +412,10 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
out[row] = true;
};
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids);
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids, on_cell_updated, on_id_not_found);
tryPushToUpdateQueueOrThrow(update_unit_ptr);
waitForCurrentUpdateFinish(update_unit_ptr);
prepareAnswer(update_unit_ptr, on_cell_updated, on_id_not_found);
}
......@@ -761,32 +760,12 @@ void CacheDictionary::updateThreadFunction()
while (update_queue.tryPop(current_unit_ptr))
update_request.emplace_back(std::move(current_unit_ptr));
/// Here we prepare total count of all requested ids
/// not to do useless allocations later.
size_t total_requested_keys_count = 0;
for (auto & unit_ptr: update_request)
total_requested_keys_count += unit_ptr->requested_ids.size();
std::vector<Key> concatenated_requested_ids;
concatenated_requested_ids.reserve(total_requested_keys_count);
for (auto & unit_ptr: update_request)
std::for_each(std::begin(unit_ptr->requested_ids), std::end(unit_ptr->requested_ids),
[&] (const Key & key) {concatenated_requested_ids.push_back(key);});
BunchUpdateUnit bunch_update_unit(update_request);
try
{
auto found_ids_mask_ptr = std::make_shared<std::unordered_map<Key, UInt8>>(concatenated_requested_ids.size());
/// Copy shared_ptr to let this map be alive until other thread finish his stuff.
/// It is thread safe because writing to the map happens before reading from multiple threads.
for (auto & unit_ptr: update_request)
unit_ptr->found_ids_mask_ptr = found_ids_mask_ptr;
for (const auto id : concatenated_requested_ids)
found_ids_mask_ptr->insert({id, 0});
/// Update a bunch of ids.
update(concatenated_requested_ids, *found_ids_mask_ptr);
update(bunch_update_unit);
/// Notify all threads about finished updating the bunch of ids
/// where their own ids were included.
......@@ -832,15 +811,16 @@ void CacheDictionary::tryPushToUpdateQueueOrThrow(UpdateUnitPtr update_unit_ptr)
std::to_string(update_queue.size()), ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL);
}
void CacheDictionary::update(const std::vector<Key> & requested_ids, std::unordered_map<Key, UInt8> & remaining_ids) const
void CacheDictionary::update(BunchUpdateUnit bunch_update_unit) const
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests};
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size());
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, bunch_update_unit.getRequestedIds().size());
const auto now = std::chrono::system_clock::now();
std::unordered_map<Key, UInt8> remaining_ids{bunch_update_unit.getRequestedIds().size()};
for (const auto id : bunch_update_unit.getRequestedIds())
remaining_ids.insert({id, 0});
size_t found_num = 0;
const auto now = std::chrono::system_clock::now();
if (now > backoff_end_time)
{
......@@ -854,48 +834,36 @@ void CacheDictionary::update(const std::vector<Key> & requested_ids, std::unorde
}
Stopwatch watch;
auto stream = source_ptr->loadIds(bunch_update_unit.getRequestedIds());
auto load_ids_start = std::chrono::system_clock::now();
/// Trip to external storage. Might be very bad, slow and blocking.
auto stream = source_ptr->loadIds(requested_ids);
auto load_ids_end = std::chrono::system_clock::now();
LOG_TRACE(log, "Loading " << requested_ids.size() << " number of ids from external storage took " <<
std::chrono::duration_cast<std::chrono::milliseconds>(load_ids_end - load_ids_start).count() << " ms");
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
stream->readPrefix();
while (const auto block = stream->read())
{
const auto id_column = typeid_cast<const ColumnUInt64 *>(block.safeGetByPosition(0).column.get());
if (!id_column)
throw Exception{name + ": id column has type different from UInt64.",
ErrorCodes::TYPE_MISMATCH};
throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH};
const auto &ids = id_column->getData();
const auto & ids = id_column->getData();
/// cache column pointers
const auto column_ptrs = ext::map<std::vector>(
ext::range(0, attributes.size()),
[&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); });
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); });
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
const auto find_result = findCellIdx(id, now);
const auto &cell_idx = find_result.cell_idx;
const auto & cell_idx = find_result.cell_idx;
auto &cell = cells[cell_idx];
auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto &attribute_column = *column_ptrs[attribute_idx];
auto &attribute = attributes[attribute_idx];
const auto & attribute_column = *column_ptrs[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[i]);
}
......@@ -907,15 +875,16 @@ void CacheDictionary::update(const std::vector<Key> & requested_ids, std::unorde
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
{
std::uniform_int_distribution<UInt64> distribution{dict_lifetime.min_sec,
dict_lifetime.max_sec};
std::uniform_int_distribution<UInt64> distribution{dict_lifetime.min_sec, dict_lifetime.max_sec};
cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)});
} else
}
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
bunch_update_unit.informCallersAboutPresentId(id, cell_idx);
/// mark corresponding id as found
remaining_ids[id] = 1;
++found_num;
}
}
......@@ -933,13 +902,74 @@ void CacheDictionary::update(const std::vector<Key> & requested_ids, std::unorde
last_exception = std::current_exception();
backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count));
tryLogException(last_exception, log, "Could not update cache dictionary '" + getName() +
tryLogException(last_exception, log, "Could not update cache dictionary '" + getFullName() +
"', next update is scheduled at " + ext::to_string(backoff_end_time));
}
}
size_t not_found_num = 0, found_num = 0;
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
/// Check which ids have not been found and require setting null_value
for (const auto & id_found_pair : remaining_ids)
{
if (id_found_pair.second)
{
++found_num;
continue;
}
++not_found_num;
const auto id = id_found_pair.first;
const auto find_result = findCellIdx(id, now);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
if (error_count)
{
if (find_result.outdated)
{
/// We have expired data for that `id` so we can continue using it.
bool was_default = cell.isDefault();
cell.setExpiresAt(backoff_end_time);
if (was_default)
cell.setDefault();
if (was_default)
bunch_update_unit.informCallersAboutAbsentId(id, cell_idx);
else
bunch_update_unit.informCallersAboutPresentId(id, cell_idx);
continue;
}
/// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`.
std::rethrow_exception(last_exception);
}
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
{
std::uniform_int_distribution<UInt64> distribution{dict_lifetime.min_sec, dict_lifetime.max_sec};
cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)});
}
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// Set null_value for each attribute
cell.setDefault();
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// inform caller that the cell has not been found
bunch_update_unit.informCallersAboutAbsentId(id, cell_idx);
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, remaining_ids.size() - found_num);
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num);
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num);
ProfileEvents::increment(ProfileEvents::DictCacheRequests);
}
......
......@@ -260,8 +260,6 @@ private:
template <typename DefaultGetter>
void getItemsString(Attribute & attribute, const PaddedPODArray<Key> & ids, ColumnString * out, DefaultGetter && get_default) const;
void update(const std::vector<Key> & requested_ids, std::unordered_map<Key, UInt8> & found_ids_mask_ptr) const;
PaddedPODArray<Key> getCachedIds() const;
bool isEmptyCell(const UInt64 idx) const;
......@@ -331,16 +329,31 @@ private:
/// Field and methods correlated with update expired and not found keys
using PresentIdHandler = std::function<void(Key, size_t)>;
using AbsentIdHandler = std::function<void(Key, size_t)>;
using FoundIdsMaskPtr = std::shared_ptr<std::unordered_map<Key, UInt8>>;
struct UpdateUnit
{
UpdateUnit(std::vector<Key> requested_ids_) : requested_ids(std::move(requested_ids_)) {}
UpdateUnit(std::vector<Key> requested_ids_,
PresentIdHandler present_id_handler_,
AbsentIdHandler absent_id_handler_) :
requested_ids(std::move(requested_ids_)),
present_id_handler(present_id_handler_),
absent_id_handler(absent_id_handler_) {}
explicit UpdateUnit(std::vector<Key> requested_ids_) :
requested_ids(std::move(requested_ids_)),
present_id_handler([](Key, size_t){}),
absent_id_handler([](Key, size_t){}) {}
std::vector<Key> requested_ids;
PresentIdHandler present_id_handler;
AbsentIdHandler absent_id_handler;
FoundIdsMaskPtr found_ids_mask_ptr{nullptr};
std::atomic<bool> is_done{false};
std::exception_ptr current_exception{nullptr};
std::vector<Key> requested_ids;
};
using UpdateUnitPtr = std::shared_ptr<UpdateUnit>;
......@@ -359,8 +372,83 @@ private:
std::atomic<bool> finished{false};
template <typename PresentIdHandler, typename AbsentIdHandler>
void prepareAnswer(UpdateUnitPtr, PresentIdHandler &&, AbsentIdHandler &&) const;
class BunchUpdateUnit
{
public:
explicit BunchUpdateUnit(std::vector<UpdateUnitPtr> update_request)
{
/// Here we prepare total count of all requested ids
/// not to do useless allocations later.
size_t total_requested_keys_count = 0;
helper.push_back(0);
for (auto & unit_ptr: update_request)
{
total_requested_keys_count += unit_ptr->requested_ids.size();
helper.push_back(unit_ptr->requested_ids.size() + helper.back());
present_id_handlers.emplace_back(unit_ptr->present_id_handler);
absent_id_handlers.emplace_back(unit_ptr->absent_id_handler);
}
concatenated_requested_ids.reserve(total_requested_keys_count);
for (auto & unit_ptr: update_request)
std::for_each(std::begin(unit_ptr->requested_ids), std::end(unit_ptr->requested_ids),
[&] (const Key & key) {concatenated_requested_ids.push_back(key);});
}
const std::vector<Key> & getRequestedIds()
{
return concatenated_requested_ids;
}
void informCallersAboutPresentId(Key id, size_t cell_idx)
{
for (size_t i = 0; i < concatenated_requested_ids.size(); ++i)
{
auto & curr = concatenated_requested_ids[i];
if (curr == id)
getPresentIdHandlerForPosition(i)(id, cell_idx);
}
}
void informCallersAboutAbsentId(Key id, size_t cell_idx)
{
for (size_t i = 0; i < concatenated_requested_ids.size(); ++i) {
auto &curr = concatenated_requested_ids[i];
if (curr == id)
getAbsentIdHandlerForPosition(i)(id, cell_idx);
}
}
private:
PresentIdHandler & getPresentIdHandlerForPosition(size_t position)
{
auto i = getUpdateUnitNumberForRequestedIdPosition(position);
return present_id_handlers[i];
}
AbsentIdHandler & getAbsentIdHandlerForPosition(size_t position)
{
return absent_id_handlers[getUpdateUnitNumberForRequestedIdPosition((position))];
}
size_t getUpdateUnitNumberForRequestedIdPosition(size_t position)
{
return std::lower_bound(helper.begin(), helper.end(), position) - helper.begin();
}
std::vector<Key> concatenated_requested_ids;
std::vector<PresentIdHandler> present_id_handlers;
std::vector<AbsentIdHandler> absent_id_handlers;
std::vector<size_t> helper;
};
void update(BunchUpdateUnit bunch_update_unit) const;
};
}
......@@ -137,9 +137,6 @@ void CacheDictionary::getItemsNumberImpl(
std::begin(cache_expired_ids), std::end(cache_expired_ids),
std::back_inserter(required_ids), [](auto & pair) { return pair.first; });
/// Request new values
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids);
auto on_cell_updated = [&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
......@@ -160,9 +157,11 @@ void CacheDictionary::getItemsNumberImpl(
out[row] = get_default(row);
};
/// Request new values
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids, on_cell_updated, on_id_not_found);
tryPushToUpdateQueueOrThrow(update_unit_ptr);
waitForCurrentUpdateFinish(update_unit_ptr);
prepareAnswer(update_unit_ptr, on_cell_updated, on_id_not_found);
}
template <typename DefaultGetter>
......@@ -317,11 +316,10 @@ void CacheDictionary::getItemsString(
total_length += get_default(row).size + 1;
};
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids);
auto update_unit_ptr = std::make_shared<UpdateUnit>(required_ids, on_cell_updated, on_id_not_found);
tryPushToUpdateQueueOrThrow(update_unit_ptr);
waitForCurrentUpdateFinish(update_unit_ptr);
prepareAnswer(update_unit_ptr, on_cell_updated, on_id_not_found);
}
out->getChars().reserve(total_length);
......@@ -336,74 +334,4 @@ void CacheDictionary::getItemsString(
}
}
template <typename PresentIdHandler, typename AbsentIdHandler>
void CacheDictionary::prepareAnswer(
UpdateUnitPtr update_unit_ptr,
PresentIdHandler && on_cell_updated,
AbsentIdHandler && on_id_not_found) const
{
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
/// Prepare answer
const auto now = std::chrono::system_clock::now();
for (const auto & id : update_unit_ptr->requested_ids)
{
const auto find_result = findCellIdx(id, now);
assert(find_result.valid);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
const auto was_id_updated = update_unit_ptr->found_ids_mask_ptr->at(id);
if (was_id_updated)
{
on_cell_updated(id, find_result.cell_idx);
continue;
}
if (error_count)
{
if (find_result.outdated)
{
/// We have expired data for that `id` so we can continue using it.
bool was_default = cell.isDefault();
cell.setExpiresAt(backoff_end_time);
if (was_default)
cell.setDefault();
if (was_default)
on_id_not_found(id, cell_idx);
else
on_cell_updated(id, cell_idx);
continue;
}
/// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`.
std::rethrow_exception(last_exception);
}
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
{
std::uniform_int_distribution<UInt64> distribution{dict_lifetime.min_sec, dict_lifetime.max_sec};
cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)});
}
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// Set null_value for each attribute
cell.setDefault();
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// inform caller that the cell has not been found
on_id_not_found(id, cell_idx);
}
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册