Replacing std::function to Fn in some places

to avoid dynamic allocating while keeping desired function signature
clear.
Simplifying SimpleCache (CachedFn) using C++20
This commit is contained in:
Mike Kot 2021-09-28 23:01:16 +02:00
parent d4823d41b0
commit 7670ea50c1
16 changed files with 150 additions and 124 deletions

74
base/common/CachedFn.h Normal file
View File

@ -0,0 +1,74 @@
#pragma once
#include <map>
#include <mutex>
#include "FnTraits.h"
/**
* Cache for a functor that decays to a pointer-to-function.
* The size is unlimited. Values are stored permanently and never evicted.
* Suitable only for simplest cases.
*
* Invocation/update is O(log(saved cache values)).
*
* @example CachedFn<&my_func> cached; cached(arg);
*/
template <auto * Func>
struct CachedFn
{
private:
using Traits = FnTraits<decltype(Func)>;
using Key = typename Traits::DecayedArgs;
using Result = typename Traits::Ret;
std::map<Key, Result> cache; // Can't use hashmap as tuples are unhashable by default
mutable std::mutex mutex;
public:
template <class ...Args>
Result operator()(Args && ...args)
{
Key key{std::forward<Args>(args)...};
{
std::lock_guard lock(mutex);
if (auto it = cache.find(key); it != cache.end())
return it->second;
}
/// The calculations themselves are not done under mutex.
Result res = std::apply(Func, key);
{
std::lock_guard lock(mutex);
cache.emplace(std::move(key), res);
}
return res;
}
template <class ...Args>
void update(Args && ...args)
{
Key key{std::forward<Args>(args)...};
Result res = std::apply(Func, key);
{
std::lock_guard lock(mutex);
cache.emplace(std::move(key), std::move(res));
}
}
size_t size() const
{
std::lock_guard lock(mutex);
return cache.size();
}
void drop()
{
std::lock_guard lock(mutex);
cache.clear();
}
};

37
base/common/FnTraits.h Normal file
View File

@ -0,0 +1,37 @@
#pragma once
#include <tuple>
#include <concepts>
#include <type_traits>
namespace detail
{
template <class T>
struct FnTraits { template <class> static constexpr bool value = false; };
template <class R, class ...A>
struct FnTraits<R(A...)>
{
template <class F>
static constexpr bool value = std::is_invocable_r_v<R, F, A...>;
using Ret = R;
using Args = std::tuple<A...>;
using DecayedArgs = std::tuple<typename std::decay<A>::type...>;
};
template <class R, class ...A>
struct FnTraits<R(*)(A...)> : FnTraits<R(A...)> {};
}
template <class T> using FnTraits = detail::FnTraits<T>;
/**
* A less-typing alias for std::is_invokable_r_v.
* @example void foo(Fn<bool(int, char)> auto && functor)
*/
template <class F, class FS>
concept Fn = FnTraits<FS>::template value<F>;
template <auto Value>
using Constant = std::integral_constant<decltype(Value), Value>;

View File

@ -1,82 +0,0 @@
#pragma once
#include <map>
#include <tuple>
#include <mutex>
#include <common/function_traits.h>
/** The simplest cache for a free function.
* You can also pass a static class method or lambda without captures.
* The size is unlimited. Values are stored permanently and never evicted.
* But single record or all cache can be manually dropped.
* Mutex is used for synchronization.
* Suitable only for the simplest cases.
*
* Usage
*
* SimpleCache<decltype(func), &func> func_cached;
* std::cerr << func_cached(args...);
*/
template <typename F, F* f>
class SimpleCache
{
private:
using Key = typename function_traits<F>::arguments_decay;
using Result = typename function_traits<F>::result;
std::map<Key, Result> cache;
mutable std::mutex mutex;
public:
template <typename... Args>
Result operator() (Args &&... args)
{
Key key{std::forward<Args>(args)...};
{
std::lock_guard lock(mutex);
auto it = cache.find(key);
if (cache.end() != it)
return it->second;
}
/// The calculations themselves are not done under mutex.
Result res = std::apply(f, key);
{
std::lock_guard lock(mutex);
cache.emplace(std::forward_as_tuple(args...), res);
}
return res;
}
template <typename... Args>
void update(Args &&... args)
{
Key key{std::forward<Args>(args)...};
Result res = std::apply(f, key);
{
std::lock_guard lock(mutex);
cache[key] = std::move(res);
}
}
size_t size() const
{
std::lock_guard lock(mutex);
return cache.size();
}
void drop()
{
std::lock_guard lock(mutex);
cache.clear();
}
};

View File

@ -1,16 +0,0 @@
#pragma once
#include <tuple>
#include <type_traits>
template <typename T>
struct function_traits;
template <typename ReturnType, typename... Args>
struct function_traits<ReturnType(Args...)>
{
using result = ReturnType;
using arguments = std::tuple<Args...>;
using arguments_decay = std::tuple<typename std::decay<Args>::type...>;
};

View File

@ -1,6 +1,5 @@
#include <Access/AllowedClientHosts.h>
#include <Common/Exception.h>
#include <common/SimpleCache.h>
#include <common/logger_useful.h>
#include <common/scope_guard.h>
#include <Functions/likePatternToRegexp.h>

View File

@ -6,6 +6,7 @@
#include <IO/WriteHelpers.h>
#include <Poco/UUIDGenerator.h>
#include <Poco/Logger.h>
#include <common/FnTraits.h>
namespace DB
@ -96,7 +97,7 @@ namespace
bool errors() const { return exception.has_value(); }
void showErrors(const char * format, const std::function<String(size_t)> & get_name_function)
void showErrors(const char * format, Fn<String(size_t)> auto && get_name_function)
{
if (!exception)
return;

View File

@ -3,6 +3,7 @@
#include <Access/EnabledRolesInfo.h>
#include <Access/AccessControlManager.h>
#include <boost/container/flat_set.hpp>
#include <common/FnTraits.h>
namespace DB
@ -11,7 +12,7 @@ namespace
{
void collectRoles(EnabledRolesInfo & roles_info,
boost::container::flat_set<UUID> & skip_ids,
const std::function<RolePtr(const UUID &)> & get_role_function,
Fn<RolePtr(const UUID &)> auto && get_role_function,
const UUID & role_id,
bool is_current_role,
bool with_admin_option)
@ -113,7 +114,9 @@ void RoleCache::collectEnabledRoles(EnabledRoles & enabled, scope_guard & notifi
/// Collect enabled roles. That includes the current roles, the roles granted to the current roles, and so on.
auto new_info = std::make_shared<EnabledRolesInfo>();
boost::container::flat_set<UUID> skip_ids;
auto get_role_function = [this](const UUID & id) { return getRole(id); };
for (const auto & current_role : enabled.params.current_roles)
collectRoles(*new_info, skip_ids, get_role_function, current_role, true, false);

View File

@ -18,6 +18,7 @@
#include <boost/range/adaptor/map.hpp>
#include <cstring>
#include <filesystem>
#include <common/FnTraits.h>
namespace DB
@ -362,7 +363,7 @@ namespace
SettingsProfileElements parseSettingsConstraints(const Poco::Util::AbstractConfiguration & config,
const String & path_to_constraints,
const std::function<void(const std::string_view &)> & check_setting_name_function)
Fn<void(std::string_view)> auto && check_setting_name_function)
{
SettingsProfileElements profile_elements;
Poco::Util::AbstractConfiguration::Keys keys;
@ -399,7 +400,7 @@ namespace
std::shared_ptr<SettingsProfile> parseSettingsProfile(
const Poco::Util::AbstractConfiguration & config,
const String & profile_name,
const std::function<void(const std::string_view &)> & check_setting_name_function)
Fn<void(std::string_view)> auto && check_setting_name_function)
{
auto profile = std::make_shared<SettingsProfile>();
profile->setName(profile_name);
@ -441,7 +442,7 @@ namespace
std::vector<AccessEntityPtr> parseSettingsProfiles(
const Poco::Util::AbstractConfiguration & config,
const std::function<void(const std::string_view &)> & check_setting_name_function)
Fn<void(std::string_view)> auto && check_setting_name_function)
{
std::vector<AccessEntityPtr> profiles;
Poco::Util::AbstractConfiguration::Keys profile_names;

View File

@ -1,5 +1,5 @@
#include "DNSResolver.h"
#include <common/SimpleCache.h>
#include <common/CachedFn.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <Core/Names.h>
@ -146,8 +146,8 @@ static String reverseResolveImpl(const Poco::Net::IPAddress & address)
struct DNSResolver::Impl
{
SimpleCache<decltype(resolveIPAddressImpl), &resolveIPAddressImpl> cache_host;
SimpleCache<decltype(reverseResolveImpl), &reverseResolveImpl> cache_address;
CachedFn<&resolveIPAddressImpl> cache_host;
CachedFn<&reverseResolveImpl> cache_address;
std::mutex drop_mutex;
std::mutex update_mutex;

View File

@ -5,7 +5,7 @@
#include <Common/Elf.h>
#include <Common/SymbolIndex.h>
#include <Common/MemorySanitizer.h>
#include <common/SimpleCache.h>
#include <common/CachedFn.h>
#include <common/demangle.h>
#include <cstring>
@ -21,7 +21,7 @@
# include <libunwind.h>
#endif
std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context)
std::string signalToErrorMessage(int sig, const siginfo_t & info, [[maybe_unused]] const ucontext_t & context)
{
std::stringstream error; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
error.exceptions(std::ios::failbit);
@ -41,8 +41,6 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext
error << " Access: write.";
else
error << " Access: read.";
#else
UNUSED(context);
#endif
switch (info.si_code)
@ -197,7 +195,9 @@ static void * getCallerAddress(const ucontext_t & context)
#endif
}
void StackTrace::symbolize(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size, StackTrace::Frames & frames)
void StackTrace::symbolize(
const StackTrace::FramePointers & frame_pointers, [[maybe_unused]] size_t offset,
size_t size, StackTrace::Frames & frames)
{
#if defined(__ELF__) && !defined(__FreeBSD__) && !defined(ARCADIA_BUILD)
@ -256,7 +256,6 @@ void StackTrace::symbolize(const StackTrace::FramePointers & frame_pointers, siz
{
frames[i].virtual_addr = frame_pointers[i];
}
UNUSED(offset);
#endif
}
@ -322,7 +321,7 @@ const StackTrace::FramePointers & StackTrace::getFramePointers() const
}
static void toStringEveryLineImpl(
bool fatal,
[[maybe_unused]] bool fatal,
const StackTrace::FramePointers & frame_pointers,
size_t offset,
size_t size,
@ -386,7 +385,6 @@ static void toStringEveryLineImpl(
out.str({});
}
#else
UNUSED(fatal);
std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
out.exceptions(std::ios::failbit);
@ -431,9 +429,9 @@ std::string StackTrace::toString(void ** frame_pointers_, size_t offset, size_t
return toStringStatic(frame_pointers_copy, offset, size);
}
static SimpleCache<decltype(toStringImpl), &toStringImpl> & cacheInstance()
static CachedFn<&toStringImpl> & cacheInstance()
{
static SimpleCache<decltype(toStringImpl), &toStringImpl> cache;
static CachedFn<&toStringImpl> cache;
return cache;
}

View File

@ -1,5 +1,5 @@
#include <iostream>
#include <common/SimpleCache.h>
#include <common/CachedFn.h>
static int func(int x, int y)
@ -11,7 +11,7 @@ static int func(int x, int y)
int main(int, char **)
{
SimpleCache<decltype(func), &func> func_cached;
CachedFn<&func> func_cached;
std::cerr << func_cached(1, 2) << "\n";
std::cerr << func_cached(1, 2) << "\n";

View File

@ -15,6 +15,7 @@
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <filesystem>
#include <common/FnTraits.h>
namespace fs = std::filesystem;
@ -221,7 +222,7 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover:
return false;
}
void commitMetadata(const std::function<void()> & function, const String & persistent_tmp_path, const String & persistent_path)
void commitMetadata(Fn<void()> auto && function, const String & persistent_tmp_path, const String & persistent_path)
{
try
{

View File

@ -14,6 +14,7 @@
#include <boost/algorithm/string/trim.hpp>
#include <Common/quoteString.h>
#include <Core/PostgreSQL/Utils.h>
#include <common/FnTraits.h>
namespace DB
@ -41,7 +42,7 @@ std::unordered_set<std::string> fetchPostgreSQLTablesList(T & tx, const String &
}
static DataTypePtr convertPostgreSQLDataType(String & type, const std::function<void()> & recheck_array, bool is_nullable = false, uint16_t dimensions = 0)
static DataTypePtr convertPostgreSQLDataType(String & type, Fn<void()> auto && recheck_array, bool is_nullable = false, uint16_t dimensions = 0)
{
DataTypePtr res;
bool is_array = false;
@ -290,12 +291,20 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(
pqxx::ReplicationTransaction & tx, const String & postgres_table_name, bool use_nulls,
bool with_primary_key, bool with_replica_identity_index);
template
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
pqxx::nontransaction & tx, const String & postgres_table_name, bool use_nulls,
bool with_primary_key, bool with_replica_identity_index);
template
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::work & tx, const String & postgres_schema);
template
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::ReadTransaction & tx, const String & postgres_schema);
template
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::nontransaction & tx, const String & postgres_schema);
}
#endif

View File

@ -6,6 +6,7 @@
#include <Disks/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/WriteIndirectBufferFromRemoteFS.h>
#include <common/logger_useful.h>
#include <common/FnTraits.h>
namespace DB
@ -36,7 +37,7 @@ public:
chunks.back().push_back(path.data());
}
void removePaths(const std::function<void(Chunk &&)> & remove_chunk_func)
void removePaths(Fn<void(Chunk &&)> auto && remove_chunk_func)
{
for (auto & chunk : chunks)
remove_chunk_func(std::move(chunk));

View File

@ -10,6 +10,7 @@
#include <boost/algorithm/string.hpp>
#include <common/unit.h>
#include <common/FnTraits.h>
#include <Common/checkStackSize.h>
#include <Common/createHardLink.h>
@ -74,7 +75,7 @@ public:
chunks.back().push_back(obj);
}
void removePaths(const std::function<void(Chunk &&)> & remove_chunk_func)
void removePaths(Fn<void(Chunk &&)> auto && remove_chunk_func)
{
for (auto & chunk : chunks)
remove_chunk_func(std::move(chunk));

View File

@ -1,5 +1,4 @@
#include <Interpreters/Cluster.h>
#include <common/SimpleCache.h>
#include <Common/DNSResolver.h>
#include <Common/escapeForFileName.h>
#include <Common/isLocalAddress.h>