Merge remote-tracking branch 'origin/master' into fixes-clang-12-cmake-3.19

This commit is contained in:
Yatsishin Ilya 2021-03-11 09:44:30 +03:00
commit abbf4b01c2
322 changed files with 28185 additions and 21219 deletions

View File

@ -173,10 +173,12 @@ Pool::Entry Pool::tryGet()
Connection * connection_ptr = *connection_it;
/// Fixme: There is a race condition here b/c we do not synchronize with Pool::Entry's copy-assignment operator
if (connection_ptr->ref_count == 0)
{
{
Entry res(connection_ptr, this);
if (res.tryForceConnected()) /// Tries to reestablish connection as well
return res;
}
logger.debug("(%s): Idle connection to MySQL server cannot be recovered, dropping it.", getDescription());

View File

@ -32,6 +32,7 @@ endif()
set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1)
add_subdirectory (abseil-cpp-cmake)
add_subdirectory (antlr4-runtime-cmake)
add_subdirectory (boost-cmake)
add_subdirectory (cctz-cmake)

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 9a0d78de4b90546368d954b6434f0e9a823e8d80
Subproject commit ff9049bcc8ea6a02276ccdc8629d764e9e5de853

View File

@ -0,0 +1,18 @@
set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp")
if(NOT EXISTS "${ABSL_ROOT_DIR}/CMakeLists.txt")
message(FATAL_ERROR " submodule third_party/abseil-cpp is missing. To fix try run: \n git submodule update --init --recursive")
endif()
add_subdirectory("${ABSL_ROOT_DIR}" "${ClickHouse_BINARY_DIR}/contrib/abseil-cpp")
add_library(abseil_swiss_tables INTERFACE)
target_link_libraries(abseil_swiss_tables INTERFACE
absl::flat_hash_map
absl::flat_hash_set
)
get_target_property(FLAT_HASH_MAP_INCLUDE_DIR absl::flat_hash_map INTERFACE_INCLUDE_DIRECTORIES)
target_include_directories (abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_MAP_INCLUDE_DIR})
get_target_property(FLAT_HASH_SET_INCLUDE_DIR absl::flat_hash_set INTERFACE_INCLUDE_DIRECTORIES)
target_include_directories (abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_SET_INCLUDE_DIR})

View File

@ -39,11 +39,6 @@ set(_gRPC_SSL_LIBRARIES ${OPENSSL_LIBRARIES})
# Use abseil-cpp from ClickHouse contrib, not from gRPC third_party.
set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE)
set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp")
if(NOT EXISTS "${ABSL_ROOT_DIR}/CMakeLists.txt")
message(FATAL_ERROR " grpc: submodule third_party/abseil-cpp is missing. To fix try run: \n git submodule update --init --recursive")
endif()
add_subdirectory("${ABSL_ROOT_DIR}" "${ClickHouse_BINARY_DIR}/contrib/abseil-cpp")
# Choose to build static or shared library for c-ares.
if (MAKE_STATIC_LIBRARIES)

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit fbaaba4a02e29987b8c584747a496c79528f125f
Subproject commit c55b91f394efa9c238c33957682501681ef9b716

View File

@ -151,6 +151,7 @@ function clone_submodules
cd "$FASTTEST_SOURCE"
SUBMODULES_TO_UPDATE=(
contrib/abseil-cpp
contrib/antlr4-runtime
contrib/boost
contrib/zlib-ng

View File

@ -81,8 +81,8 @@ clickhouse-client --query "SHOW TABLES FROM test"
./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" && echo "OK" > /test_output/script_exit_code.txt || echo "FAIL" > /test_output/script_exit_code.txt
stop
# TODO remove me when persistent snapshots will be ready
rm -fr /var/lib/clickhouse/coordination ||:
start
clickhouse-client --query "SELECT 'Server successfuly started'" > /test_output/alive_check.txt || echo 'Server failed to start' > /test_output/alive_check.txt
tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||:

View File

@ -73,7 +73,6 @@ Clusters are set like this:
``` xml
<remote_servers>
<logs>
<shard>
<!-- Inter-server per-cluster secret for Distributed queries
default: no secret (no authentication will be performed)
@ -85,7 +84,7 @@ Clusters are set like this:
be used as current user for the query.
-->
<!-- <secret></secret> -->
<shard>
<!-- Optional. Shard weight when writing data. Default: 1. -->
<weight>1</weight>
<!-- Optional. Whether to write data to just one of the replicas. Default: false (write data to all replicas). -->

View File

@ -45,7 +45,7 @@ Restrictions on the «maximum amount of something» can take the value 0, which
用于在单个服务器上运行用户查询的最大RAM量。
默认值定义在 [设置。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L244). 默认情况下,额不受限制 (`max_memory_usage_for_user = 0`).
默认值定义在 [Settings.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L244). 默认情况下,额不受限制 (`max_memory_usage_for_user = 0`).
另请参阅说明 [max_memory_usage](#settings_max_memory_usage).
@ -53,7 +53,7 @@ Restrictions on the «maximum amount of something» can take the value 0, which
用于在单个服务器上运行所有查询的最大RAM数量。
默认值定义在 [设置。h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L245). 默认情况下,额不受限制 (`max_memory_usage_for_all_queries = 0`).
默认值定义在 [Settings.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Interpreters/Settings.h#L245). 默认情况下,额不受限制 (`max_memory_usage_for_all_queries = 0`).
另请参阅说明 [max_memory_usage](#settings_max_memory_usage).

View File

@ -715,7 +715,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
config().getString("path", ""),
std::move(main_config_zk_node_cache),
main_config_zk_changed_event,
[&](ConfigurationPtr config)
[&](ConfigurationPtr config, bool initial_loading)
{
Settings::checkNoSettingNamesAtTopLevel(*config, config_path);
@ -765,14 +765,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (config->has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop"));
if (!initial_loading)
{
/// We do not load ZooKeeper configuration on the first config loading
/// because TestKeeper server is not started yet.
if (config->has("zookeeper"))
global_context->reloadZooKeeperIfChanged(config);
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
}
global_context->updateStorageConfiguration(*config);
},
/* already_loaded = */ true);
/* already_loaded = */ false); /// Reload it right now (initial loading)
auto & access_control = global_context->getAccessControlManager();
if (config().has("custom_settings_prefixes"))

View File

@ -518,7 +518,7 @@ void UsersConfigAccessStorage::load(
preprocessed_dir,
zkutil::ZooKeeperNodeCache(get_zookeeper_function),
std::make_shared<Poco::Event>(),
[&](Poco::AutoPtr<Poco::Util::AbstractConfiguration> new_config)
[&](Poco::AutoPtr<Poco::Util::AbstractConfiguration> new_config, bool /*initial_loading*/)
{
parseFromConfig(*new_config);
Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path);

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -553,6 +554,11 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char *
return read_buffer.position();
}
const char * ColumnAggregateFunction::skipSerializedInArena(const char *) const
{
throw Exception("Method skipSerializedInArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void ColumnAggregateFunction::popBack(size_t n)
{
size_t size = data.size();

View File

@ -155,6 +155,8 @@ public:
const char * deserializeAndInsertFromArena(const char * src_arena) override;
const char * skipSerializedInArena(const char *) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;

View File

@ -239,6 +239,16 @@ const char * ColumnArray::deserializeAndInsertFromArena(const char * pos)
return pos;
}
const char * ColumnArray::skipSerializedInArena(const char * pos) const
{
size_t array_size = unalignedLoad<size_t>(pos);
pos += sizeof(array_size);
for (size_t i = 0; i < array_size; ++i)
pos = getData().skipSerializedInArena(pos);
return pos;
}
void ColumnArray::updateHashWithValue(size_t n, SipHash & hash) const
{

View File

@ -61,6 +61,7 @@ public:
void insertData(const char * pos, size_t length) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;

View File

@ -85,6 +85,7 @@ public:
void popBack(size_t) override { throwMustBeDecompressed(); }
StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override { throwMustBeDecompressed(); }
const char * deserializeAndInsertFromArena(const char *) override { throwMustBeDecompressed(); }
const char * skipSerializedInArena(const char *) const override { throwMustBeDecompressed(); }
void updateHashWithValue(size_t, SipHash &) const override { throwMustBeDecompressed(); }
void updateWeakHash32(WeakHash32 &) const override { throwMustBeDecompressed(); }
void updateHashFast(SipHash &) const override { throwMustBeDecompressed(); }

View File

@ -163,6 +163,11 @@ public:
return res;
}
const char * skipSerializedInArena(const char * pos) const override
{
return data->skipSerializedInArena(pos);
}
void updateHashWithValue(size_t, SipHash & hash) const override
{
data->updateHashWithValue(0, hash);

View File

@ -79,6 +79,12 @@ const char * ColumnDecimal<T>::deserializeAndInsertFromArena(const char * pos)
return pos + sizeof(T);
}
template <typename T>
const char * ColumnDecimal<T>::skipSerializedInArena(const char * pos) const
{
return pos + sizeof(T);
}
template <typename T>
UInt64 ColumnDecimal<T>::get64([[maybe_unused]] size_t n) const
{

View File

@ -129,6 +129,7 @@ public:
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;

View File

@ -100,6 +100,11 @@ const char * ColumnFixedString::deserializeAndInsertFromArena(const char * pos)
return pos + n;
}
const char * ColumnFixedString::skipSerializedInArena(const char * pos) const
{
return pos + n;
}
void ColumnFixedString::updateHashWithValue(size_t index, SipHash & hash) const
{
hash.update(reinterpret_cast<const char *>(&chars[n * index]), n);

View File

@ -112,6 +112,8 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t index, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;

View File

@ -98,6 +98,11 @@ public:
throw Exception("Cannot deserialize to " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
const char * skipSerializedInArena(const char*) const override
{
throw Exception("Cannot skip serialized " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void updateHashWithValue(size_t, SipHash &) const override
{
throw Exception("updateHashWithValue is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -247,6 +247,11 @@ const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * po
return new_pos;
}
const char * ColumnLowCardinality::skipSerializedInArena(const char * pos) const
{
return getDictionary().skipSerializedInArena(pos);
}
void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const
{
auto s = size();

View File

@ -94,6 +94,8 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override
{
return getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash);

View File

@ -116,6 +116,11 @@ const char * ColumnMap::deserializeAndInsertFromArena(const char * pos)
return nested->deserializeAndInsertFromArena(pos);
}
const char * ColumnMap::skipSerializedInArena(const char * pos) const
{
return nested->skipSerializedInArena(pos);
}
void ColumnMap::updateHashWithValue(size_t n, SipHash & hash) const
{
nested->updateHashWithValue(n, hash);

View File

@ -58,6 +58,7 @@ public:
void popBack(size_t n) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;

View File

@ -152,6 +152,17 @@ const char * ColumnNullable::deserializeAndInsertFromArena(const char * pos)
return pos;
}
const char * ColumnNullable::skipSerializedInArena(const char * pos) const
{
UInt8 val = unalignedLoad<UInt8>(pos);
pos += sizeof(val);
if (val == 0)
return getNestedColumn().skipSerializedInArena(pos);
return pos;
}
void ColumnNullable::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const ColumnNullable & nullable_col = assert_cast<const ColumnNullable &>(src);

View File

@ -71,6 +71,7 @@ public:
void insertData(const char * pos, size_t length) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override;
void insertFrom(const IColumn & src, size_t n) override;

View File

@ -237,6 +237,12 @@ const char * ColumnString::deserializeAndInsertFromArena(const char * pos)
return pos + string_size;
}
const char * ColumnString::skipSerializedInArena(const char * pos) const
{
const size_t string_size = unalignedLoad<size_t>(pos);
pos += sizeof(string_size);
return pos + string_size;
}
ColumnPtr ColumnString::index(const IColumn & indexes, size_t limit) const
{

View File

@ -189,6 +189,8 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override
{
size_t string_size = sizeAt(n);

View File

@ -180,6 +180,14 @@ const char * ColumnTuple::deserializeAndInsertFromArena(const char * pos)
return pos;
}
const char * ColumnTuple::skipSerializedInArena(const char * pos) const
{
for (const auto & column : columns)
pos = column->skipSerializedInArena(pos);
return pos;
}
void ColumnTuple::updateHashWithValue(size_t n, SipHash & hash) const
{
for (const auto & column : columns)

View File

@ -61,6 +61,7 @@ public:
void popBack(size_t n) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void updateHashFast(SipHash & hash) const override;

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
}
/** Stores another column with unique values
@ -78,6 +79,7 @@ public:
bool getBool(size_t n) const override { return getNestedColumn()->getBool(n); }
bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); }
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash_func) const override
{
return getNestedColumn()->updateHashWithValue(n, hash_func);
@ -373,6 +375,12 @@ size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char
return uniqueInsertData(pos, string_size - 1);
}
template <typename ColumnType>
const char * ColumnUnique<ColumnType>::skipSerializedInArena(const char *) const
{
throw Exception("Method skipSerializedInArena is not supported for " + this->getName(), ErrorCodes::NOT_IMPLEMENTED);
}
template <typename ColumnType>
int ColumnUnique<ColumnType>::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const
{

View File

@ -50,6 +50,12 @@ const char * ColumnVector<T>::deserializeAndInsertFromArena(const char * pos)
return pos + sizeof(T);
}
template <typename T>
const char * ColumnVector<T>::skipSerializedInArena(const char * pos) const
{
return pos + sizeof(T);
}
template <typename T>
void ColumnVector<T>::updateHashWithValue(size_t n, SipHash & hash) const
{

View File

@ -154,6 +154,8 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override;
const char * skipSerializedInArena(const char * pos) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;

View File

@ -207,6 +207,10 @@ public:
/// Returns pointer to the position after the read data.
virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
/// Skip previously serialized value that was serialized using IColumn::serializeValueIntoArena method.
/// Returns a pointer to the position after the deserialized data.
virtual const char * skipSerializedInArena(const char *) const = 0;
/// Update state of hash function with value of n-th element.
/// On subsequent calls of this method for sequence of column values of arbitrary types,
/// passed bytes to hash must identify sequence of values unambiguously.

View File

@ -67,6 +67,11 @@ public:
return pos;
}
const char * skipSerializedInArena(const char * pos) const override
{
return pos;
}
void updateHashWithValue(size_t /*n*/, SipHash & /*hash*/) const override
{
}

View File

@ -27,7 +27,7 @@ ConfigReloader::ConfigReloader(
, updater(std::move(updater_))
{
if (!already_loaded)
reloadIfNewer(/* force = */ true, /* throw_on_error = */ true, /* fallback_to_preprocessed = */ true);
reloadIfNewer(/* force = */ true, /* throw_on_error = */ true, /* fallback_to_preprocessed = */ true, /* initial_loading = */ true);
}
@ -66,7 +66,7 @@ void ConfigReloader::run()
if (quit)
return;
reloadIfNewer(zk_changed, /* throw_on_error = */ false, /* fallback_to_preprocessed = */ false);
reloadIfNewer(zk_changed, /* throw_on_error = */ false, /* fallback_to_preprocessed = */ false, /* initial_loading = */ false);
}
catch (...)
{
@ -76,7 +76,7 @@ void ConfigReloader::run()
}
}
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed)
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading)
{
std::lock_guard lock(reload_mutex);
@ -131,7 +131,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
try
{
updater(loaded_config.configuration);
updater(loaded_config.configuration, initial_loading);
}
catch (...)
{

View File

@ -27,7 +27,7 @@ class Context;
class ConfigReloader
{
public:
using Updater = std::function<void(ConfigurationPtr)>;
using Updater = std::function<void(ConfigurationPtr, bool)>;
/** include_from_path is usually /etc/metrika.xml (i.e. value of <include_from> tag)
*/
@ -46,12 +46,12 @@ public:
void start();
/// Reload immediately. For SYSTEM RELOAD CONFIG query.
void reload() { reloadIfNewer(/* force */ true, /* throw_on_error */ true, /* fallback_to_preprocessed */ false); }
void reload() { reloadIfNewer(/* force */ true, /* throw_on_error */ true, /* fallback_to_preprocessed */ false, /* initial_loading = */ false); }
private:
void run();
void reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed);
void reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed, bool initial_loading);
struct FileWithTimestamp;

View File

@ -541,6 +541,7 @@
M(572, TOO_MANY_QUERY_PLAN_OPTIMIZATIONS) \
M(573, EPOLL_ERROR) \
M(574, DISTRIBUTED_TOO_MANY_PENDING_BYTES) \
M(575, UNKNOWN_SNAPSHOT) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -1,8 +1,9 @@
#pragma once
#include <common/types.h>
#include <Common/UInt128.h>
#include <common/unaligned.h>
#include <common/StringRef.h>
#include <Common/UInt128.h>
#include <type_traits>
@ -178,13 +179,19 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) <= sizeof(UInt64)), T> k
}
template <typename T>
inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> key)
static constexpr bool UseDefaultHashForBigInts =
std::is_same_v<T, DB::Int128> ||
std::is_same_v<T, DB::UInt128> ||
(is_big_int_v<T> && sizeof(T) == 32);
template <typename T>
inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64) && UseDefaultHashForBigInts<T>), T> key)
{
if constexpr (std::is_same_v<T, DB::Int128>)
{
return intHash64(static_cast<UInt64>(key) ^ static_cast<UInt64>(key >> 64));
}
if constexpr (std::is_same_v<T, DB::UInt128>)
else if constexpr (std::is_same_v<T, DB::UInt128>)
{
return intHash64(key.low ^ key.high);
}
@ -195,6 +202,8 @@ inline size_t DefaultHash64(std::enable_if_t<(sizeof(T) > sizeof(UInt64)), T> ke
static_cast<UInt64>(key >> 128) ^
static_cast<UInt64>(key >> 256));
}
assert(false);
__builtin_unreachable();
}
@ -341,6 +350,11 @@ struct IntHash32
}
else if constexpr (sizeof(T) <= sizeof(UInt64))
return intHash32<salt>(key);
assert(false);
__builtin_unreachable();
}
};
template <>
struct DefaultHash<StringRef> : public StringRefHash {};

View File

@ -77,7 +77,7 @@ struct LRUHashMapCellNodeTraits
static void set_previous(node * __restrict ptr, node * __restrict prev) { ptr->prev = prev; }
};
template <typename TKey, typename TValue, typename Hash, bool save_hash_in_cells>
template <typename TKey, typename TValue, typename Disposer, typename Hash, bool save_hash_in_cells>
class LRUHashMapImpl :
private HashMapTable<
TKey,
@ -108,24 +108,33 @@ public:
boost::intrusive::value_traits<LRUHashMapCellIntrusiveValueTraits>,
boost::intrusive::constant_time_size<false>>;
using LookupResult = typename Base::LookupResult;
using ConstLookupResult = typename Base::ConstLookupResult;
using iterator = typename LRUList::iterator;
using const_iterator = typename LRUList::const_iterator;
using reverse_iterator = typename LRUList::reverse_iterator;
using const_reverse_iterator = typename LRUList::const_reverse_iterator;
LRUHashMapImpl(size_t max_size_, bool preallocate_max_size_in_hash_map = false)
explicit LRUHashMapImpl(size_t max_size_, bool preallocate_max_size_in_hash_map = false, Disposer disposer_ = Disposer())
: Base(preallocate_max_size_in_hash_map ? max_size_ : 32)
, max_size(max_size_)
, disposer(std::move(disposer_))
{
assert(max_size > 0);
}
std::pair<Cell *, bool> insert(const Key & key, const Value & value)
~LRUHashMapImpl()
{
clear();
}
std::pair<Cell *, bool> ALWAYS_INLINE insert(const Key & key, const Value & value)
{
return emplace(key, value);
}
std::pair<Cell *, bool> insert(const Key & key, Value && value)
std::pair<Cell *, bool> ALWAYS_INLINE insert(const Key & key, Value && value)
{
return emplace(key, std::move(value));
}
@ -147,15 +156,16 @@ public:
if (size() == max_size)
{
/// Erase least recently used element from front of the list
Cell & node = lru_list.front();
Cell copy_node = lru_list.front();
const Key & element_to_remove_key = node.getKey();
size_t key_hash = node.getHash(*this);
const Key & element_to_remove_key = copy_node.getKey();
lru_list.pop_front();
[[maybe_unused]] bool erased = Base::erase(element_to_remove_key, key_hash);
[[maybe_unused]] bool erased = Base::erase(element_to_remove_key);
assert(erased);
disposer(element_to_remove_key, copy_node.getMapped());
}
[[maybe_unused]] bool inserted;
@ -174,46 +184,64 @@ public:
return std::make_pair(it, true);
}
using Base::find;
Value & get(const Key & key)
LookupResult ALWAYS_INLINE find(const Key & key)
{
auto it = Base::find(key);
assert(it);
Value & value = it->getMapped();
if (!it)
return nullptr;
/// Put cell to the end of lru list
lru_list.splice(lru_list.end(), lru_list, lru_list.iterator_to(*it));
return value;
return it;
}
const Value & get(const Key & key) const
ConstLookupResult ALWAYS_INLINE find(const Key & key) const
{
return const_cast<std::decay_t<decltype(*this)> *>(this)->find(key);
}
Value & ALWAYS_INLINE get(const Key & key)
{
auto it = find(key);
assert(it);
return it->getMapped();
}
const Value & ALWAYS_INLINE get(const Key & key) const
{
return const_cast<std::decay_t<decltype(*this)> *>(this)->get(key);
}
bool contains(const Key & key) const
bool ALWAYS_INLINE contains(const Key & key) const
{
return Base::has(key);
return find(key) != nullptr;
}
bool erase(const Key & key)
bool ALWAYS_INLINE erase(const Key & key)
{
auto hash = Base::hash(key);
auto it = Base::find(key, hash);
auto key_hash = Base::hash(key);
auto it = Base::find(key, key_hash);
if (!it)
return false;
lru_list.erase(lru_list.iterator_to(*it));
return Base::erase(key, hash);
Cell copy_node = *it;
Base::erase(key, key_hash);
disposer(copy_node.getKey(), copy_node.getMapped());
return true;
}
void clear()
void ALWAYS_INLINE clear()
{
for (auto & cell : lru_list)
disposer(cell.getKey(), cell.getMapped());
lru_list.clear();
Base::clear();
}
@ -222,6 +250,10 @@ public:
size_t getMaxSize() const { return max_size; }
size_t getSizeInBytes() const { return Base::getBufferSizeInBytes(); }
using Base::hash;
iterator begin() { return lru_list.begin(); }
const_iterator begin() const { return lru_list.cbegin(); }
iterator end() { return lru_list.end(); }
@ -235,10 +267,17 @@ public:
private:
size_t max_size;
LRUList lru_list;
Disposer disposer;
};
template <typename Key, typename Value, typename Hash = DefaultHash<Key>>
using LRUHashMap = LRUHashMapImpl<Key, Value, Hash, false>;
template <typename Key, typename Mapped>
struct DefaultCellDisposer
{
void operator()(const Key &, const Mapped &) const {}
};
template <typename Key, typename Value, typename Hash = DefaultHash<Key>>
using LRUHashMapWithSavedHash = LRUHashMapImpl<Key, Value, Hash, true>;
template <typename Key, typename Value, typename Disposer = DefaultCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
using LRUHashMap = LRUHashMapImpl<Key, Value, Disposer, Hash, false>;
template <typename Key, typename Value, typename Disposer = DefaultCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
using LRUHashMapWithSavedHash = LRUHashMapImpl<Key, Value, Disposer, Hash, true>;

View File

@ -117,8 +117,11 @@ protected:
template <typename ... TAllocatorParams>
void alloc(size_t bytes, TAllocatorParams &&... allocator_params)
{
c_start = c_end = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...)) + pad_left;
c_end_of_storage = c_start + bytes - pad_right - pad_left;
char * allocated = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...));
c_start = allocated + pad_left;
c_end = c_start;
c_end_of_storage = allocated + bytes - pad_right;
if (pad_left)
memset(c_start - ELEMENT_SIZE, 0, ELEMENT_SIZE);
@ -147,12 +150,12 @@ protected:
ptrdiff_t end_diff = c_end - c_start;
c_start = reinterpret_cast<char *>(
TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward<TAllocatorParams>(allocator_params)...))
+ pad_left;
char * allocated = reinterpret_cast<char *>(
TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward<TAllocatorParams>(allocator_params)...));
c_start = allocated + pad_left;
c_end = c_start + end_diff;
c_end_of_storage = c_start + bytes - pad_right - pad_left;
c_end_of_storage = allocated + bytes - pad_right;
}
bool isInitialized() const
@ -318,11 +321,9 @@ protected:
T * t_start() { return reinterpret_cast<T *>(this->c_start); }
T * t_end() { return reinterpret_cast<T *>(this->c_end); }
T * t_end_of_storage() { return reinterpret_cast<T *>(this->c_end_of_storage); }
const T * t_start() const { return reinterpret_cast<const T *>(this->c_start); }
const T * t_end() const { return reinterpret_cast<const T *>(this->c_end); }
const T * t_end_of_storage() const { return reinterpret_cast<const T *>(this->c_end_of_storage); }
public:
using value_type = T;
@ -334,7 +335,7 @@ public:
using const_iterator = const T *;
PODArray() {}
PODArray() = default;
PODArray(size_t n)
{
@ -430,7 +431,7 @@ public:
template <typename U, typename ... TAllocatorParams>
void push_back(U && x, TAllocatorParams &&... allocator_params)
{
if (unlikely(this->c_end == this->c_end_of_storage))
if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
this->reserveForNextSize(std::forward<TAllocatorParams>(allocator_params)...);
new (t_end()) T(std::forward<U>(x));
@ -443,7 +444,7 @@ public:
template <typename... Args>
void emplace_back(Args &&... args)
{
if (unlikely(this->c_end == this->c_end_of_storage))
if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
this->reserveForNextSize();
new (t_end()) T(std::forward<Args>(args)...);

View File

@ -114,7 +114,6 @@ bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds)
while (timeout_in_seconds != 0)
{
int waitpid_res = waitpid(pid, &status, WNOHANG);
bool process_terminated_normally = (waitpid_res == pid);
if (process_terminated_normally)
@ -271,8 +270,11 @@ int ShellCommand::tryWait()
LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid);
int status = 0;
if (-1 == waitpid(pid, &status, 0))
while (waitpid(pid, &status, 0) < 0)
{
if (errno != EINTR)
throwFromErrno("Cannot waitpid", ErrorCodes::CANNOT_WAITPID);
}
LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);

View File

@ -3,7 +3,6 @@
namespace Coordination
{
void write(size_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);

View File

@ -66,3 +66,29 @@ TEST(Common, PODNoOverallocation)
EXPECT_EQ(capacities, (std::vector<size_t>{4065, 8161, 16353, 32737, 65505, 131041, 262113, 524257, 1048545}));
}
template <size_t size>
struct ItemWithSize
{
char v[size] {};
};
TEST(Common, PODInsertElementSizeNotMultipleOfLeftPadding)
{
using ItemWith24Size = ItemWithSize<24>;
PaddedPODArray<ItemWith24Size> arr1_initially_empty;
size_t items_to_insert_size = 120000;
for (size_t test = 0; test < items_to_insert_size; ++test)
arr1_initially_empty.emplace_back();
EXPECT_EQ(arr1_initially_empty.size(), items_to_insert_size);
PaddedPODArray<ItemWith24Size> arr2_initially_nonempty;
for (size_t test = 0; test < items_to_insert_size; ++test)
arr2_initially_nonempty.emplace_back();
EXPECT_EQ(arr1_initially_empty.size(), items_to_insert_size);
}

View File

@ -98,6 +98,8 @@ public:
if (sync)
plain_buf.sync();
else
plain_buf.next();
return result;
}
@ -143,6 +145,7 @@ private:
struct ChangelogReadResult
{
size_t entries_read;
size_t first_read_index;
off_t last_position;
bool error;
};
@ -207,9 +210,13 @@ public:
result.entries_read += 1;
if (record.header.index < start_log_index)
{
continue;
}
auto log_entry = nuraft::cs_new<nuraft::log_entry>(record.header.term, record.blob, record.header.value_type);
if (result.first_read_index == 0)
result.first_read_index = record.header.index;
logs.emplace(record.header.index, log_entry);
index_to_offset[record.header.index] = result.last_position;
@ -256,30 +263,46 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval
}
}
void Changelog::readChangelogAndInitWriter(size_t from_log_index)
void Changelog::readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep)
{
start_index = from_log_index == 0 ? 1 : from_log_index;
size_t total_read = 0;
size_t entries_in_last = 0;
size_t incomplete_log_index = 0;
ChangelogReadResult result{};
size_t first_read_index = 0;
size_t start_to_read_from = last_commited_log_index;
if (start_to_read_from > logs_to_keep)
start_to_read_from -= logs_to_keep;
else
start_to_read_from = 1;
bool started = false;
for (const auto & [changelog_start_index, changelog_description] : existing_changelogs)
{
entries_in_last = changelog_description.to_log_index - changelog_description.from_log_index + 1;
if (changelog_description.to_log_index >= from_log_index)
if (changelog_description.to_log_index >= start_to_read_from)
{
if (!started)
{
if (changelog_description.from_log_index > start_index)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Cannot read changelog from index {}, smallest available index {}", start_index, changelog_description.from_log_index);
started = true;
if (changelog_description.from_log_index > last_commited_log_index && (changelog_description.from_log_index - last_commited_log_index) > 1)
{
LOG_ERROR(log, "Some records was lost, last committed log index {}, smallest available log index on disk {}. Hopefully will receive missing records from leader.", last_commited_log_index, changelog_description.from_log_index);
incomplete_log_index = changelog_start_index;
break;
}
else if (changelog_description.from_log_index > start_to_read_from)
LOG_WARNING(log, "Don't have required amount of reserved log records. Need to read from {}, smalled available log index on disk {}.", start_to_read_from, changelog_description.from_log_index);
}
started = true;
ChangelogReader reader(changelog_description.path);
result = reader.readChangelog(logs, from_log_index, index_to_start_pos, log);
result = reader.readChangelog(logs, start_to_read_from, index_to_start_pos, log);
if (first_read_index == 0)
first_read_index = result.first_read_index;
total_read += result.entries_read;
/// May happen after truncate, crash or simply unfinished log
@ -291,13 +314,19 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index)
}
}
if (!started && start_index != 1)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Required to read data from {}, but we don't have any active changelogs", from_log_index);
if (first_read_index != 0)
start_index = first_read_index;
else
start_index = last_commited_log_index;
if (incomplete_log_index != 0)
{
auto start_remove_from = existing_changelogs.begin();
if (started)
start_remove_from = existing_changelogs.upper_bound(incomplete_log_index);
/// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them.
for (auto itr = existing_changelogs.upper_bound(incomplete_log_index); itr != existing_changelogs.end();)
for (auto itr = start_remove_from; itr != existing_changelogs.end();)
{
LOG_WARNING(log, "Removing changelog {}, because it's goes after broken changelog entry", itr->second.path);
std::filesystem::remove(itr->second.path);

View File

@ -67,7 +67,7 @@ public:
/// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index
/// Truncate broken entries, remove files after broken entries.
void readChangelogAndInitWriter(size_t from_log_index);
void readChangelogAndInitWriter(size_t last_commited_log_index, size_t logs_to_keep);
/// Add entry to log with index. Call fsync if force_sync true.
void appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync);

View File

@ -22,14 +22,15 @@ struct Settings;
M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \
M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \
M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Lower bound of election timer (avoid too often leader elections)", 0) \
M(UInt64, reserved_log_items, 50000, "How many log items to store (don't remove during compaction)", 0) \
M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \
M(UInt64, max_stored_snapshots, 3, "How many snapshots we want to store", 0) \
M(UInt64, reserved_log_items, 10000, "How many log items to store (don't remove during compaction)", 0) \
M(UInt64, snapshot_distance, 10000, "How many log items we have to collect to write new snapshot", 0) \
M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \
M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \
M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \
M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
M(UInt64, rotate_log_storage_interval, 100000, "How many records will be stored in one log storage file", 0) \
M(UInt64, rotate_log_storage_interval, 10000, "How many records will be stored in one log storage file", 0) \
M(UInt64, snapshots_to_keep, 3, "How many compressed snapshots to keep on disk", 0) \
M(UInt64, stale_log_gap, 10000, "When node became stale and should receive snapshots from leader", 0) \
M(Bool, force_sync, true, " Call fsync on each change in RAFT changelog", 0)
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)

View File

@ -16,10 +16,10 @@ size_t NuKeeperLogStore::start_index() const
return changelog.getStartIndex();
}
void NuKeeperLogStore::init(size_t from_log_idx)
void NuKeeperLogStore::init(size_t last_commited_log_index, size_t logs_to_keep)
{
std::lock_guard lock(changelog_lock);
changelog.readChangelogAndInitWriter(from_log_idx);
changelog.readChangelogAndInitWriter(last_commited_log_index, logs_to_keep);
}
size_t NuKeeperLogStore::next_slot() const

View File

@ -14,7 +14,7 @@ class NuKeeperLogStore : public nuraft::log_store
public:
NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_);
void init(size_t from_log_idx);
void init(size_t last_commited_log_index, size_t logs_to_keep);
size_t start_index() const override;

View File

@ -22,10 +22,11 @@ NuKeeperServer::NuKeeperServer(
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
ResponsesQueue & responses_queue_)
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_)
: server_id(server_id_)
, coordination_settings(coordination_settings_)
, state_machine(nuraft::cs_new<NuKeeperStateMachine>(responses_queue_, coordination_settings))
, state_machine(nuraft::cs_new<NuKeeperStateMachine>(responses_queue_, snapshots_queue_, config.getString("test_keeper_server.snapshot_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/snapshots"), coordination_settings))
, state_manager(nuraft::cs_new<NuKeeperStateManager>(server_id, "test_keeper_server", config, coordination_settings))
, responses_queue(responses_queue_)
{
@ -34,7 +35,10 @@ NuKeeperServer::NuKeeperServer(
void NuKeeperServer::startup()
{
state_manager->loadLogStore(state_machine->last_commit_index());
state_machine->init();
state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items);
bool single_server = state_manager->getTotalServers() == 1;
nuraft::raft_params params;
@ -54,6 +58,7 @@ void NuKeeperServer::startup()
params.reserved_log_items_ = coordination_settings->reserved_log_items;
params.snapshot_distance_ = coordination_settings->snapshot_distance;
params.stale_log_gap_ = coordination_settings->stale_log_gap;
params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds();
params.auto_forwarding_ = coordination_settings->auto_forwarding;
params.auto_forwarding_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds() * 2;
@ -174,8 +179,11 @@ bool NuKeeperServer::isLeaderAlive() const
nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
{
/// Only initial record
bool empty_store = state_manager->getLogStore()->size() == 1;
size_t last_commited = state_machine->last_commit_index();
size_t next_index = state_manager->getLogStore()->next_slot();
bool commited_store = false;
if (next_index < last_commited || next_index - last_commited <= 1)
commited_store = true;
auto set_initialized = [this] ()
{
@ -188,7 +196,7 @@ nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type t
{
case nuraft::cb_func::BecomeLeader:
{
if (empty_store) /// We become leader and store is empty, ready to serve requests
if (commited_store) /// We become leader and store is empty, ready to serve requests
set_initialized();
return nuraft::cb_func::ReturnCode::Ok;
}

View File

@ -41,7 +41,8 @@ public:
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
ResponsesQueue & responses_queue_);
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_);
void startup();

View File

@ -0,0 +1,363 @@
#include <Coordination/NuKeeperSnapshotManager.h>
#include <IO/WriteHelpers.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/copyData.h>
#include <filesystem>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_FORMAT_VERSION;
extern const int UNKNOWN_SNAPSHOT;
extern const int LOGICAL_ERROR;
}
namespace
{
size_t getSnapshotPathUpToLogIdx(const String & snapshot_path)
{
std::filesystem::path path(snapshot_path);
std::string filename = path.stem();
Strings name_parts;
splitInto<'_'>(name_parts, filename);
return parse<size_t>(name_parts[1]);
}
std::string getSnapshotFileName(size_t up_to_log_idx)
{
return std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin";
}
std::string getBaseName(const String & path)
{
size_t basename_start = path.rfind('/');
return std::string{&path[basename_start + 1], path.length() - basename_start - 1};
}
String parentPath(const String & path)
{
auto rslash_pos = path.rfind('/');
if (rslash_pos > 0)
return path.substr(0, rslash_pos);
return "/";
}
void writeNode(const NuKeeperStorage::Node & node, WriteBuffer & out)
{
writeBinary(node.data, out);
/// Serialize ACL
writeBinary(node.acls.size(), out);
for (const auto & acl : node.acls)
{
writeBinary(acl.permissions, out);
writeBinary(acl.scheme, out);
writeBinary(acl.id, out);
}
writeBinary(node.is_sequental, out);
/// Serialize stat
writeBinary(node.stat.czxid, out);
writeBinary(node.stat.mzxid, out);
writeBinary(node.stat.ctime, out);
writeBinary(node.stat.mtime, out);
writeBinary(node.stat.version, out);
writeBinary(node.stat.cversion, out);
writeBinary(node.stat.aversion, out);
writeBinary(node.stat.ephemeralOwner, out);
writeBinary(node.stat.dataLength, out);
writeBinary(node.stat.numChildren, out);
writeBinary(node.stat.pzxid, out);
writeBinary(node.seq_num, out);
}
void readNode(NuKeeperStorage::Node & node, ReadBuffer & in)
{
readBinary(node.data, in);
/// Deserialize ACL
size_t acls_size;
readBinary(acls_size, in);
for (size_t i = 0; i < acls_size; ++i)
{
Coordination::ACL acl;
readBinary(acl.permissions, in);
readBinary(acl.scheme, in);
readBinary(acl.id, in);
node.acls.push_back(acl);
}
readBinary(node.is_sequental, in);
/// Deserialize stat
readBinary(node.stat.czxid, in);
readBinary(node.stat.mzxid, in);
readBinary(node.stat.ctime, in);
readBinary(node.stat.mtime, in);
readBinary(node.stat.version, in);
readBinary(node.stat.cversion, in);
readBinary(node.stat.aversion, in);
readBinary(node.stat.ephemeralOwner, in);
readBinary(node.stat.dataLength, in);
readBinary(node.stat.numChildren, in);
readBinary(node.stat.pzxid, in);
readBinary(node.seq_num, in);
}
void serializeSnapshotMetadata(const SnapshotMetadataPtr & snapshot_meta, WriteBuffer & out)
{
auto buffer = snapshot_meta->serialize();
writeVarUInt(buffer->size(), out);
out.write(reinterpret_cast<const char *>(buffer->data_begin()), buffer->size());
}
SnapshotMetadataPtr deserializeSnapshotMetadata(ReadBuffer & in)
{
size_t data_size;
readVarUInt(data_size, in);
auto buffer = nuraft::buffer::alloc(data_size);
in.readStrict(reinterpret_cast<char *>(buffer->data_begin()), data_size);
buffer->pos(0);
return SnapshotMetadata::deserialize(*buffer);
}
}
void NuKeeperStorageSnapshot::serialize(const NuKeeperStorageSnapshot & snapshot, WriteBuffer & out)
{
writeBinary(static_cast<uint8_t>(snapshot.version), out);
serializeSnapshotMetadata(snapshot.snapshot_meta, out);
writeBinary(snapshot.session_id, out);
writeBinary(snapshot.snapshot_container_size, out);
size_t counter = 0;
for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++it, ++counter)
{
const auto & path = it->key;
const auto & node = it->value;
if (static_cast<size_t>(node.stat.mzxid) > snapshot.snapshot_meta->get_last_log_idx())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to serialize node with mzxid {}, but last snapshot index {}", node.stat.mzxid, snapshot.snapshot_meta->get_last_log_idx());
writeBinary(path, out);
writeNode(node, out);
}
size_t size = snapshot.session_and_timeout.size();
writeBinary(size, out);
for (const auto & [session_id, timeout] : snapshot.session_and_timeout)
{
writeBinary(session_id, out);
writeBinary(timeout, out);
}
}
SnapshotMetadataPtr NuKeeperStorageSnapshot::deserialize(NuKeeperStorage & storage, ReadBuffer & in)
{
uint8_t version;
readBinary(version, in);
if (static_cast<SnapshotVersion>(version) > SnapshotVersion::V0)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version);
SnapshotMetadataPtr result = deserializeSnapshotMetadata(in);
int64_t session_id;
readBinary(session_id, in);
storage.zxid = result->get_last_log_idx();
storage.session_id_counter = session_id;
size_t snapshot_container_size;
readBinary(snapshot_container_size, in);
size_t current_size = 0;
while (current_size < snapshot_container_size)
{
std::string path;
readBinary(path, in);
NuKeeperStorage::Node node;
readNode(node, in);
storage.container.insertOrReplace(path, node);
if (node.stat.ephemeralOwner != 0)
storage.ephemerals[node.stat.ephemeralOwner].insert(path);
current_size++;
}
for (const auto & itr : storage.container)
{
if (itr.key != "/")
{
auto parent_path = parentPath(itr.key);
storage.container.updateValue(parent_path, [&path = itr.key] (NuKeeperStorage::Node & value) { value.children.insert(getBaseName(path)); });
}
}
size_t active_sessions_size;
readBinary(active_sessions_size, in);
size_t current_session_size = 0;
while (current_session_size < active_sessions_size)
{
int64_t active_session_id, timeout;
readBinary(active_session_id, in);
readBinary(timeout, in);
storage.addSessionID(active_session_id, timeout);
current_session_size++;
}
return result;
}
NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, size_t up_to_log_idx_)
: storage(storage_)
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
, session_id(storage->session_id_counter)
{
storage->enableSnapshotMode();
snapshot_container_size = storage->container.snapshotSize();
begin = storage->getSnapshotIteratorBegin();
session_and_timeout = storage->getActiveSessions();
}
NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_)
: storage(storage_)
, snapshot_meta(snapshot_meta_)
, session_id(storage->session_id_counter)
{
storage->enableSnapshotMode();
snapshot_container_size = storage->container.snapshotSize();
begin = storage->getSnapshotIteratorBegin();
session_and_timeout = storage->getActiveSessions();
}
NuKeeperStorageSnapshot::~NuKeeperStorageSnapshot()
{
storage->disableSnapshotMode();
}
NuKeeperSnapshotManager::NuKeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_)
: snapshots_path(snapshots_path_)
, snapshots_to_keep(snapshots_to_keep_)
{
namespace fs = std::filesystem;
if (!fs::exists(snapshots_path))
fs::create_directories(snapshots_path);
for (const auto & p : fs::directory_iterator(snapshots_path))
{
if (startsWith(p.path(), "tmp_")) /// Unfinished tmp files
{
std::filesystem::remove(p);
continue;
}
size_t snapshot_up_to = getSnapshotPathUpToLogIdx(p.path());
existing_snapshots[snapshot_up_to] = p.path();
}
removeOutdatedSnapshotsIfNeeded();
}
std::string NuKeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx)
{
ReadBufferFromNuraftBuffer reader(buffer);
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx);
auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name;
std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name;
std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name;
WriteBufferFromFile plain_buf(tmp_snapshot_path);
copyData(reader, plain_buf);
plain_buf.sync();
std::filesystem::rename(tmp_snapshot_path, new_snapshot_path);
existing_snapshots.emplace(up_to_log_idx, new_snapshot_path);
removeOutdatedSnapshotsIfNeeded();
return new_snapshot_path;
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
{
while (!existing_snapshots.empty())
{
auto latest_itr = existing_snapshots.rbegin();
try
{
return deserializeSnapshotBufferFromDisk(latest_itr->first);
}
catch (const DB::Exception &)
{
std::filesystem::remove(latest_itr->second);
existing_snapshots.erase(latest_itr->first);
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
return nullptr;
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const
{
const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx);
WriteBufferFromNuraftBuffer writer;
ReadBufferFromFile reader(snapshot_path);
copyData(reader, writer);
return writer.getBuffer();
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::serializeSnapshotToBuffer(const NuKeeperStorageSnapshot & snapshot)
{
WriteBufferFromNuraftBuffer writer;
CompressedWriteBuffer compressed_writer(writer);
NuKeeperStorageSnapshot::serialize(snapshot, compressed_writer);
compressed_writer.finalize();
return writer.getBuffer();
}
SnapshotMetadataPtr NuKeeperSnapshotManager::deserializeSnapshotFromBuffer(NuKeeperStorage * storage, nuraft::ptr<nuraft::buffer> buffer)
{
ReadBufferFromNuraftBuffer reader(buffer);
CompressedReadBuffer compressed_reader(reader);
return NuKeeperStorageSnapshot::deserialize(*storage, compressed_reader);
}
SnapshotMetadataPtr NuKeeperSnapshotManager::restoreFromLatestSnapshot(NuKeeperStorage * storage)
{
if (existing_snapshots.empty())
return nullptr;
auto buffer = deserializeLatestSnapshotBufferFromDisk();
if (!buffer)
return nullptr;
return deserializeSnapshotFromBuffer(storage, buffer);
}
void NuKeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
{
while (existing_snapshots.size() > snapshots_to_keep)
removeSnapshot(existing_snapshots.begin()->first);
}
void NuKeeperSnapshotManager::removeSnapshot(size_t log_idx)
{
auto itr = existing_snapshots.find(log_idx);
if (itr == existing_snapshots.end())
throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx);
std::filesystem::remove(itr->second);
existing_snapshots.erase(itr);
}
}

View File

@ -0,0 +1,85 @@
#pragma once
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Coordination/NuKeeperStorage.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
namespace DB
{
using SnapshotMetadata = nuraft::snapshot;
using SnapshotMetadataPtr = std::shared_ptr<SnapshotMetadata>;
enum SnapshotVersion : uint8_t
{
V0 = 0,
};
struct NuKeeperStorageSnapshot
{
public:
NuKeeperStorageSnapshot(NuKeeperStorage * storage_, size_t up_to_log_idx_);
NuKeeperStorageSnapshot(NuKeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_);
~NuKeeperStorageSnapshot();
static void serialize(const NuKeeperStorageSnapshot & snapshot, WriteBuffer & out);
static SnapshotMetadataPtr deserialize(NuKeeperStorage & storage, ReadBuffer & in);
NuKeeperStorage * storage;
SnapshotVersion version = SnapshotVersion::V0;
SnapshotMetadataPtr snapshot_meta;
int64_t session_id;
size_t snapshot_container_size;
NuKeeperStorage::Container::const_iterator begin;
SessionAndTimeout session_and_timeout;
};
using NuKeeperStorageSnapshotPtr = std::shared_ptr<NuKeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<void(NuKeeperStorageSnapshotPtr &&)>;
class NuKeeperSnapshotManager
{
public:
NuKeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_);
SnapshotMetadataPtr restoreFromLatestSnapshot(NuKeeperStorage * storage);
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const NuKeeperStorageSnapshot & snapshot);
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx);
static SnapshotMetadataPtr deserializeSnapshotFromBuffer(NuKeeperStorage * storage, nuraft::ptr<nuraft::buffer> buffer);
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const;
nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk();
void removeSnapshot(size_t log_idx);
size_t totalSnapshots() const
{
return existing_snapshots.size();
}
size_t getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}
private:
void removeOutdatedSnapshotsIfNeeded();
const std::string snapshots_path;
const size_t snapshots_to_keep;
std::map<size_t, std::string> existing_snapshots;
};
struct CreateSnapshotTask
{
NuKeeperStorageSnapshotPtr snapshot;
CreateSnapshotCallback create_snapshot;
};
}

View File

@ -3,11 +3,16 @@
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/NuKeeperStorageSerializer.h>
#include <Coordination/NuKeeperSnapshotManager.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
{
ReadBufferFromNuraftBuffer buffer(data);
@ -21,6 +26,7 @@ NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
Coordination::read(xid, buffer);
Coordination::OpNum opnum;
Coordination::read(opnum, buffer);
request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum);
@ -29,26 +35,54 @@ NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
return request_for_session;
}
nuraft::ptr<nuraft::buffer> writeResponses(NuKeeperStorage::ResponsesForSessions & responses)
{
WriteBufferFromNuraftBuffer buffer;
for (const auto & response_and_session : responses)
{
writeIntBinary(response_and_session.session_id, buffer);
response_and_session.response->write(buffer);
}
return buffer.getBuffer();
}
NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_)
NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_)
: coordination_settings(coordination_settings_)
, storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds())
, snapshot_manager(snapshots_path_, coordination_settings->snapshots_to_keep)
, responses_queue(responses_queue_)
, snapshots_queue(snapshots_queue_)
, last_committed_idx(0)
, log(&Poco::Logger::get("NuKeeperStateMachine"))
{
LOG_DEBUG(log, "Created nukeeper state machine");
}
void NuKeeperStateMachine::init()
{
/// Do everything without mutexes, no other threads exist.
LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots());
bool loaded = false;
bool has_snapshots = snapshot_manager.totalSnapshots() != 0;
while (snapshot_manager.totalSnapshots() != 0)
{
size_t latest_log_index = snapshot_manager.getLatestSnapshotIndex();
LOG_DEBUG(log, "Trying to load state machine from snapshot up to log index {}", latest_log_index);
try
{
latest_snapshot_buf = snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index);
latest_snapshot_meta = snapshot_manager.deserializeSnapshotFromBuffer(&storage, latest_snapshot_buf);
last_committed_idx = latest_snapshot_meta->get_last_log_idx();
loaded = true;
break;
}
catch (const DB::Exception & ex)
{
LOG_WARNING(log, "Failed to load from snapshot with index {}, with error {}, will remove it from disk", latest_log_index, ex.displayText());
snapshot_manager.removeSnapshot(latest_log_index);
}
}
if (has_snapshots)
{
if (loaded)
LOG_DEBUG(log, "Loaded snapshot with last committed log index {}", last_committed_idx);
else
LOG_WARNING(log, "All snapshots broken, last committed log index {}", last_committed_idx);
}
else
{
LOG_DEBUG(log, "No existing snapshots, last committed log index {}", last_committed_idx);
}
}
nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
@ -75,7 +109,7 @@ nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, n
NuKeeperStorage::ResponsesForSessions responses_for_sessions;
{
std::lock_guard lock(storage_lock);
responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id);
responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id, log_idx);
for (auto & response_for_session : responses_for_sessions)
responses_queue.push(response_for_session);
}
@ -88,59 +122,28 @@ nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, n
bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
{
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
StorageSnapshotPtr snapshot;
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
{
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
return false;
snapshot = entry->second;
std::lock_guard lock(snapshots_lock);
if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Required to apply snapshot with last log index {}, but our last log index is {}",
s.get_last_log_idx(), latest_snapshot_meta->get_last_log_idx());
latest_snapshot_ptr = latest_snapshot_buf;
}
{
std::lock_guard lock(storage_lock);
storage = snapshot->storage;
snapshot_manager.deserializeSnapshotFromBuffer(&storage, latest_snapshot_ptr);
}
last_committed_idx = s.get_last_log_idx();
return true;
}
nuraft::ptr<nuraft::snapshot> NuKeeperStateMachine::last_snapshot()
{
// Just return the latest snapshot.
/// Just return the latest snapshot.
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.rbegin();
if (entry == snapshots.rend())
return nullptr;
return entry->second->snapshot;
}
NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::createSnapshotInternal(nuraft::snapshot & s)
{
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> ss = nuraft::snapshot::deserialize(*snp_buf);
std::lock_guard lock(storage_lock);
return std::make_shared<NuKeeperStateMachine::StorageSnapshot>(ss, storage);
}
NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nuraft::snapshot & s, nuraft::buffer & in)
{
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> ss = nuraft::snapshot::deserialize(*snp_buf);
NuKeeperStorageSerializer serializer;
ReadBufferFromNuraftBuffer reader(in);
NuKeeperStorage new_storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds());
serializer.deserialize(new_storage, reader);
return std::make_shared<StorageSnapshot>(ss, new_storage);
}
void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out)
{
NuKeeperStorageSerializer serializer;
WriteBufferFromNuraftBuffer writer;
serializer.serialize(snapshot->storage, writer);
out = writer.getBuffer();
return latest_snapshot_meta;
}
void NuKeeperStateMachine::create_snapshot(
@ -148,29 +151,53 @@ void NuKeeperStateMachine::create_snapshot(
nuraft::async_result<bool>::handler_type & when_done)
{
LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx());
auto snapshot = createSnapshotInternal(s);
{
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots[s.get_last_log_idx()] = snapshot;
size_t num = snapshots.size();
if (num > coordination_settings->max_stored_snapshots)
{
auto entry = snapshots.begin();
for (size_t i = 0; i < num - coordination_settings->max_stored_snapshots; ++i)
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
auto snapshot_meta_copy = nuraft::snapshot::deserialize(*snp_buf);
CreateSnapshotTask snapshot_task;
{
if (entry == snapshots.end())
break;
entry = snapshots.erase(entry);
}
std::lock_guard lock(storage_lock);
snapshot_task.snapshot = std::make_shared<NuKeeperStorageSnapshot>(&storage, snapshot_meta_copy);
}
}
LOG_DEBUG(log, "Created snapshot {}", s.get_last_log_idx());
nuraft::ptr<std::exception> except(nullptr);
snapshot_task.create_snapshot = [this, when_done] (NuKeeperStorageSnapshotPtr && snapshot)
{
nuraft::ptr<std::exception> exception(nullptr);
bool ret = true;
when_done(ret, except);
try
{
auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot);
auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx());
{
std::lock_guard lock(snapshots_lock);
latest_snapshot_buf = snapshot_buf;
latest_snapshot_meta = snapshot->snapshot_meta;
}
LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), result_path);
{
/// Must do it with lock (clearing elements from list)
std::lock_guard lock(storage_lock);
storage.clearGarbageAfterSnapshot();
/// Destroy snapshot with lock
snapshot.reset();
LOG_TRACE(log, "Cleared garbage after snapshot");
}
}
catch (...)
{
LOG_TRACE(log, "Exception happened during snapshot");
tryLogCurrentException(log);
ret = false;
}
when_done(ret, exception);
};
LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flash to disk", s.get_last_log_idx());
snapshots_queue.push(std::move(snapshot_task));
}
void NuKeeperStateMachine::save_logical_snp_obj(
@ -182,20 +209,32 @@ void NuKeeperStateMachine::save_logical_snp_obj(
{
LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
nuraft::ptr<nuraft::buffer> cloned_buffer;
nuraft::ptr<nuraft::snapshot> cloned_meta;
if (obj_id == 0)
{
auto new_snapshot = createSnapshotInternal(s);
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots.try_emplace(s.get_last_log_idx(), std::move(new_snapshot));
std::lock_guard lock(storage_lock);
NuKeeperStorageSnapshot snapshot(&storage, s.get_last_log_idx());
cloned_buffer = snapshot_manager.serializeSnapshotToBuffer(snapshot);
}
else
{
auto received_snapshot = readSnapshot(s, data);
std::lock_guard<std::mutex> lock(snapshots_lock);
snapshots[s.get_last_log_idx()] = std::move(received_snapshot);
cloned_buffer = nuraft::buffer::clone(data);
}
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
cloned_meta = nuraft::snapshot::deserialize(*snp_buf);
auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*cloned_buffer, s.get_last_log_idx());
{
std::lock_guard lock(snapshots_lock);
latest_snapshot_buf = cloned_buffer;
latest_snapshot_meta = cloned_meta;
}
LOG_DEBUG(log, "Created snapshot {} with path {}", s.get_last_log_idx(), result_path);
obj_id++;
}
@ -208,29 +247,20 @@ int NuKeeperStateMachine::read_logical_snp_obj(
{
LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
StorageSnapshotPtr required_snapshot;
{
std::lock_guard<std::mutex> lock(snapshots_lock);
auto entry = snapshots.find(s.get_last_log_idx());
if (entry == snapshots.end())
{
// Snapshot doesn't exist.
data_out = nullptr;
is_last_obj = true;
return 0;
}
required_snapshot = entry->second;
}
if (obj_id == 0)
{
auto new_snapshot = createSnapshotInternal(s);
writeSnapshot(new_snapshot, data_out);
data_out = nuraft::buffer::alloc(sizeof(int32_t));
nuraft::buffer_serializer bs(data_out);
bs.put_i32(0);
is_last_obj = false;
}
else
{
writeSnapshot(required_snapshot, data_out);
std::lock_guard lock(snapshots_lock);
if (s.get_last_log_idx() != latest_snapshot_meta->get_last_log_idx())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Required to apply snapshot with last log index {}, but our last log index is {}",
s.get_last_log_idx(), latest_snapshot_meta->get_last_log_idx());
data_out = nuraft::buffer::clone(*latest_snapshot_buf);
is_last_obj = true;
}
return 0;
@ -241,7 +271,7 @@ void NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForS
NuKeeperStorage::ResponsesForSessions responses;
{
std::lock_guard lock(storage_lock);
responses = storage.processRequest(request_for_session.request, request_for_session.session_id);
responses = storage.processRequest(request_for_session.request, request_for_session.session_id, std::nullopt);
}
for (const auto & response : responses)
responses_queue.push(response);

View File

@ -5,16 +5,20 @@
#include <common/logger_useful.h>
#include <Coordination/ThreadSafeQueue.h>
#include <Coordination/CoordinationSettings.h>
#include <Coordination/NuKeeperSnapshotManager.h>
namespace DB
{
using ResponsesQueue = ThreadSafeQueue<NuKeeperStorage::ResponseForSession>;
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
class NuKeeperStateMachine : public nuraft::state_machine
{
public:
NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_);
NuKeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_);
void init();
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
@ -58,39 +62,25 @@ public:
void shutdownStorage();
private:
struct StorageSnapshot
{
StorageSnapshot(const nuraft::ptr<nuraft::snapshot> & s, const NuKeeperStorage & storage_)
: snapshot(s)
, storage(storage_)
{}
nuraft::ptr<nuraft::snapshot> snapshot;
NuKeeperStorage storage;
};
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s);
StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in);
static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out);
SnapshotMetadataPtr latest_snapshot_meta = nullptr;
nuraft::ptr<nuraft::buffer> latest_snapshot_buf = nullptr;
CoordinationSettingsPtr coordination_settings;
NuKeeperStorage storage;
NuKeeperSnapshotManager snapshot_manager;
ResponsesQueue & responses_queue;
SnapshotsQueue & snapshots_queue;
/// Mutex for snapshots
std::mutex snapshots_lock;
/// Lock for storage
std::mutex storage_lock;
/// Fake snapshot storage
std::map<uint64_t, StorageSnapshotPtr> snapshots;
/// Last committed Raft log number.
std::atomic<size_t> last_committed_idx;
Poco::Logger * log;

View File

@ -64,9 +64,9 @@ NuKeeperStateManager::NuKeeperStateManager(
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
}
void NuKeeperStateManager::loadLogStore(size_t start_log_index)
void NuKeeperStateManager::loadLogStore(size_t last_commited_index, size_t logs_to_keep)
{
log_store->init(start_log_index);
log_store->init(last_commited_index, logs_to_keep);
}
void NuKeeperStateManager::flushLogStore()

View File

@ -25,7 +25,7 @@ public:
int port,
const std::string & logs_path);
void loadLogStore(size_t start_log_index);
void loadLogStore(size_t last_commited_index, size_t logs_to_keep);
void flushLogStore();

View File

@ -70,7 +70,7 @@ static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & p
NuKeeperStorage::NuKeeperStorage(int64_t tick_time_ms)
: session_expiry_queue(tick_time_ms)
{
container.emplace("/", Node());
container.insert("/", Node());
}
using Undo = std::function<void()>;
@ -124,19 +124,20 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
Coordination::ZooKeeperCreateResponse & response = dynamic_cast<Coordination::ZooKeeperCreateResponse &>(*response_ptr);
Coordination::ZooKeeperCreateRequest & request = dynamic_cast<Coordination::ZooKeeperCreateRequest &>(*zk_request);
if (container.count(request.path))
if (container.contains(request.path))
{
response.error = Coordination::Error::ZNODEEXISTS;
}
else
{
auto it = container.find(parentPath(request.path));
auto parent_path = parentPath(request.path);
auto it = container.find(parent_path);
if (it == container.end())
{
response.error = Coordination::Error::ZNONODE;
}
else if (it->second.is_ephemeral)
else if (it->value.stat.ephemeralOwner != 0)
{
response.error = Coordination::Error::ZNOCHILDRENFOREPHEMERALS;
}
@ -149,14 +150,14 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
created_node.stat.mtime = created_node.stat.ctime;
created_node.stat.numChildren = 0;
created_node.stat.dataLength = request.data.length();
created_node.stat.ephemeralOwner = request.is_ephemeral ? session_id : 0;
created_node.data = request.data;
created_node.is_ephemeral = request.is_ephemeral;
created_node.is_sequental = request.is_sequential;
std::string path_created = request.path;
if (request.is_sequential)
{
auto seq_num = it->second.seq_num;
auto seq_num = it->value.seq_num;
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
@ -165,33 +166,37 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
path_created += seq_num_str.str();
}
/// Increment sequential number even if node is not sequential
++it->second.seq_num;
response.path_created = path_created;
container.emplace(path_created, std::move(created_node));
auto child_path = getBaseName(path_created);
it->second.children.insert(child_path);
container.updateValue(parent_path, [child_path] (NuKeeperStorage::Node & parent)
{
/// Increment sequential number even if node is not sequential
++parent.seq_num;
parent.children.insert(child_path);
++parent.stat.cversion;
++parent.stat.numChildren;
});
response.path_created = path_created;
container.insert(path_created, std::move(created_node));
if (request.is_ephemeral)
ephemerals[session_id].emplace(path_created);
undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path]
undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path, child_path]
{
container.erase(path_created);
if (is_ephemeral)
ephemerals[session_id].erase(path_created);
auto & undo_parent = container.at(parent_path);
container.updateValue(parent_path, [child_path] (NuKeeperStorage::Node & undo_parent)
{
--undo_parent.stat.cversion;
--undo_parent.stat.numChildren;
--undo_parent.seq_num;
undo_parent.children.erase(child_path);
});
};
++it->second.stat.cversion;
++it->second.stat.numChildren;
response.error = Coordination::Error::ZOK;
}
}
@ -216,8 +221,8 @@ struct NuKeeperStorageGetRequest final : public NuKeeperStorageRequest
}
else
{
response.stat = it->second.stat;
response.data = it->second.data;
response.stat = it->value.stat;
response.data = it->value.data;
response.error = Coordination::Error::ZOK;
}
@ -240,39 +245,44 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
{
response.error = Coordination::Error::ZNONODE;
}
else if (request.version != -1 && request.version != it->second.stat.version)
else if (request.version != -1 && request.version != it->value.stat.version)
{
response.error = Coordination::Error::ZBADVERSION;
}
else if (it->second.stat.numChildren)
else if (it->value.stat.numChildren)
{
response.error = Coordination::Error::ZNOTEMPTY;
}
else
{
auto prev_node = it->second;
if (prev_node.is_ephemeral)
auto prev_node = it->value;
if (prev_node.stat.ephemeralOwner != 0)
ephemerals[session_id].erase(request.path);
auto child_basename = getBaseName(it->first);
auto & parent = container.at(parentPath(request.path));
auto child_basename = getBaseName(it->key);
container.updateValue(parentPath(request.path), [&child_basename] (NuKeeperStorage::Node & parent)
{
--parent.stat.numChildren;
++parent.stat.cversion;
parent.children.erase(child_basename);
});
response.error = Coordination::Error::ZOK;
container.erase(it);
container.erase(request.path);
undo = [prev_node, &container, &ephemerals, session_id, path = request.path, child_basename]
{
if (prev_node.is_ephemeral)
if (prev_node.stat.ephemeralOwner != 0)
ephemerals[session_id].emplace(path);
container.emplace(path, prev_node);
auto & undo_parent = container.at(parentPath(path));
++undo_parent.stat.numChildren;
--undo_parent.stat.cversion;
undo_parent.children.insert(child_basename);
container.insert(path, prev_node);
container.updateValue(parentPath(path), [&child_basename] (NuKeeperStorage::Node & parent)
{
++parent.stat.numChildren;
--parent.stat.cversion;
parent.children.insert(child_basename);
});
};
}
@ -297,7 +307,7 @@ struct NuKeeperStorageExistsRequest final : public NuKeeperStorageRequest
auto it = container.find(request.path);
if (it != container.end())
{
response.stat = it->second.stat;
response.stat = it->value.stat;
response.error = Coordination::Error::ZOK;
}
else
@ -324,24 +334,35 @@ struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
{
response.error = Coordination::Error::ZNONODE;
}
else if (request.version == -1 || request.version == it->second.stat.version)
else if (request.version == -1 || request.version == it->value.stat.version)
{
auto prev_node = it->second;
auto prev_node = it->value;
it->second.data = request.data;
++it->second.stat.version;
it->second.stat.mzxid = zxid;
it->second.stat.mtime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
it->second.stat.dataLength = request.data.length();
it->second.data = request.data;
++container.at(parentPath(request.path)).stat.cversion;
response.stat = it->second.stat;
auto itr = container.updateValue(request.path, [zxid, request] (NuKeeperStorage::Node & value)
{
value.data = request.data;
value.stat.version++;
value.stat.mzxid = zxid;
value.stat.mtime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
value.stat.dataLength = request.data.length();
value.data = request.data;
});
container.updateValue(parentPath(request.path), [] (NuKeeperStorage::Node & parent)
{
parent.stat.cversion++;
});
response.stat = itr->value.stat;
response.error = Coordination::Error::ZOK;
undo = [prev_node, &container, path = request.path]
{
container.at(path) = prev_node;
--container.at(parentPath(path)).stat.cversion;
container.updateValue(path, [&prev_node] (NuKeeperStorage::Node & value) { value = prev_node; });
container.updateValue(parentPath(path), [] (NuKeeperStorage::Node & parent)
{
parent.stat.cversion--;
});
};
}
else
@ -378,11 +399,11 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest
if (path_prefix.empty())
throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR);
response.names.insert(response.names.end(), it->second.children.begin(), it->second.children.end());
response.names.insert(response.names.end(), it->value.children.begin(), it->value.children.end());
std::sort(response.names.begin(), response.names.end());
response.stat = it->second.stat;
response.stat = it->value.stat;
response.error = Coordination::Error::ZOK;
}
@ -403,7 +424,7 @@ struct NuKeeperStorageCheckRequest final : public NuKeeperStorageRequest
{
response.error = Coordination::Error::ZNONODE;
}
else if (request.version != -1 && request.version != it->second.stat.version)
else if (request.version != -1 && request.version != it->value.stat.version)
{
response.error = Coordination::Error::ZBADVERSION;
}
@ -601,9 +622,17 @@ NuKeeperWrapperFactory::NuKeeperWrapperFactory()
}
NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id)
NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, std::optional<int64_t> new_last_zxid)
{
NuKeeperStorage::ResponsesForSessions results;
if (new_last_zxid)
{
if (zxid >= *new_last_zxid)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got new ZXID {} smaller or equal than current {}. It's a bug", *new_last_zxid, zxid);
zxid = *new_last_zxid;
}
session_expiry_queue.update(session_id, session_and_timeout[session_id]);
if (zk_request->getOpNum() == Coordination::OpNum::Close)
{
auto it = ephemerals.find(session_id);
@ -629,7 +658,6 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
}
else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat)
{
session_expiry_queue.update(session_id, session_and_timeout[session_id]);
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);
response->xid = zk_request->xid;
@ -639,7 +667,6 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
}
else
{
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);

View File

@ -5,6 +5,7 @@
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Coordination/SessionExpiryQueue.h>
#include <Coordination/SnapshotableHashTable.h>
#include <unordered_map>
#include <unordered_set>
#include <vector>
@ -17,17 +18,19 @@ struct NuKeeperStorageRequest;
using NuKeeperStorageRequestPtr = std::shared_ptr<NuKeeperStorageRequest>;
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
using ChildrenSet = std::unordered_set<std::string>;
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
struct NuKeeperStorageSnapshot;
class NuKeeperStorage
{
public:
int64_t session_id_counter{0};
int64_t session_id_counter{1};
struct Node
{
String data;
Coordination::ACLs acls{};
bool is_ephemeral = false;
bool is_sequental = false;
Coordination::Stat stat{};
int32_t seq_num = 0;
@ -50,10 +53,9 @@ public:
using RequestsForSessions = std::vector<RequestForSession>;
using Container = std::unordered_map<std::string, Node>;
using Container = SnapshotableHashTable<Node>;
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
using SessionAndTimeout = std::unordered_map<int64_t, long>;
using SessionIDs = std::vector<int64_t>;
using Watches = std::map<String /* path, relative of root_path */, SessionIDs>;
@ -72,9 +74,9 @@ public:
void clearDeadWatches(int64_t session_id);
int64_t getZXID()
int64_t getZXID() const
{
return zxid++;
return zxid;
}
public:
@ -88,10 +90,41 @@ public:
return result;
}
ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);
void addSessionID(int64_t session_id, int64_t session_timeout_ms)
{
session_and_timeout.emplace(session_id, session_timeout_ms);
session_expiry_queue.update(session_id, session_timeout_ms);
}
ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id, std::optional<int64_t> new_last_zxid);
void finalize();
void enableSnapshotMode()
{
container.enableSnapshotMode();
}
void disableSnapshotMode()
{
container.disableSnapshotMode();
}
Container::const_iterator getSnapshotIteratorBegin() const
{
return container.begin();
}
void clearGarbageAfterSnapshot()
{
container.clearOutdatedNodes();
}
const SessionAndTimeout & getActiveSessions() const
{
return session_and_timeout;
}
std::unordered_set<int64_t> getDeadSessions()
{
return session_expiry_queue.getExpiredSessions();

View File

@ -69,6 +69,28 @@ void NuKeeperStorageDispatcher::responseThread()
}
}
void NuKeeperStorageDispatcher::snapshotThread()
{
setThreadName("NuKeeperSnpT");
while (!shutdown_called)
{
CreateSnapshotTask task;
snapshots_queue.pop(task);
if (shutdown_called)
break;
try
{
task.create_snapshot(std::move(task.snapshot));
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(session_to_response_callback_mutex);
@ -110,7 +132,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati
coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config);
server = std::make_unique<NuKeeperServer>(myid, coordination_settings, config, responses_queue);
server = std::make_unique<NuKeeperServer>(myid, coordination_settings, config, responses_queue, snapshots_queue);
try
{
LOG_DEBUG(log, "Waiting server to initialize");
@ -129,6 +151,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati
request_thread = ThreadFromGlobalPool([this] { requestThread(); });
responses_thread = ThreadFromGlobalPool([this] { responseThread(); });
session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); });
snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); });
LOG_DEBUG(log, "Dispatcher initialized");
}
@ -149,11 +172,18 @@ void NuKeeperStorageDispatcher::shutdown()
if (session_cleaner_thread.joinable())
session_cleaner_thread.join();
/// FIXME not the best way to notify
requests_queue.push({});
if (request_thread.joinable())
request_thread.join();
responses_queue.push({});
if (responses_thread.joinable())
responses_thread.join();
snapshots_queue.push({});
if (snapshot_thread.joinable())
snapshot_thread.join();
}
if (server)
@ -161,11 +191,18 @@ void NuKeeperStorageDispatcher::shutdown()
NuKeeperStorage::RequestForSession request_for_session;
while (requests_queue.tryPop(request_for_session))
{
if (request_for_session.request)
{
auto response = request_for_session.request->makeResponse();
response->error = Coordination::Error::ZSESSIONEXPIRED;
setResponse(request_for_session.session_id, response);
}
else
{
break;
}
}
session_to_response_callback.clear();
}
catch (...)

View File

@ -30,18 +30,21 @@ private:
CoordinationSettingsPtr coordination_settings;
using RequestsQueue = ConcurrentBoundedQueue<NuKeeperStorage::RequestForSession>;
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
RequestsQueue requests_queue{1};
ResponsesQueue responses_queue;
SnapshotsQueue snapshots_queue{1};
std::atomic<bool> shutdown_called{false};
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
std::mutex session_to_response_callback_mutex;
SessionToResponseCallback session_to_response_callback;
ThreadFromGlobalPool request_thread;
ThreadFromGlobalPool responses_thread;
ThreadFromGlobalPool session_cleaner_thread;
ThreadFromGlobalPool snapshot_thread;
std::unique_ptr<NuKeeperServer> server;
@ -51,6 +54,7 @@ private:
void requestThread();
void responseThread();
void sessionCleanerTask();
void snapshotThread();
void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response);
public:

View File

@ -1,90 +0,0 @@
#include <Coordination/NuKeeperStorageSerializer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
namespace DB
{
namespace
{
void writeNode(const NuKeeperStorage::Node & node, WriteBuffer & out)
{
Coordination::write(node.data, out);
Coordination::write(node.acls, out);
Coordination::write(node.is_ephemeral, out);
Coordination::write(node.is_sequental, out);
Coordination::write(node.stat, out);
Coordination::write(node.seq_num, out);
}
void readNode(NuKeeperStorage::Node & node, ReadBuffer & in)
{
Coordination::read(node.data, in);
Coordination::read(node.acls, in);
Coordination::read(node.is_ephemeral, in);
Coordination::read(node.is_sequental, in);
Coordination::read(node.stat, in);
Coordination::read(node.seq_num, in);
}
}
void NuKeeperStorageSerializer::serialize(const NuKeeperStorage & storage, WriteBuffer & out)
{
Coordination::write(storage.zxid, out);
Coordination::write(storage.session_id_counter, out);
Coordination::write(storage.container.size(), out);
for (const auto & [path, node] : storage.container)
{
Coordination::write(path, out);
writeNode(node, out);
}
Coordination::write(storage.ephemerals.size(), out);
for (const auto & [session_id, paths] : storage.ephemerals)
{
Coordination::write(session_id, out);
Coordination::write(paths.size(), out);
for (const auto & path : paths)
Coordination::write(path, out);
}
}
void NuKeeperStorageSerializer::deserialize(NuKeeperStorage & storage, ReadBuffer & in)
{
int64_t session_id_counter, zxid;
Coordination::read(zxid, in);
Coordination::read(session_id_counter, in);
storage.zxid = zxid;
storage.session_id_counter = session_id_counter;
size_t container_size;
Coordination::read(container_size, in);
size_t current_size = 0;
while (current_size < container_size)
{
std::string path;
Coordination::read(path, in);
NuKeeperStorage::Node node;
readNode(node, in);
storage.container[path] = node;
current_size++;
}
size_t ephemerals_size;
Coordination::read(ephemerals_size, in);
while (storage.ephemerals.size() < ephemerals_size)
{
int64_t session_id;
size_t ephemerals_for_session;
Coordination::read(session_id, in);
Coordination::read(ephemerals_for_session, in);
while (storage.ephemerals[session_id].size() < ephemerals_for_session)
{
std::string ephemeral_path;
Coordination::read(ephemeral_path, in);
storage.ephemerals[session_id].emplace(ephemeral_path);
}
}
}
}

View File

@ -1,17 +0,0 @@
#pragma once
#include <Coordination/NuKeeperStorage.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
namespace DB
{
class NuKeeperStorageSerializer
{
public:
static void serialize(const NuKeeperStorage & storage, WriteBuffer & out);
static void deserialize(NuKeeperStorage & storage, ReadBuffer & in);
};
}

View File

@ -0,0 +1,197 @@
#pragma once
#include <common/StringRef.h>
#include <unordered_map>
#include <list>
#include <atomic>
namespace DB
{
template<typename V>
struct ListNode
{
std::string key;
V value;
bool active_in_map;
};
template <class V>
class SnapshotableHashTable
{
private:
using ListElem = ListNode<V>;
using List = std::list<ListElem>;
using IndexMap = std::unordered_map<StringRef, typename List::iterator, StringRefHash>;
List list;
IndexMap map;
bool snapshot_mode{false};
public:
using iterator = typename List::iterator;
using const_iterator = typename List::const_iterator;
using reverse_iterator = typename List::reverse_iterator;
using const_reverse_iterator = typename List::const_reverse_iterator;
using ValueUpdater = std::function<void(V & value)>;
bool insert(const std::string & key, const V & value)
{
auto it = map.find(key);
if (it == map.end())
{
ListElem elem{key, value, true};
auto itr = list.insert(list.end(), elem);
map.emplace(itr->key, itr);
return true;
}
return false;
}
void insertOrReplace(const std::string & key, const V & value)
{
auto it = map.find(key);
if (it == map.end())
{
ListElem elem{key, value, true};
auto itr = list.insert(list.end(), elem);
map.emplace(itr->key, itr);
}
else
{
auto list_itr = it->second;
if (snapshot_mode)
{
ListElem elem{key, value, true};
list_itr->active_in_map = false;
auto new_list_itr = list.insert(list.end(), elem);
map.erase(it);
map.emplace(new_list_itr->key, new_list_itr);
}
else
{
list_itr->value = value;
}
}
}
bool erase(const std::string & key)
{
auto it = map.find(key);
if (it == map.end())
return false;
auto list_itr = it->second;
if (snapshot_mode)
{
list_itr->active_in_map = false;
map.erase(it);
}
else
{
map.erase(it);
list.erase(list_itr);
}
return true;
}
bool contains(const std::string & key) const
{
return map.find(key) != map.end();
}
const_iterator updateValue(const std::string & key, ValueUpdater updater)
{
auto it = map.find(key);
assert(it != map.end());
if (snapshot_mode)
{
auto list_itr = it->second;
auto elem_copy = *(list_itr);
list_itr->active_in_map = false;
map.erase(it);
updater(elem_copy.value);
auto itr = list.insert(list.end(), elem_copy);
map.emplace(itr->key, itr);
return itr;
}
else
{
auto list_itr = it->second;
updater(list_itr->value);
return list_itr;
}
}
const_iterator find(const std::string & key) const
{
auto map_it = map.find(key);
if (map_it != map.end())
return map_it->second;
return list.end();
}
const V & getValue(const std::string & key) const
{
auto it = map.find(key);
assert(it != map.end());
return it->second->value;
}
void clearOutdatedNodes()
{
auto start = list.begin();
auto end = list.end();
for (auto itr = start; itr != end;)
{
if (!itr->active_in_map)
itr = list.erase(itr);
else
itr++;
}
}
void clear()
{
list.clear();
map.clear();
}
void enableSnapshotMode()
{
snapshot_mode = true;
}
void disableSnapshotMode()
{
snapshot_mode = false;
}
size_t size() const
{
return map.size();
}
size_t snapshotSize() const
{
return list.size();
}
iterator begin() { return list.begin(); }
const_iterator begin() const { return list.cbegin(); }
iterator end() { return list.end(); }
const_iterator end() const { return list.cend(); }
reverse_iterator rbegin() { return list.rbegin(); }
const_reverse_iterator rbegin() const { return list.crbegin(); }
reverse_iterator rend() { return list.rend(); }
const_reverse_iterator rend() const { return list.crend(); }
};
}

View File

@ -10,7 +10,7 @@
#include <Poco/Logger.h>
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/NuKeeperStorageSerializer.h>
#include <Coordination/NuKeeperSnapshotManager.h>
#include <Coordination/SummingStateMachine.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/LoggerWrapper.h>
@ -28,6 +28,8 @@
#include <Coordination/Changelog.h>
#include <filesystem>
#include <Coordination/SnapshotableHashTable.h>
namespace fs = std::filesystem;
struct ChangelogDirTest
{
@ -102,7 +104,7 @@ struct SimpliestRaftServer
, state_machine(nuraft::cs_new<StateMachine>())
, state_manager(nuraft::cs_new<DB::NuKeeperStateManager>(server_id, hostname, port, logs_path))
{
state_manager->loadLogStore(1);
state_manager->loadLogStore(1, 0);
nuraft::raft_params params;
params.heart_beat_interval_ = 100;
params.election_timeout_lower_bound_ = 200;
@ -195,68 +197,6 @@ TEST(CoordinationTest, TestSummingRaft1)
s1.launcher.shutdown(5);
}
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
{
DB::WriteBufferFromNuraftBuffer buf;
DB::writeIntBinary(session_id, buf);
request->write(buf);
return buf.getBuffer();
}
DB::NuKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr<nuraft::buffer> & buffer, const Coordination::ZooKeeperRequestPtr & request)
{
DB::NuKeeperStorage::ResponsesForSessions results;
DB::ReadBufferFromNuraftBuffer buf(buffer);
while (!buf.eof())
{
int64_t session_id;
DB::readIntBinary(session_id, buf);
int32_t length;
Coordination::XID xid;
int64_t zxid;
Coordination::Error err;
Coordination::read(length, buf);
Coordination::read(xid, buf);
Coordination::read(zxid, buf);
Coordination::read(err, buf);
auto response = request->makeResponse();
response->readImpl(buf);
results.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response});
}
return results;
}
TEST(CoordinationTest, TestStorageSerialization)
{
DB::NuKeeperStorage storage(500);
storage.container["/hello"] = DB::NuKeeperStorage::Node{.data="world"};
storage.container["/hello/somepath"] = DB::NuKeeperStorage::Node{.data="somedata"};
storage.session_id_counter = 5;
storage.zxid = 156;
storage.ephemerals[3] = {"/hello", "/"};
storage.ephemerals[1] = {"/hello/somepath"};
DB::WriteBufferFromOwnString buffer;
DB::NuKeeperStorageSerializer serializer;
serializer.serialize(storage, buffer);
std::string serialized = buffer.str();
EXPECT_NE(serialized.size(), 0);
DB::ReadBufferFromString read(serialized);
DB::NuKeeperStorage new_storage(500);
serializer.deserialize(new_storage, read);
EXPECT_EQ(new_storage.container.size(), 3);
EXPECT_EQ(new_storage.container["/hello"].data, "world");
EXPECT_EQ(new_storage.container["/hello/somepath"].data, "somedata");
EXPECT_EQ(new_storage.session_id_counter, 5);
EXPECT_EQ(new_storage.zxid, 156);
EXPECT_EQ(new_storage.ephemerals.size(), 2);
EXPECT_EQ(new_storage.ephemerals[3].size(), 2);
EXPECT_EQ(new_storage.ephemerals[1].size(), 1);
}
DB::LogEntryPtr getLogEntry(const std::string & s, size_t term)
{
DB::WriteBufferFromNuraftBuffer bufwriter;
@ -268,7 +208,7 @@ TEST(CoordinationTest, ChangelogTestSimple)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
auto entry = getLogEntry("hello world", 77);
changelog.append(entry);
EXPECT_EQ(changelog.next_slot(), 2);
@ -282,7 +222,7 @@ TEST(CoordinationTest, ChangelogTestFile)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
auto entry = getLogEntry("hello world", 77);
changelog.append(entry);
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin"));
@ -303,7 +243,7 @@ TEST(CoordinationTest, ChangelogReadWrite)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 1000, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
auto entry = getLogEntry("hello world", i * 10);
@ -311,7 +251,7 @@ TEST(CoordinationTest, ChangelogReadWrite)
}
EXPECT_EQ(changelog.size(), 10);
DB::NuKeeperLogStore changelog_reader("./logs", 1000, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term());
EXPECT_EQ(changelog_reader.start_index(), changelog.start_index());
@ -330,7 +270,7 @@ TEST(CoordinationTest, ChangelogWriteAt)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 1000, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
auto entry = getLogEntry("hello world", i * 10);
@ -346,7 +286,7 @@ TEST(CoordinationTest, ChangelogWriteAt)
EXPECT_EQ(changelog.next_slot(), 8);
DB::NuKeeperLogStore changelog_reader("./logs", 1000, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), changelog.size());
EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term());
@ -359,7 +299,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 7; ++i)
{
auto entry = getLogEntry("hello world", i * 10);
@ -371,7 +311,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead)
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin"));
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 7);
for (size_t i = 7; i < 10; ++i)
@ -407,7 +347,7 @@ TEST(CoordinationTest, ChangelogTestCompaction)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 3; ++i)
{
@ -448,7 +388,7 @@ TEST(CoordinationTest, ChangelogTestCompaction)
EXPECT_EQ(changelog.last_entry()->get_term(), 60);
/// And we able to read it
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(7);
changelog_reader.init(7, 0);
EXPECT_EQ(changelog_reader.size(), 1);
EXPECT_EQ(changelog_reader.start_index(), 7);
EXPECT_EQ(changelog_reader.next_slot(), 8);
@ -459,7 +399,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 100, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10);
@ -471,7 +411,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations)
auto entries = changelog.pack(1, 5);
DB::NuKeeperLogStore apply_changelog("./logs", 100, true);
apply_changelog.init(1);
apply_changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
@ -501,7 +441,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 100, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10);
@ -514,7 +454,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
ChangelogDirTest test1("./logs1");
DB::NuKeeperLogStore changelog_new("./logs1", 100, true);
changelog_new.init(1);
changelog_new.init(1, 0);
EXPECT_EQ(changelog_new.size(), 0);
changelog_new.apply_pack(5, *entries);
@ -533,7 +473,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
EXPECT_EQ(changelog_new.next_slot(), 11);
DB::NuKeeperLogStore changelog_reader("./logs1", 100, true);
changelog_reader.init(5);
changelog_reader.init(5, 0);
}
@ -541,7 +481,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
{
@ -576,7 +516,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile)
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_read("./logs", 5, true);
changelog_read.init(1);
changelog_read.init(1, 0);
EXPECT_EQ(changelog_read.size(), 7);
EXPECT_EQ(changelog_read.start_index(), 1);
EXPECT_EQ(changelog_read.next_slot(), 8);
@ -587,7 +527,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
{
@ -622,7 +562,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder)
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_read("./logs", 5, true);
changelog_read.init(1);
changelog_read.init(1, 0);
EXPECT_EQ(changelog_read.size(), 11);
EXPECT_EQ(changelog_read.start_index(), 1);
EXPECT_EQ(changelog_read.next_slot(), 12);
@ -633,7 +573,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtAllFiles)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
{
@ -672,7 +612,7 @@ TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
{
@ -691,7 +631,7 @@ TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead)
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
auto entry = getLogEntry("36_hello_world", 360);
changelog_reader.append(entry);
@ -713,7 +653,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
{
@ -733,7 +673,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
plain_buf.truncate(0);
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 90);
@ -762,7 +702,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_reader2("./logs", 5, true);
changelog_reader2.init(1);
changelog_reader2.init(1, 0);
EXPECT_EQ(changelog_reader2.size(), 11);
EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777);
}
@ -772,7 +712,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 20, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
{
@ -787,7 +727,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
plain_buf.truncate(140);
DB::NuKeeperLogStore changelog_reader("./logs", 20, true);
changelog_reader.init(1);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 2);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 450);
@ -800,7 +740,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
DB::NuKeeperLogStore changelog_reader2("./logs", 20, true);
changelog_reader2.init(1);
changelog_reader2.init(1, 0);
EXPECT_EQ(changelog_reader2.size(), 3);
EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777);
}
@ -810,7 +750,7 @@ TEST(CoordinationTest, ChangelogTestLostFiles)
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 20, true);
changelog.init(1);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
{
@ -824,10 +764,476 @@ TEST(CoordinationTest, ChangelogTestLostFiles)
fs::remove("./logs/changelog_1_20.bin");
DB::NuKeeperLogStore changelog_reader("./logs", 20, true);
EXPECT_THROW(changelog_reader.init(5), DB::Exception);
/// It should print error message, but still able to start
changelog_reader.init(5, 0);
EXPECT_FALSE(fs::exists("./logs/changelog_1_20.bin"));
EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin"));
}
fs::remove("./logs/changelog_21_40.bin");
EXPECT_THROW(changelog_reader.init(3), DB::Exception);
TEST(CoordinationTest, SnapshotableHashMapSimple)
{
DB::SnapshotableHashTable<int> hello;
EXPECT_TRUE(hello.insert("hello", 5));
EXPECT_TRUE(hello.contains("hello"));
EXPECT_EQ(hello.getValue("hello"), 5);
EXPECT_FALSE(hello.insert("hello", 145));
EXPECT_EQ(hello.getValue("hello"), 5);
hello.updateValue("hello", [](int & value) { value = 7; });
EXPECT_EQ(hello.getValue("hello"), 7);
EXPECT_EQ(hello.size(), 1);
EXPECT_TRUE(hello.erase("hello"));
EXPECT_EQ(hello.size(), 0);
}
TEST(CoordinationTest, SnapshotableHashMapTrySnapshot)
{
DB::SnapshotableHashTable<int> map_snp;
EXPECT_TRUE(map_snp.insert("/hello", 7));
EXPECT_FALSE(map_snp.insert("/hello", 145));
map_snp.enableSnapshotMode();
EXPECT_FALSE(map_snp.insert("/hello", 145));
map_snp.updateValue("/hello", [](int & value) { value = 554; });
EXPECT_EQ(map_snp.getValue("/hello"), 554);
EXPECT_EQ(map_snp.snapshotSize(), 2);
EXPECT_EQ(map_snp.size(), 1);
auto itr = map_snp.begin();
EXPECT_EQ(itr->key, "/hello");
EXPECT_EQ(itr->value, 7);
EXPECT_EQ(itr->active_in_map, false);
itr = std::next(itr);
EXPECT_EQ(itr->key, "/hello");
EXPECT_EQ(itr->value, 554);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
EXPECT_EQ(itr, map_snp.end());
for (size_t i = 0; i < 5; ++i)
{
EXPECT_TRUE(map_snp.insert("/hello" + std::to_string(i), i));
}
EXPECT_EQ(map_snp.getValue("/hello3"), 3);
EXPECT_EQ(map_snp.snapshotSize(), 7);
EXPECT_EQ(map_snp.size(), 6);
itr = std::next(map_snp.begin(), 2);
for (size_t i = 0; i < 5; ++i)
{
EXPECT_EQ(itr->key, "/hello" + std::to_string(i));
EXPECT_EQ(itr->value, i);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
}
EXPECT_TRUE(map_snp.erase("/hello3"));
EXPECT_TRUE(map_snp.erase("/hello2"));
EXPECT_EQ(map_snp.snapshotSize(), 7);
EXPECT_EQ(map_snp.size(), 4);
itr = std::next(map_snp.begin(), 2);
for (size_t i = 0; i < 5; ++i)
{
EXPECT_EQ(itr->key, "/hello" + std::to_string(i));
EXPECT_EQ(itr->value, i);
EXPECT_EQ(itr->active_in_map, i != 3 && i != 2);
itr = std::next(itr);
}
map_snp.clearOutdatedNodes();
EXPECT_EQ(map_snp.snapshotSize(), 4);
EXPECT_EQ(map_snp.size(), 4);
itr = map_snp.begin();
EXPECT_EQ(itr->key, "/hello");
EXPECT_EQ(itr->value, 554);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
EXPECT_EQ(itr->key, "/hello0");
EXPECT_EQ(itr->value, 0);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
EXPECT_EQ(itr->key, "/hello1");
EXPECT_EQ(itr->value, 1);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
EXPECT_EQ(itr->key, "/hello4");
EXPECT_EQ(itr->value, 4);
EXPECT_EQ(itr->active_in_map, true);
itr = std::next(itr);
EXPECT_EQ(itr, map_snp.end());
map_snp.disableSnapshotMode();
}
void addNode(DB::NuKeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0)
{
using Node = DB::NuKeeperStorage::Node;
Node node{};
node.data = data;
node.stat.ephemeralOwner = ephemeral_owner;
storage.container.insertOrReplace(path, node);
}
TEST(CoordinationTest, TestStorageSnapshotSimple)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
addNode(storage, "/hello", "world", 1);
addNode(storage, "/hello/somepath", "somedata", 3);
storage.session_id_counter = 5;
storage.zxid = 2;
storage.ephemerals[3] = {"/hello"};
storage.ephemerals[1] = {"/hello/somepath"};
storage.getSessionID(130);
storage.getSessionID(130);
DB::NuKeeperStorageSnapshot snapshot(&storage, 2);
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2);
EXPECT_EQ(snapshot.session_id, 7);
EXPECT_EQ(snapshot.snapshot_container_size, 3);
EXPECT_EQ(snapshot.session_and_timeout.size(), 2);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 2);
EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin"));
DB::NuKeeperStorage restored_storage(500);
auto debuf = manager.deserializeSnapshotBufferFromDisk(2);
manager.deserializeSnapshotFromBuffer(&restored_storage, debuf);
EXPECT_EQ(restored_storage.container.size(), 3);
EXPECT_EQ(restored_storage.container.getValue("/").children.size(), 1);
EXPECT_EQ(restored_storage.container.getValue("/hello").children.size(), 1);
EXPECT_EQ(restored_storage.container.getValue("/hello/somepath").children.size(), 0);
EXPECT_EQ(restored_storage.container.getValue("/").data, "");
EXPECT_EQ(restored_storage.container.getValue("/hello").data, "world");
EXPECT_EQ(restored_storage.container.getValue("/hello/somepath").data, "somedata");
EXPECT_EQ(restored_storage.session_id_counter, 7);
EXPECT_EQ(restored_storage.zxid, 2);
EXPECT_EQ(restored_storage.ephemerals.size(), 2);
EXPECT_EQ(restored_storage.ephemerals[3].size(), 1);
EXPECT_EQ(restored_storage.ephemerals[1].size(), 1);
EXPECT_EQ(restored_storage.session_and_timeout.size(), 2);
}
TEST(CoordinationTest, TestStorageSnapshotMoreWrites)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
storage.getSessionID(130);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50);
EXPECT_EQ(snapshot.snapshot_container_size, 51);
for (size_t i = 50; i < 100; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
EXPECT_EQ(storage.container.size(), 101);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50);
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin"));
DB::NuKeeperStorage restored_storage(500);
auto debuf = manager.deserializeSnapshotBufferFromDisk(50);
manager.deserializeSnapshotFromBuffer(&restored_storage, debuf);
EXPECT_EQ(restored_storage.container.size(), 51);
for (size_t i = 0; i < 50; ++i)
{
EXPECT_EQ(restored_storage.container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i));
}
}
TEST(CoordinationTest, TestStorageSnapshotManySnapshots)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
storage.getSessionID(130);
for (size_t j = 1; j <= 5; ++j)
{
for (size_t i = (j - 1) * 50; i < j * 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
DB::NuKeeperStorageSnapshot snapshot(&storage, j * 50);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, j * 50);
EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin"));
}
EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin"));
EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin"));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin"));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin"));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin"));
DB::NuKeeperStorage restored_storage(500);
manager.restoreFromLatestSnapshot(&restored_storage);
EXPECT_EQ(restored_storage.container.size(), 251);
for (size_t i = 0; i < 250; ++i)
{
EXPECT_EQ(restored_storage.container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i));
}
}
TEST(CoordinationTest, TestStorageSnapshotMode)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
{
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i));
}
for (size_t i = 0; i < 50; ++i)
{
EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).data, "wlrd_" + std::to_string(i));
}
for (size_t i = 0; i < 50; ++i)
{
if (i % 2 == 0)
storage.container.erase("/hello_" + std::to_string(i));
}
EXPECT_EQ(storage.container.size(), 26);
EXPECT_EQ(storage.container.snapshotSize(), 101);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50);
}
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin"));
EXPECT_EQ(storage.container.size(), 26);
storage.clearGarbageAfterSnapshot();
EXPECT_EQ(storage.container.snapshotSize(), 26);
for (size_t i = 0; i < 50; ++i)
{
if (i % 2 != 0)
EXPECT_EQ(storage.container.getValue("/hello_" + std::to_string(i)).data, "wlrd_" + std::to_string(i));
else
EXPECT_FALSE(storage.container.contains("/hello_" + std::to_string(i)));
}
DB::NuKeeperStorage restored_storage(500);
manager.restoreFromLatestSnapshot(&restored_storage);
for (size_t i = 0; i < 50; ++i)
{
EXPECT_EQ(restored_storage.container.getValue("/hello_" + std::to_string(i)).data, "world_" + std::to_string(i));
}
}
TEST(CoordinationTest, TestStorageSnapshotBroken)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
{
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50);
}
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin"));
/// Let's corrupt file
DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
plain_buf.truncate(34);
plain_buf.sync();
DB::NuKeeperStorage restored_storage(500);
EXPECT_THROW(manager.restoreFromLatestSnapshot(&restored_storage), DB::Exception);
}
nuraft::ptr<nuraft::buffer> getBufferFromZKRequest(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
{
DB::WriteBufferFromNuraftBuffer buf;
DB::writeIntBinary(session_id, buf);
request->write(buf);
return buf.getBuffer();
}
nuraft::ptr<nuraft::log_entry> getLogEntryFromZKRequest(size_t term, int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
{
auto buffer = getBufferFromZKRequest(session_id, request);
return nuraft::cs_new<nuraft::log_entry>(term, buffer);
}
void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, size_t total_logs)
{
using namespace Coordination;
using namespace DB;
ChangelogDirTest snapshots("./snapshots");
ChangelogDirTest logs("./logs");
ResponsesQueue queue;
SnapshotsQueue snapshots_queue{1};
auto state_machine = std::make_shared<NuKeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings);
state_machine->init();
DB::NuKeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true);
changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items);
for (size_t i = 1; i < total_logs + 1; ++i)
{
std::shared_ptr<ZooKeeperCreateRequest> request = std::make_shared<ZooKeeperCreateRequest>();
request->path = "/hello_" + std::to_string(i);
auto entry = getLogEntryFromZKRequest(0, 1, request);
changelog.append(entry);
state_machine->commit(i, changelog.entry_at(i)->get_buf());
bool snapshot_created = false;
if (i % settings->snapshot_distance == 0)
{
nuraft::snapshot s(i, 0, std::make_shared<nuraft::cluster_config>());
nuraft::async_result<bool>::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr<std::exception> &/*exception*/)
{
snapshot_created = ret;
std::cerr << "Snapshot finised\n";
};
state_machine->create_snapshot(s, when_done);
CreateSnapshotTask snapshot_task;
snapshots_queue.pop(snapshot_task);
snapshot_task.create_snapshot(std::move(snapshot_task.snapshot));
}
if (snapshot_created)
{
if (changelog.size() > settings->reserved_log_items)
{
changelog.compact(i - settings->reserved_log_items);
}
}
}
SnapshotsQueue snapshots_queue1{1};
auto restore_machine = std::make_shared<NuKeeperStateMachine>(queue, snapshots_queue1, "./snapshots", settings);
restore_machine->init();
EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance);
DB::NuKeeperLogStore restore_changelog("./logs", settings->rotate_log_storage_interval, true);
restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items);
EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs));
EXPECT_EQ(restore_changelog.next_slot(), total_logs + 1);
if (total_logs > settings->reserved_log_items + 1)
EXPECT_EQ(restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1);
else
EXPECT_EQ(restore_changelog.start_index(), 1);
for (size_t i = restore_machine->last_commit_index() + 1; i < restore_changelog.next_slot(); ++i)
{
restore_machine->commit(i, changelog.entry_at(i)->get_buf());
}
auto & source_storage = state_machine->getStorage();
auto & restored_storage = restore_machine->getStorage();
EXPECT_EQ(source_storage.container.size(), restored_storage.container.size());
for (size_t i = 1; i < total_logs + 1; ++i)
{
auto path = "/hello_" + std::to_string(i);
EXPECT_EQ(source_storage.container.getValue(path).data, restored_storage.container.getValue(path).data);
}
}
TEST(CoordinationTest, TestStateMachineAndLogStore)
{
using namespace Coordination;
using namespace DB;
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 10;
settings->rotate_log_storage_interval = 10;
testLogAndStateMachine(settings, 37);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 10;
settings->rotate_log_storage_interval = 10;
testLogAndStateMachine(settings, 11);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 10;
settings->rotate_log_storage_interval = 10;
testLogAndStateMachine(settings, 40);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 20;
settings->rotate_log_storage_interval = 30;
testLogAndStateMachine(settings, 40);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 0;
settings->rotate_log_storage_interval = 10;
testLogAndStateMachine(settings, 40);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 1;
settings->reserved_log_items = 1;
settings->rotate_log_storage_interval = 32;
testLogAndStateMachine(settings, 32);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 10;
settings->reserved_log_items = 7;
settings->rotate_log_storage_interval = 1;
testLogAndStateMachine(settings, 33);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 37;
settings->reserved_log_items = 1000;
settings->rotate_log_storage_interval = 5000;
testLogAndStateMachine(settings, 33);
}
{
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
settings->snapshot_distance = 37;
settings->reserved_log_items = 1000;
settings->rotate_log_storage_interval = 5000;
testLogAndStateMachine(settings, 45);
}
}
int main(int argc, char ** argv)

View File

@ -68,6 +68,7 @@ void MySQLClient::disconnect()
socket->close();
socket = nullptr;
connected = false;
seq = 0;
}
/// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
extern const int UNKNOWN_EXCEPTION;
extern const int LOGICAL_ERROR;
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int CANNOT_READ_ALL_DATA;
}
namespace MySQLReplication
@ -740,7 +741,7 @@ namespace MySQLReplication
switch (header)
{
case PACKET_EOF:
throw ReplicationError("Master maybe lost", ErrorCodes::UNKNOWN_EXCEPTION);
throw ReplicationError("Master maybe lost", ErrorCodes::CANNOT_READ_ALL_DATA);
case PACKET_ERR:
ERRPacket err;
err.readPayloadWithUnpacked(payload);

View File

@ -518,6 +518,7 @@ class IColumn;
M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \
M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \
M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \
// End of FORMAT_FACTORY_SETTINGS

View File

@ -366,4 +366,20 @@ private:
}
};
template <typename TLeftColumns, typename TRightColumns>
bool less(const TLeftColumns & lhs, const TRightColumns & rhs, size_t i, size_t j, const SortDescription & descr)
{
for (const auto & elem : descr)
{
size_t ind = elem.column_number;
int res = elem.direction * lhs[ind]->compareAt(i, j, *rhs[ind], elem.nulls_direction);
if (res < 0)
return true;
else if (res > 0)
return false;
}
return false;
}
}

View File

@ -108,6 +108,13 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
void DatabaseAtomic::dropTable(const Context & context, const String & table_name, bool no_delay)
{
if (auto * mv = dynamic_cast<StorageMaterializedView *>(tryGetTable(table_name, context).get()))
{
/// Remove the inner table (if any) to avoid deadlock
/// (due to attempt to execute DROP from the worker thread)
mv->dropInnerTable(no_delay, context);
}
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop;
StoragePtr table;
@ -131,10 +138,7 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam
}
if (table->storesDataOnDisk())
tryRemoveSymlink(table_name);
/// Remove the inner table (if any) to avoid deadlock
/// (due to attempt to execute DROP from the worker thread)
if (auto * mv = dynamic_cast<StorageMaterializedView *>(table.get()))
mv->dropInnerTable(no_delay, context);
/// Notify DatabaseCatalog that table was dropped. It will remove table data in background.
/// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete.
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);

View File

@ -82,18 +82,11 @@ template<typename Base>
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
{
Base::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
try
{
if (!force_attach)
materialize_thread.assertMySQLAvailable();
materialize_thread.startSynchronization();
started_up = true;
}
catch (...)
{
tryLogCurrentException(Base::log, "Cannot load MySQL nested database stored objects.");
if (!force_attach)
throw;
}
}
template<typename Base>

View File

@ -159,6 +159,9 @@ static void checkSyncUserPriv(const mysqlxx::PoolWithFailover::Entry & connectio
bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover::Entry & connection) const
{
if (binlog_file.empty())
return false;
Block logs_header {
{std::make_shared<DataTypeString>(), "Log_name"},
{std::make_shared<DataTypeUInt64>(), "File_size"}
@ -219,13 +222,8 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio
commitMetadata(std::move(fun), persistent_tmp_path, persistent_path);
}
MaterializeMetadata::MaterializeMetadata(
mysqlxx::PoolWithFailover::Entry & connection, const String & path_,
const String & database, bool & opened_transaction)
: persistent_path(path_)
MaterializeMetadata::MaterializeMetadata(const String & path_) : persistent_path(path_)
{
checkSyncUserPriv(connection);
if (Poco::File(persistent_path).exists())
{
ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE);
@ -239,9 +237,17 @@ MaterializeMetadata::MaterializeMetadata(
assertString("\nData Version:\t", in);
readIntText(data_version, in);
}
}
void MaterializeMetadata::startReplication(
mysqlxx::PoolWithFailover::Entry & connection, const String & database,
bool & opened_transaction, std::unordered_map<String, String> & need_dumping_tables)
{
checkSyncUserPriv(connection);
if (checkBinlogFileExists(connection))
return;
}
bool locked_tables = false;

View File

@ -35,7 +35,6 @@ struct MaterializeMetadata
size_t data_version = 1;
size_t meta_version = 2;
String binlog_checksum = "CRC32";
std::unordered_map<String, String> need_dumping_tables;
void fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & connection);
@ -45,9 +44,13 @@ struct MaterializeMetadata
void transaction(const MySQLReplication::Position & position, const std::function<void()> & fun);
MaterializeMetadata(
mysqlxx::PoolWithFailover::Entry & connection, const String & path
, const String & database, bool & opened_transaction);
void startReplication(
mysqlxx::PoolWithFailover::Entry & connection,
const String & database,
bool & opened_transaction,
std::unordered_map<String, String> & need_dumping_tables);
MaterializeMetadata(const String & path_);
};
}

View File

@ -14,7 +14,7 @@ class ASTStorage;
M(UInt64, max_rows_in_buffers, DEFAULT_BLOCK_SIZE, "Max rows that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \
M(UInt64, max_bytes_in_buffers, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes that data is allowed to cache in memory(for database and the cache data unable to query). when rows is exceeded, the data will be materialized", 0) \
M(UInt64, max_flush_data_time, 1000, "Max milliseconds that data is allowed to cache in memory(for database and the cache data unable to query). when this time is exceeded, the data will be materialized", 0) \
M(UInt64, max_wait_time_when_mysql_unavailable, 1000, "Dump full data retry interval when MySQL is not available(milliseconds).", 0) \
M(Int64, max_wait_time_when_mysql_unavailable, 1000, "Retry interval when MySQL is not available (milliseconds). Negative value disable retry.", 0) \
M(Bool, allows_query_when_mysql_lost, false, "Allow query materialized table when mysql is lost.", 0) \
DECLARE_SETTINGS_TRAITS(MaterializeMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS)

View File

@ -35,6 +35,8 @@ namespace ErrorCodes
extern const int ILLEGAL_MYSQL_VARIABLE;
extern const int SYNC_MYSQL_USER_ACCESS_ERROR;
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_EXCEPTION;
extern const int CANNOT_READ_ALL_DATA;
}
static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync";
@ -155,32 +157,50 @@ void MaterializeMySQLSyncThread::synchronization()
try
{
if (std::optional<MaterializeMetadata> metadata = prepareSynchronized())
{
MaterializeMetadata metadata(
DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata");
bool need_reconnect = true;
Stopwatch watch;
Buffers buffers(database_name);
while (!isCancelled())
{
if (need_reconnect)
{
if (!prepareSynchronized(metadata))
break;
need_reconnect = false;
}
/// TODO: add gc task for `sign = -1`(use alter table delete, execute by interval. need final state)
UInt64 max_flush_time = settings->max_flush_data_time;
BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds()));
try
{
BinlogEventPtr binlog_event = client.readOneBinlogEvent(std::max(UInt64(1), max_flush_time - watch.elapsedMilliseconds()));
if (binlog_event)
onEvent(buffers, binlog_event, *metadata);
onEvent(buffers, binlog_event, metadata);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::CANNOT_READ_ALL_DATA || settings->max_wait_time_when_mysql_unavailable < 0)
throw;
flushBuffersData(buffers, metadata);
LOG_INFO(log, "Lost connection to MySQL");
need_reconnect = true;
setSynchronizationThreadException(std::current_exception());
sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable);
continue;
}
if (watch.elapsedMilliseconds() > max_flush_time || buffers.checkThresholds(
settings->max_rows_in_buffer, settings->max_bytes_in_buffer,
settings->max_rows_in_buffers, settings->max_bytes_in_buffers)
)
{
watch.restart();
if (!buffers.data.empty())
flushBuffersData(buffers, *metadata);
}
}
flushBuffersData(buffers, metadata);
}
}
}
@ -188,8 +208,7 @@ void MaterializeMySQLSyncThread::synchronization()
{
client.disconnect();
tryLogCurrentException(log);
auto db = DatabaseCatalog::instance().getDatabase(database_name);
setSynchronizationThreadException(db, std::current_exception());
setSynchronizationThreadException(std::current_exception());
}
}
@ -204,19 +223,17 @@ void MaterializeMySQLSyncThread::stopSynchronization()
}
void MaterializeMySQLSyncThread::startSynchronization()
{
background_thread_pool = std::make_unique<ThreadFromGlobalPool>([this]() { synchronization(); });
}
void MaterializeMySQLSyncThread::assertMySQLAvailable()
{
try
{
checkMySQLVariables(pool.get());
background_thread_pool = std::make_unique<ThreadFromGlobalPool>([this]() { synchronization(); });
}
catch (...)
{
try
{
throw;
}
catch (mysqlxx::ConnectionFailed & e)
catch (const mysqlxx::ConnectionFailed & e)
{
if (e.errnum() == ER_ACCESS_DENIED_ERROR
|| e.errnum() == ER_DBACCESS_DENIED_ERROR)
@ -229,7 +246,6 @@ void MaterializeMySQLSyncThread::startSynchronization()
else
throw;
}
}
}
static inline void cleanOutdatedTables(const String & database_name, const Context & context)
@ -285,12 +301,12 @@ static inline BlockOutputStreamPtr getTableOutput(const String & database_name,
}
static inline void dumpDataForTables(
mysqlxx::Pool::Entry & connection, MaterializeMetadata & master_info,
mysqlxx::Pool::Entry & connection, const std::unordered_map<String, String> & need_dumping_tables,
const String & query_prefix, const String & database_name, const String & mysql_database_name,
const Context & context, const std::function<bool()> & is_cancelled)
{
auto iterator = master_info.need_dumping_tables.begin();
for (; iterator != master_info.need_dumping_tables.end() && !is_cancelled(); ++iterator)
auto iterator = need_dumping_tables.begin();
for (; iterator != need_dumping_tables.end() && !is_cancelled(); ++iterator)
{
try
{
@ -329,7 +345,7 @@ static inline UInt32 randomNumber()
return dist6(rng);
}
std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchronized()
bool MaterializeMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metadata)
{
bool opened_transaction = false;
mysqlxx::PoolWithFailover::Entry connection;
@ -338,13 +354,22 @@ std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchroniz
{
try
{
connection = pool.get();
connection = pool.tryGet();
if (connection.isNull())
{
if (settings->max_wait_time_when_mysql_unavailable < 0)
throw Exception("Unable to connect to MySQL", ErrorCodes::UNKNOWN_EXCEPTION);
sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable);
continue;
}
opened_transaction = false;
MaterializeMetadata metadata(
connection, DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata", mysql_database_name, opened_transaction);
checkMySQLVariables(connection);
std::unordered_map<String, String> need_dumping_tables;
metadata.startReplication(connection, mysql_database_name, opened_transaction, need_dumping_tables);
if (!metadata.need_dumping_tables.empty())
if (!need_dumping_tables.empty())
{
Position position;
position.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set);
@ -352,7 +377,7 @@ std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchroniz
metadata.transaction(position, [&]()
{
cleanOutdatedTables(database_name, global_context);
dumpDataForTables(connection, metadata, query_prefix, database_name, mysql_database_name, global_context, [this] { return isCancelled(); });
dumpDataForTables(connection, need_dumping_tables, query_prefix, database_name, mysql_database_name, global_context, [this] { return isCancelled(); });
});
const auto & position_message = [&]()
@ -369,7 +394,9 @@ std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchroniz
client.connect();
client.startBinlogDumpGTID(randomNumber(), mysql_database_name, metadata.executed_gtid_set, metadata.binlog_checksum);
return metadata;
setSynchronizationThreadException(nullptr);
return true;
}
catch (...)
{
@ -382,19 +409,28 @@ std::optional<MaterializeMetadata> MaterializeMySQLSyncThread::prepareSynchroniz
{
throw;
}
catch (const mysqlxx::ConnectionFailed &)
catch (const mysqlxx::ConnectionFailed &) {}
catch (const mysqlxx::BadQuery & e)
{
// Lost connection to MySQL server during query
if (e.code() != CR_SERVER_LOST || settings->max_wait_time_when_mysql_unavailable < 0)
throw;
}
setSynchronizationThreadException(std::current_exception());
/// Avoid busy loop when MySQL is not available.
sleepForMilliseconds(settings->max_wait_time_when_mysql_unavailable);
}
}
}
return {};
return false;
}
void MaterializeMySQLSyncThread::flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata)
{
if (buffers.data.empty())
return;
metadata.transaction(client.getPosition(), [&]() { buffers.commit(global_context); });
const auto & position_message = [&]()
@ -705,6 +741,12 @@ bool MaterializeMySQLSyncThread::isMySQLSyncThread()
return getThreadName() == MYSQL_BACKGROUND_THREAD_NAME;
}
void MaterializeMySQLSyncThread::setSynchronizationThreadException(const std::exception_ptr & exception)
{
auto db = DatabaseCatalog::instance().getDatabase(database_name);
DB::setSynchronizationThreadException(db, exception);
}
void MaterializeMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes)
{
total_blocks_rows += written_rows;

View File

@ -49,6 +49,8 @@ public:
void startSynchronization();
void assertMySQLAvailable();
static bool isMySQLSyncThread();
private:
@ -69,6 +71,9 @@ private:
const int ER_DBACCESS_DENIED_ERROR = 1044;
const int ER_BAD_DB_ERROR = 1049;
// https://dev.mysql.com/doc/mysql-errors/8.0/en/client-error-reference.html
const int CR_SERVER_LOST = 2013;
struct Buffers
{
String database;
@ -98,7 +103,7 @@ private:
bool isCancelled() { return sync_quit.load(std::memory_order_relaxed); }
std::optional<MaterializeMetadata> prepareSynchronized();
bool prepareSynchronized(MaterializeMetadata & metadata);
void flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata);
@ -107,6 +112,8 @@ private:
std::atomic<bool> sync_quit{false};
std::unique_ptr<ThreadFromGlobalPool> background_thread_pool;
void executeDDLAtomic(const QueryEvent & query_event);
void setSynchronizationThreadException(const std::exception_ptr & exception);
};
}

View File

@ -1,226 +0,0 @@
#pragma once
#include <Common/HashTable/Hash.h>
#include <common/logger_useful.h>
#include <type_traits>
#include <vector>
namespace DB
{
namespace
{
inline size_t roundUpToPowerOfTwoOrZero(size_t x)
{
size_t r = 8;
while (x > r)
r <<= 1;
return r;
}
}
struct EmptyDeleter {};
struct Int64Hasher
{
size_t operator()(const size_t x) const
{
return intHash64(x);
}
};
/*
Class for storing cache index.
It consists of two arrays.
The first one is split into buckets (each stores 8 elements (cells)) determined by hash of the element key.
The second one is split into 4bit numbers, which are positions in bucket for next element write (So cache uses FIFO eviction algorithm inside each bucket).
*/
template <typename K, typename V, typename Hasher, typename Deleter = EmptyDeleter>
class BucketCacheIndex
{
struct Cell
{
K key;
V index;
};
public:
template <typename = std::enable_if<std::is_same_v<EmptyDeleter, Deleter>>>
BucketCacheIndex(size_t cells_)
: buckets(roundUpToPowerOfTwoOrZero(cells_) / bucket_size)
, bucket_mask(buckets - 1)
, cells(buckets * bucket_size)
, positions((buckets / 2) + 1)
{
for (auto & cell : cells)
cell.index.setNotExists();
for (size_t bucket = 0; bucket < buckets; ++bucket)
setPosition(bucket, 0);
}
template <typename = std::enable_if<!std::is_same_v<EmptyDeleter, Deleter>>>
BucketCacheIndex(size_t cells_, Deleter deleter_)
: deleter(deleter_)
, buckets(roundUpToPowerOfTwoOrZero(cells_) / bucket_size)
, bucket_mask(buckets - 1)
, cells(buckets * bucket_size)
, positions((buckets / 2) + 1)
{
for (auto & cell : cells)
cell.index.setNotExists();
for (size_t bucket = 0; bucket < buckets; ++bucket)
setPosition(bucket, 0);
}
void set(K key, V val)
{
const size_t bucket = (hash(key) & bucket_mask);
const size_t idx = getCellIndex(key, bucket);
if (!cells[idx].index.exists())
{
incPosition(bucket);
++sz;
}
cells[idx].key = key;
cells[idx].index = val;
}
template <typename = std::enable_if<!std::is_same_v<EmptyDeleter, Deleter>>>
void setWithDelete(K key, V val)
{
const size_t bucket = (hash(key) & bucket_mask);
const size_t idx = getCellIndex(key, bucket);
if (!cells[idx].index.exists())
{
incPosition(bucket);
++sz;
}
else
{
deleter(cells[idx].key);
}
cells[idx].key = key;
cells[idx].index = val;
}
bool get(K key, V & val) const
{
const size_t bucket = (hash(key) & bucket_mask);
const size_t idx = getCellIndex(key, bucket);
if (!cells[idx].index.exists() || cells[idx].key != key)
return false;
val = cells[idx].index;
return true;
}
bool getKeyAndValue(K & key, V & val) const
{
const size_t bucket = (hash(key) & bucket_mask);
const size_t idx = getCellIndex(key, bucket);
if (!cells[idx].index.exists() || cells[idx].key != key)
return false;
key = cells[idx].key;
val = cells[idx].index;
return true;
}
bool erase(K key)
{
const size_t bucket = (hash(key) & bucket_mask);
const size_t idx = getCellIndex(key, bucket);
if (!cells[idx].index.exists() || cells[idx].key != key)
return false;
cells[idx].index.setNotExists();
--sz;
if constexpr (!std::is_same_v<EmptyDeleter, Deleter>)
deleter(cells[idx].key);
return true;
}
size_t size() const
{
return sz;
}
size_t capacity() const
{
return cells.size();
}
auto keys() const
{
std::vector<K> res;
for (const auto & cell : cells)
{
if (cell.index.exists())
{
res.push_back(cell.key);
}
}
return res;
}
private:
/// Searches for the key in the bucket.
/// Returns index of cell with provided key.
size_t getCellIndex(const K key, const size_t bucket) const
{
const size_t pos = getPosition(bucket);
for (int idx = 7; idx >= 0; --idx)
{
const size_t cur = ((pos + 1 + idx) & pos_mask);
if (cells[bucket * bucket_size + cur].index.exists() &&
cells[bucket * bucket_size + cur].key == key)
{
return bucket * bucket_size + cur;
}
}
return bucket * bucket_size + pos;
}
/// Returns current position for write in the bucket.
size_t getPosition(const size_t bucket) const
{
const size_t idx = (bucket >> 1);
if ((bucket & 1) == 0)
return ((positions[idx] >> 4) & pos_mask);
return (positions[idx] & pos_mask);
}
/// Sets current posiotion in the bucket.
void setPosition(const size_t bucket, const size_t pos)
{
const size_t idx = bucket >> 1;
if ((bucket & 1) == 0)
positions[idx] = ((pos << 4) | (positions[idx] & ((1 << 4) - 1)));
else
positions[idx] = (pos | (positions[idx] & (((1 << 4) - 1) << 4)));
}
void incPosition(const size_t bucket)
{
setPosition(bucket, (getPosition(bucket) + 1) & pos_mask);
}
static constexpr size_t bucket_size = 8;
static constexpr size_t pos_size = 3;
static constexpr size_t pos_mask = (1 << pos_size) - 1;
Hasher hash;
Deleter deleter;
size_t buckets;
size_t bucket_mask;
std::vector<Cell> cells;
std::vector<char> positions;
size_t sz = 0;
};
}

View File

@ -20,6 +20,10 @@ target_link_libraries(clickhouse_dictionaries
string_utils
)
target_link_libraries(clickhouse_dictionaries
PUBLIC
abseil_swiss_tables)
if(USE_CASSANDRA)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${CASSANDRA_INCLUDE_DIR})
endif()

File diff suppressed because it is too large Load Diff

View File

@ -3,72 +3,76 @@
#include <atomic>
#include <chrono>
#include <cmath>
#include <map>
#include <mutex>
#include <shared_mutex>
#include <utility>
#include <variant>
#include <vector>
#include <common/logger_useful.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/ThreadPool.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <pcg_random.hpp>
#include <Common/ArenaWithFreeLists.h>
#include <common/logger_useful.h>
#include <Common/randomSeed.h>
#include <Common/ThreadPool.h>
#include <Common/CurrentMetrics.h>
#include <ext/bit_cast.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryHelpers.h"
namespace CurrentMetrics
{
extern const Metric CacheDictionaryUpdateQueueBatches;
extern const Metric CacheDictionaryUpdateQueueKeys;
}
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/ICacheDictionaryStorage.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryHelpers.h>
#include <Dictionaries/CacheDictionaryUpdateQueue.h>
namespace DB
{
/** CacheDictionary store keys in cache storage and can asynchronous and synchronous updates during keys fetch.
namespace ErrorCodes
{
}
If keys are not found in storage during fetch, dictionary start update operation with update queue.
/*
*
* This dictionary is stored in a cache that has a fixed number of cells.
* These cells contain frequently used elements.
* When searching for a dictionary, the cache is searched first and special heuristic is used:
* while looking for the key, we take a look only at max_collision_length elements.
* So, our cache is not perfect. It has errors like "the key is in cache, but the cache says that it does not".
* And in this case we simply ask external source for the key which is faster.
* You have to keep this logic in mind.
* */
During update operation necessary keys are fetched from source and inserted into storage.
After that data from storage and source are aggregated and returned to the client.
Typical flow:
1. Client request data during for example getColumn function call.
2. CacheDictionary request data from storage and if all data is found in storage it returns result to client.
3. If some data is not in storage cache dictionary try to perform update.
If all keys are just expired and allow_read_expired_keys option is set dictionary starts asynchronous update and
return result to client.
If there are not found keys dictionary start synchronous update and wait for result.
4. After getting result from synchronous update dictionary aggregates data that was previously fetched from
storage and data that was fetched during update and return result to client.
*/
template <DictionaryKeyType dictionary_key_type>
class CacheDictionary final : public IDictionary
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by cache dictionary");
CacheDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
CacheDictionaryStoragePtr cache_storage_ptr_,
CacheDictionaryUpdateQueueConfiguration update_queue_configuration_,
DictionaryLifetime dict_lifetime_,
size_t strict_max_lifetime_seconds,
size_t size_,
bool allow_read_expired_keys_,
size_t max_update_queue_size_,
size_t update_queue_push_timeout_milliseconds_,
size_t query_wait_timeout_milliseconds,
size_t max_threads_for_updates);
bool allow_read_expired_keys_);
~CacheDictionary() override;
std::string getTypeName() const override { return "Cache"; }
std::string getTypeName() const override { return cache_storage_ptr->getName(); }
size_t getElementCount() const override;
size_t getBytesAllocated() const override;
double getLoadFactor() const override;
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override
@ -76,10 +80,6 @@ public:
return static_cast<double>(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed);
}
size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); }
double getLoadFactor() const override { return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size; }
bool supportUpdates() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
@ -88,14 +88,10 @@ public:
getDictionaryID(),
dict_struct,
getSourceAndUpdateIfNeeded()->clone(),
cache_storage_ptr,
update_queue.getConfiguration(),
dict_lifetime,
strict_max_lifetime_seconds,
size,
allow_read_expired_keys,
max_update_queue_size,
update_queue_push_timeout_milliseconds,
query_wait_timeout_milliseconds,
max_threads_for_updates);
allow_read_expired_keys);
}
const IDictionarySource * getSource() const override;
@ -106,133 +102,78 @@ public:
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
return dict_struct.getAttribute(attribute_name).injective;
}
bool hasHierarchy() const override { return hierarchical_attribute; }
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
void isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
std::exception_ptr getLastException() const override;
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
DictionaryKeyType getKeyType() const override
{
return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex;
}
ColumnPtr getColumn(
const std::string& attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const override;
const ColumnPtr & default_values_column) const override;
Columns getColumns(
const Strings & attribute_names,
const DataTypes & result_types,
const Columns & key_columns,
const DataTypes & key_types,
const Columns & default_values_columns) const override;
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
template <typename T>
using ResultArrayType = std::conditional_t<IsDecimalNumber<T>, DecimalPaddedPODArray<T>, PaddedPODArray<T>>;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
std::exception_ptr getLastException() const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && hierarchical_attribute; }
void toParent(const PaddedPODArray<UInt64> & ids, PaddedPODArray<UInt64> & out) const override;
void isInVectorVector(
const PaddedPODArray<UInt64> & child_ids,
const PaddedPODArray<UInt64> & ancestor_ids,
PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(
const PaddedPODArray<UInt64> & child_ids,
const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(
const UInt64 child_id,
const PaddedPODArray<UInt64> & ancestor_ids,
PaddedPODArray<UInt8> & out) const override;
private:
template <typename Value>
using ContainerType = Value[];
template <typename Value>
using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
using time_point_t = std::chrono::system_clock::time_point;
Columns getColumnsImpl(
const Strings & attribute_names,
const Columns & key_columns,
const PaddedPODArray<KeyType> & keys,
const Columns & default_values_columns) const;
struct CellMetadata final
{
UInt64 id;
time_point_t deadline;
bool is_default{false};
static MutableColumns aggregateColumnsInOrderOfKeys(
const PaddedPODArray<KeyType> & keys,
const DictionaryStorageFetchRequest & request,
const MutableColumns & fetched_columns,
const PaddedPODArray<KeyState> & key_index_to_state);
time_point_t expiresAt() const { return deadline; }
void setExpiresAt(const time_point_t & t) { deadline = t; is_default = false; }
bool isDefault() const { return is_default; }
void setDefault() { is_default = true; }
};
static MutableColumns aggregateColumns(
const PaddedPODArray<KeyType> & keys,
const DictionaryStorageFetchRequest & request,
const MutableColumns & fetched_columns_from_storage,
const PaddedPODArray<KeyState> & key_index_to_fetched_columns_from_storage_result,
const MutableColumns & fetched_columns_during_update,
const HashMap<KeyType, size_t> & found_keys_to_fetched_columns_during_update_index);
using AttributeValue = std::variant<
UInt8, UInt16, UInt32, UInt64, UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64, String>;
void setupHierarchicalAttribute();
struct AttributeValuesForKey
{
bool found{false};
std::vector<AttributeValue> values;
std::string dump();
};
using FoundValuesForKeys = std::unordered_map<Key, AttributeValuesForKey>;
struct Attribute final
{
AttributeUnderlyingType type;
String name;
/// Default value for each type. Could be defined in config.
AttributeValue null_value;
/// We store attribute value for all keys. It is a "row" in a hand-made open addressing hashtable,
/// where "column" is key.
std::variant<
ContainerPtrType<UInt8>,
ContainerPtrType<UInt16>,
ContainerPtrType<UInt32>,
ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>,
ContainerPtrType<Int16>,
ContainerPtrType<Int32>,
ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>,
ContainerPtrType<Decimal64>,
ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>,
ContainerPtrType<Float64>,
ContainerPtrType<StringRef>>
arrays;
};
void createAttributes();
/* NOLINTNEXTLINE(readability-convert-member-functions-to-static) */
Attribute createAttributeWithTypeAndName(const AttributeUnderlyingType type, const String & name, const Field & null_value);
template <typename AttributeType, typename OutputType, typename DefaultValueExtractor>
void getItemsNumberImpl(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultValueExtractor & default_value_extractor) const;
void getItemsString(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ColumnString * out,
DictionaryDefaultValueExtractor<String> & default_value_extractor) const;
PaddedPODArray<Key> getCachedIds() const;
bool isEmptyCell(const UInt64 idx) const;
size_t getCellIdx(const Key id) const;
void setDefaultAttributeValue(Attribute & attribute, const Key idx) const;
void setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const;
static std::vector<AttributeValue> getAttributeValuesFromBlockAtPosition(const std::vector<const IColumn *> & column_ptrs, size_t position);
Attribute & getAttribute(const std::string & attribute_name) const;
size_t getAttributeIndex(const std::string & attribute_name) const;
using SharedDictionarySourcePtr = std::shared_ptr<IDictionarySource>;
void update(CacheDictionaryUpdateUnitPtr<dictionary_key_type> update_unit_ptr);
/// Update dictionary source pointer if required and return it. Thread safe.
/// MultiVersion is not used here because it works with constant pointers.
@ -252,47 +193,6 @@ private:
return source_ptr;
}
inline void setLifetime(CellMetadata & cell, time_point_t now)
{
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
{
/// This maybe not obvious, but when we define is this cell is expired or expired permanently, we add strict_max_lifetime_seconds
/// to the expiration time. And it overflows pretty well.
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max() - 2 * std::chrono::seconds(strict_max_lifetime_seconds));
}
}
inline bool isExpired(time_point_t now, time_point_t deadline) const
{
return now > deadline;
}
inline bool isExpiredPermanently(time_point_t now, time_point_t deadline) const
{
return now > deadline + std::chrono::seconds(strict_max_lifetime_seconds);
}
enum class ResultState
{
NotFound,
FoundAndValid,
FoundButExpired,
/// Here is a gap between there two states in which a key could be read
/// with an enabled setting in config enable_read_expired_keys.
FoundButExpiredPermanently
};
using FindResult = std::pair<size_t, ResultState>;
FindResult findCellIdxForGet(const Key & id, const time_point_t now) const;
size_t findCellIdxForSet(const Key & id) const;
template <typename AncestorType>
void isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
@ -302,110 +202,34 @@ private:
mutable std::mutex source_mutex;
mutable SharedDictionarySourcePtr source_ptr;
CacheDictionaryStoragePtr cache_storage_ptr;
mutable CacheDictionaryUpdateQueue<dictionary_key_type> update_queue;
const DictionaryLifetime dict_lifetime;
const size_t strict_max_lifetime_seconds;
const bool allow_read_expired_keys;
const size_t max_update_queue_size;
const size_t update_queue_push_timeout_milliseconds;
const size_t query_wait_timeout_milliseconds;
const size_t max_threads_for_updates;
Poco::Logger * log;
const bool allow_read_expired_keys;
mutable pcg64 rnd_engine;
/// This lock is used for the inner cache state update function lock it for
/// write, when it need to update cache state all other functions just
/// readers. Surprisingly this lock is also used for last_exception pointer.
mutable std::shared_mutex rw_lock;
/// Actual size will be increased to match power of 2
const size_t size;
/// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111)
const size_t size_overlap_mask;
/// Max tries to find cell, overlapped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3
static constexpr size_t max_collision_length = 10;
const size_t zero_cell_idx{getCellIdx(0)};
std::map<std::string, size_t> attribute_index_by_name;
mutable std::vector<Attribute> attributes;
mutable std::vector<CellMetadata> cells;
Attribute * hierarchical_attribute = nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena;
const DictionaryAttribute * hierarchical_attribute = nullptr;
mutable std::exception_ptr last_exception;
mutable std::atomic<size_t> error_count{0};
mutable std::atomic<size_t> error_count {0};
mutable std::atomic<std::chrono::system_clock::time_point> backoff_end_time{std::chrono::system_clock::time_point{}};
mutable pcg64 rnd_engine;
mutable size_t bytes_allocated = 0;
mutable std::atomic<size_t> element_count{0};
mutable std::atomic<size_t> hit_count{0};
mutable std::atomic<size_t> query_count{0};
/*
* How the update goes: we basically have a method like get(keys)->values. Values are cached, so sometimes we
* can return them from the cache. For values not in cache, we query them from the source, and add to the
* cache. The cache is lossy, so we can't expect it to store all the keys, and we store them separately.
* So, there is a map of found keys to all its attributes.
*/
struct UpdateUnit
{
explicit UpdateUnit(std::vector<Key> && requested_ids_) :
requested_ids(std::move(requested_ids_)),
alive_keys(CurrentMetrics::CacheDictionaryUpdateQueueKeys, requested_ids.size())
{
found_ids.reserve(requested_ids.size());
for (const auto id : requested_ids)
found_ids.insert({id, {}});
}
std::vector<Key> requested_ids;
FoundValuesForKeys found_ids;
std::atomic<bool> is_done{false};
std::exception_ptr current_exception{nullptr};
/// While UpdateUnit is alive, it is accounted in update_queue size.
CurrentMetrics::Increment alive_batch{CurrentMetrics::CacheDictionaryUpdateQueueBatches};
CurrentMetrics::Increment alive_keys;
std::string dumpFoundIds();
};
using UpdateUnitPtr = std::shared_ptr<UpdateUnit>;
using UpdateQueue = ConcurrentBoundedQueue<UpdateUnitPtr>;
mutable UpdateQueue update_queue;
ThreadPool update_pool;
/*
* Actually, we can divide all requested keys into two 'buckets'. There are only four possible states and they
* are described in the table.
*
* cache_not_found_ids |0|0|1|1|
* cache_expired_ids |0|1|0|1|
*
* 0 - if set is empty, 1 - otherwise
*
* Only if there are no cache_not_found_ids and some cache_expired_ids
* (with allow_read_expired_keys setting) we can perform async update.
* Otherwise we have no concatenate ids and update them sync.
*
*/
void updateThreadFunction();
void update(UpdateUnitPtr & update_unit_ptr);
void tryPushToUpdateQueueOrThrow(UpdateUnitPtr & update_unit_ptr) const;
void waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr) const;
mutable std::mutex update_mutex;
mutable std::condition_variable is_update_finished;
std::atomic<bool> finished{false};
};
extern template class CacheDictionary<DictionaryKeyType::simple>;
extern template class CacheDictionary<DictionaryKeyType::complex>;
}

View File

@ -0,0 +1,418 @@
#pragma once
#include <chrono>
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <Common/Arena.h>
#include <Common/ArenaWithFreeLists.h>
#include <Common/HashTable/LRUHashMap.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/ICacheDictionaryStorage.h>
#include <Dictionaries/DictionaryHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct CacheDictionaryStorageConfiguration
{
/// Max size of storage in cells
const size_t max_size_in_cells;
/// Needed to perform check if cell is expired or not found. Default value is dictionary max lifetime.
const size_t strict_max_lifetime_seconds;
/// Lifetime of dictionary. Cell deadline is random value between lifetime min and max seconds.
const DictionaryLifetime lifetime;
};
/** Keys are stored in LRUCache and column values are serialized into arena.
Cell in LRUCache consists of allocated size and place in arena were columns serialized data is stored.
Columns are serialized by rows.
When cell is removed from LRUCache data associated with it is also removed from arena.
In case of complex key we also store key data in arena and it is removed from arena.
*/
template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryStorage final : public ICacheDictionaryStorage
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryStorage");
explicit CacheDictionaryStorage(CacheDictionaryStorageConfiguration & configuration_)
: configuration(configuration_)
, rnd_engine(randomSeed())
, cache(configuration.max_size_in_cells, false, { arena })
{
}
bool returnsFetchedColumnsInOrderOfRequestedKeys() const override { return true; }
String getName() const override
{
if (dictionary_key_type == DictionaryKeyType::simple)
return "Cache";
else
return "ComplexKeyCache";
}
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
{
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
}
else
throw Exception("Method fetchColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
}
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
}
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
}
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return getCachedKeysImpl();
else
throw Exception("Method getCachedSimpleKeys is not supported for complex key storage", ErrorCodes::NOT_IMPLEMENTED);
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
{
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
}
else
throw Exception("Method fetchColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
}
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception("Method insertColumnsForKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
}
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
insertDefaultKeysImpl(keys);
else
throw Exception("Method insertDefaultKeysImpl is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
}
PaddedPODArray<StringRef> getCachedComplexKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
return getCachedKeysImpl();
else
throw Exception("Method getCachedComplexKeys is not supported for simple key storage", ErrorCodes::NOT_IMPLEMENTED);
}
size_t getSize() const override { return cache.size(); }
size_t getMaxSize() const override { return cache.getMaxSize(); }
size_t getBytesAllocated() const override { return arena.size() + cache.getSizeInBytes(); }
private:
template <typename KeysStorageFetchResult>
ALWAYS_INLINE KeysStorageFetchResult fetchColumnsForKeysImpl(
const PaddedPODArray<KeyType> & keys,
const DictionaryStorageFetchRequest & fetch_request)
{
KeysStorageFetchResult result;
result.fetched_columns = fetch_request.makeAttributesResultColumns();
result.key_index_to_state.resize_fill(keys.size(), {KeyState::not_found});
const auto now = std::chrono::system_clock::now();
size_t fetched_columns_index = 0;
std::chrono::seconds max_lifetime_seconds(configuration.strict_max_lifetime_seconds);
size_t keys_size = keys.size();
for (size_t key_index = 0; key_index < keys_size; ++key_index)
{
auto key = keys[key_index];
auto * it = cache.find(key);
if (it)
{
/// Columns values for key are serialized in cache now deserialize them
const auto & cell = it->getMapped();
bool has_deadline = cellHasDeadline(cell);
if (has_deadline && now > cell.deadline + max_lifetime_seconds)
{
result.key_index_to_state[key_index] = {KeyState::not_found};
++result.not_found_keys_size;
continue;
}
else if (has_deadline && now > cell.deadline)
{
result.key_index_to_state[key_index] = {KeyState::expired, fetched_columns_index};
++result.expired_keys_size;
}
else
{
result.key_index_to_state[key_index] = {KeyState::found, fetched_columns_index};
++result.found_keys_size;
}
++fetched_columns_index;
if (cell.isDefault())
{
result.key_index_to_state[key_index].setDefault();
++result.default_keys_size;
insertDefaultValuesIntoColumns(result.fetched_columns, fetch_request, key_index);
}
else
{
const char * place_for_serialized_columns = cell.place_for_serialized_columns;
deserializeAndInsertIntoColumns(result.fetched_columns, fetch_request, place_for_serialized_columns);
}
}
else
{
result.key_index_to_state[key_index] = {KeyState::not_found};
++result.not_found_keys_size;
}
}
return result;
}
void insertColumnsForKeysImpl(const PaddedPODArray<KeyType> & keys, Columns columns)
{
Arena temporary_values_pool;
size_t columns_to_serialize_size = columns.size();
PaddedPODArray<StringRef> temporary_column_data(columns_to_serialize_size);
const auto now = std::chrono::system_clock::now();
size_t keys_size = keys.size();
for (size_t key_index = 0; key_index < keys_size; ++key_index)
{
size_t allocated_size_for_columns = 0;
const char * block_start = nullptr;
auto key = keys[key_index];
auto * it = cache.find(key);
for (size_t column_index = 0; column_index < columns_to_serialize_size; ++column_index)
{
auto & column = columns[column_index];
temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_values_pool, block_start);
allocated_size_for_columns += temporary_column_data[column_index].size;
}
char * place_for_serialized_columns = arena.alloc(allocated_size_for_columns);
memcpy(reinterpret_cast<void*>(place_for_serialized_columns), reinterpret_cast<const void*>(block_start), allocated_size_for_columns);
if (it)
{
/// Cell exists need to free previous serialized place and update deadline
auto & cell = it->getMapped();
if (cell.place_for_serialized_columns)
arena.free(cell.place_for_serialized_columns, cell.allocated_size_for_columns);
setCellDeadline(cell, now);
cell.allocated_size_for_columns = allocated_size_for_columns;
cell.place_for_serialized_columns = place_for_serialized_columns;
}
else
{
/// No cell exists so create and put in cache
Cell cell;
setCellDeadline(cell, now);
cell.allocated_size_for_columns = allocated_size_for_columns;
cell.place_for_serialized_columns = place_for_serialized_columns;
insertCellInCache(key, cell);
}
temporary_values_pool.rollback(allocated_size_for_columns);
}
}
void insertDefaultKeysImpl(const PaddedPODArray<KeyType> & keys)
{
const auto now = std::chrono::system_clock::now();
for (auto key : keys)
{
auto * it = cache.find(key);
if (it)
{
auto & cell = it->getMapped();
setCellDeadline(cell, now);
if (cell.place_for_serialized_columns)
arena.free(cell.place_for_serialized_columns, cell.allocated_size_for_columns);
cell.allocated_size_for_columns = 0;
cell.place_for_serialized_columns = nullptr;
}
else
{
Cell cell;
setCellDeadline(cell, now);
cell.allocated_size_for_columns = 0;
cell.place_for_serialized_columns = nullptr;
insertCellInCache(key, cell);
}
}
}
PaddedPODArray<KeyType> getCachedKeysImpl() const
{
PaddedPODArray<KeyType> result;
result.reserve(cache.size());
for (auto & node : cache)
{
auto & cell = node.getMapped();
if (cell.isDefault())
continue;
result.emplace_back(node.getKey());
}
return result;
}
using TimePoint = std::chrono::system_clock::time_point;
struct Cell
{
TimePoint deadline;
size_t allocated_size_for_columns;
char * place_for_serialized_columns;
inline bool isDefault() const { return place_for_serialized_columns == nullptr; }
inline void setDefault()
{
place_for_serialized_columns = nullptr;
allocated_size_for_columns = 0;
}
};
void insertCellInCache(KeyType & key, const Cell & cell)
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
{
/// Copy complex key into arena and put in cache
size_t key_size = key.size;
char * place_for_key = arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(key.data), key_size);
KeyType updated_key{place_for_key, key_size};
key = updated_key;
}
cache.insert(key, cell);
}
inline static bool cellHasDeadline(const Cell & cell)
{
return cell.deadline != std::chrono::system_clock::from_time_t(0);
}
inline void setCellDeadline(Cell & cell, TimePoint now)
{
if (configuration.lifetime.min_sec == 0 && configuration.lifetime.max_sec == 0)
{
cell.deadline = std::chrono::system_clock::from_time_t(0);
return;
}
size_t min_sec_lifetime = configuration.lifetime.min_sec;
size_t max_sec_lifetime = configuration.lifetime.max_sec;
std::uniform_int_distribution<UInt64> distribution{min_sec_lifetime, max_sec_lifetime};
cell.deadline = now + std::chrono::seconds(distribution(rnd_engine));
}
template <typename>
friend class ArenaCellDisposer;
CacheDictionaryStorageConfiguration configuration;
ArenaWithFreeLists arena;
pcg64 rnd_engine;
class ArenaCellDisposer
{
public:
ArenaWithFreeLists & arena;
template <typename Key, typename Value>
void operator()(const Key & key, const Value & value) const
{
/// In case of complex key we keep it in arena
if constexpr (std::is_same_v<Key, StringRef>)
arena.free(const_cast<char *>(key.data), key.size);
if (value.place_for_serialized_columns)
arena.free(value.place_for_serialized_columns, value.allocated_size_for_columns);
}
};
using SimpleKeyLRUHashMap = LRUHashMap<UInt64, Cell, ArenaCellDisposer>;
using ComplexKeyLRUHashMap = LRUHashMapWithSavedHash<StringRef, Cell, ArenaCellDisposer>;
using CacheLRUHashMap = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple,
SimpleKeyLRUHashMap,
ComplexKeyLRUHashMap>;
CacheLRUHashMap cache;
};
}

View File

@ -0,0 +1,162 @@
#include "CacheDictionaryUpdateQueue.h"
#include <Dictionaries/CacheDictionaryUpdateQueue.h>
#include <Common/setThreadName.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CACHE_DICTIONARY_UPDATE_FAIL;
extern const int UNSUPPORTED_METHOD;
extern const int TIMEOUT_EXCEEDED;
}
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_,
CacheDictionaryUpdateQueueConfiguration configuration_,
UpdateFunction && update_func_)
: dictionary_name_for_logs(std::move(dictionary_name_for_logs_))
, configuration(configuration_)
, update_func(std::move(update_func_))
, update_queue(configuration.max_update_queue_size)
, update_pool(configuration.max_threads_for_updates)
{
for (size_t i = 0; i < configuration.max_threads_for_updates; ++i)
update_pool.scheduleOrThrowOnError([this] { updateThreadFunction(); });
}
template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::~CacheDictionaryUpdateQueue()
{
try {
if (!finished)
stopAndWait();
}
catch (...)
{
/// TODO: Write log
}
}
template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::tryPushToUpdateQueueOrThrow(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr)
{
if (finished)
throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD};
if (!update_queue.tryPush(update_unit_ptr, configuration.update_queue_push_timeout_milliseconds))
throw DB::Exception(
ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL,
"Cannot push to internal update queue in dictionary {}. "
"Timelimit of {} ms. exceeded. Current queue size is {}",
dictionary_name_for_logs,
std::to_string(configuration.update_queue_push_timeout_milliseconds),
std::to_string(update_queue.size()));
}
template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::waitForCurrentUpdateFinish(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr) const
{
if (finished)
throw Exception{"CacheDictionaryUpdateQueue finished", ErrorCodes::UNSUPPORTED_METHOD};
std::unique_lock<std::mutex> update_lock(update_mutex);
bool result = is_update_finished.wait_for(
update_lock,
std::chrono::milliseconds(configuration.query_wait_timeout_milliseconds),
[&]
{
return update_unit_ptr->is_done || update_unit_ptr->current_exception;
});
if (!result)
{
throw DB::Exception(
ErrorCodes::TIMEOUT_EXCEEDED,
"Dictionary {} source seems unavailable, because {} ms timeout exceeded.",
dictionary_name_for_logs,
toString(configuration.query_wait_timeout_milliseconds));
}
if (update_unit_ptr->current_exception)
{
// Don't just rethrow it, because sharing the same exception object
// between multiple threads can lead to weird effects if they decide to
// modify it, for example, by adding some error context.
try
{
std::rethrow_exception(update_unit_ptr->current_exception);
}
catch (...)
{
throw DB::Exception(
ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL,
"Update failed for dictionary '{}': {}",
dictionary_name_for_logs,
getCurrentExceptionMessage(true /*with stack trace*/, true /*check embedded stack trace*/));
}
}
}
template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::stopAndWait()
{
finished = true;
update_queue.clear();
for (size_t i = 0; i < configuration.max_threads_for_updates; ++i)
{
auto empty_finishing_ptr = std::make_shared<CacheDictionaryUpdateUnit<dictionary_key_type>>();
update_queue.push(empty_finishing_ptr);
}
update_pool.wait();
}
template <DictionaryKeyType dictionary_key_type>
void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
{
setThreadName("UpdQueue");
while (!finished)
{
CacheDictionaryUpdateUnitPtr<dictionary_key_type> unit_to_update;
update_queue.pop(unit_to_update);
if (finished)
break;
try
{
/// Update
update_func(unit_to_update);
/// Notify thread about finished updating the bunch of ids
/// where their own ids were included.
std::unique_lock<std::mutex> lock(update_mutex);
unit_to_update->is_done = true;
is_update_finished.notify_all();
}
catch (...)
{
std::unique_lock<std::mutex> lock(update_mutex);
unit_to_update->current_exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing)
is_update_finished.notify_all();
}
}
}
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
}

View File

@ -0,0 +1,172 @@
#pragma once
#include <atomic>
#include <mutex>
#include <shared_mutex>
#include <utility>
#include <vector>
#include <functional>
#include <Common/ThreadPool.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentMetrics.h>
#include <Common/PODArray.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/IColumn.h>
#include <Dictionaries/ICacheDictionaryStorage.h>
namespace CurrentMetrics
{
extern const Metric CacheDictionaryUpdateQueueBatches;
extern const Metric CacheDictionaryUpdateQueueKeys;
}
namespace DB
{
/** This class is passed between update queue and update queue client during update.
For simple keys we pass simple keys.
For complex keys we pass complex keys columns and requested rows to update.
During update cache dictionary should fill requested_keys_to_fetched_columns_during_update_index and
fetched_columns_during_update.
For complex key to extend lifetime of key complex key arena should be used.
*/
template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateUnit
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
/// Constructor for complex keys update request
explicit CacheDictionaryUpdateUnit(
const Columns & key_columns_,
const PaddedPODArray<KeyState> & key_index_to_state_from_storage_,
const DictionaryStorageFetchRequest & request_,
size_t keys_to_update_size_)
: key_columns(key_columns_)
, key_index_to_state(key_index_to_state_from_storage_.begin(), key_index_to_state_from_storage_.end())
, request(request_)
, keys_to_update_size(keys_to_update_size_)
, alive_keys(CurrentMetrics::CacheDictionaryUpdateQueueKeys, keys_to_update_size)
{}
CacheDictionaryUpdateUnit()
: keys_to_update_size(0)
, alive_keys(CurrentMetrics::CacheDictionaryUpdateQueueKeys, 0)
{}
const Columns key_columns;
const PaddedPODArray<KeyState> key_index_to_state;
const DictionaryStorageFetchRequest request;
const size_t keys_to_update_size;
HashMap<KeyType, size_t> requested_keys_to_fetched_columns_during_update_index;
MutableColumns fetched_columns_during_update;
/// Complex keys are serialized in this arena
Arena complex_key_arena;
private:
template <DictionaryKeyType>
friend class CacheDictionaryUpdateQueue;
std::atomic<bool> is_done{false};
std::exception_ptr current_exception{nullptr};
/// While UpdateUnit is alive, it is accounted in update_queue size.
CurrentMetrics::Increment alive_batch{CurrentMetrics::CacheDictionaryUpdateQueueBatches};
CurrentMetrics::Increment alive_keys;
};
template <DictionaryKeyType dictionary_key_type>
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
struct CacheDictionaryUpdateQueueConfiguration
{
/// Size of update queue
const size_t max_update_queue_size;
/// Size in thead pool of update queue
const size_t max_threads_for_updates;
/// Timeout for trying to push update unit into queue
const size_t update_queue_push_timeout_milliseconds;
/// Timeout during sync waititing of update unit
const size_t query_wait_timeout_milliseconds;
};
/** Responsibility of this class is to provide asynchronous and synchronous update support for CacheDictionary
It is responsibility of CacheDictionary to perform update with UpdateUnit using UpdateFunction.
*/
template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateQueue
{
public:
/// Client of update queue must provide this function in constructor and perform update using update unit.
using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_,
CacheDictionaryUpdateQueueConfiguration configuration_,
UpdateFunction && update_func_);
~CacheDictionaryUpdateQueue();
/// Get configuration that was passed to constructor
const CacheDictionaryUpdateQueueConfiguration & getConfiguration() const { return configuration; }
/// Is queue finished
bool isFinished() const { return finished; }
/// Synchronous wait for update queue to stop
void stopAndWait();
/** Try to add update unit into queue.
If queue is full and oush cannot be performed in update_queue_push_timeout_milliseconds from configuration
an exception will be thrown.
If queue already finished an exception will be thrown.
*/
void tryPushToUpdateQueueOrThrow(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr);
/** Try to synchronously wait for update completion.
If exception was passed from update function during update it will be rethrowed.
If update will not be finished in query_wait_timeout_milliseconds from configuration
an exception will be thrown.
If queue already finished an exception will be thrown.
*/
void waitForCurrentUpdateFinish(CacheDictionaryUpdateUnitPtr<dictionary_key_type> & update_unit_ptr) const;
private:
void updateThreadFunction();
using UpdateQueue = ConcurrentBoundedQueue<CacheDictionaryUpdateUnitPtr<dictionary_key_type>>;
String dictionary_name_for_logs;
CacheDictionaryUpdateQueueConfiguration configuration;
UpdateFunction update_func;
UpdateQueue update_queue;
ThreadPool update_pool;
mutable std::mutex update_mutex;
mutable std::condition_variable is_update_finished;
std::atomic<bool> finished{false};
};
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
}

View File

@ -133,41 +133,25 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
{
/** Query to local ClickHouse is marked internal in order to avoid
* the necessity of holding process_list_element shared pointer.
*/
if (is_local)
{
auto stream = executeQuery(load_all_query, context, true).getInputStream();
/// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return
stream = std::make_shared<ConvertingBlockInputStream>(stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
return stream;
}
return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context);
return createStreamForQuery(load_all_query);
}
BlockInputStreamPtr ClickHouseDictionarySource::loadUpdatedAll()
{
std::string load_update_query = getUpdateFieldAndDate();
if (is_local)
{
auto stream = executeQuery(load_update_query, context, true).getInputStream();
stream = std::make_shared<ConvertingBlockInputStream>(stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
return stream;
}
return std::make_shared<RemoteBlockInputStream>(pool, load_update_query, sample_block, context);
String load_update_query = getUpdateFieldAndDate();
return createStreamForQuery(load_update_query);
}
BlockInputStreamPtr ClickHouseDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
return createStreamForSelectiveLoad(query_builder.composeLoadIdsQuery(ids));
return createStreamForQuery(query_builder.composeLoadIdsQuery(ids));
}
BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
return createStreamForSelectiveLoad(
query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES));
String query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES);
return createStreamForQuery(query);
}
bool ClickHouseDictionarySource::isModified() const
@ -194,17 +178,19 @@ std::string ClickHouseDictionarySource::toString() const
}
BlockInputStreamPtr ClickHouseDictionarySource::createStreamForSelectiveLoad(const std::string & query)
BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const String & query)
{
/// Sample block should not contain first row default values
auto empty_sample_block = sample_block.cloneEmpty();
if (is_local)
{
auto res = executeQuery(query, context, true).getInputStream();
res = std::make_shared<ConvertingBlockInputStream>(
res, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
return res;
auto stream = executeQuery(query, context, true).getInputStream();
stream = std::make_shared<ConvertingBlockInputStream>(stream, empty_sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
return stream;
}
return std::make_shared<RemoteBlockInputStream>(pool, query, sample_block, context);
return std::make_shared<RemoteBlockInputStream>(pool, query, empty_sample_block, context);
}
std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const

View File

@ -55,7 +55,7 @@ public:
private:
std::string getUpdateFieldAndDate();
BlockInputStreamPtr createStreamForSelectiveLoad(const std::string & query);
BlockInputStreamPtr createStreamForQuery(const String & query);
std::string doInvalidateQuery(const std::string & request) const;

View File

@ -1,915 +0,0 @@
#include "ComplexKeyCacheDictionary.h"
#include <Common/Arena.h>
#include <Common/BitHelpers.h>
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/Stopwatch.h>
#include <Common/randomSeed.h>
#include <ext/map.h>
#include <ext/range.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
namespace ProfileEvents
{
extern const Event DictCacheKeysRequested;
extern const Event DictCacheKeysRequestedMiss;
extern const Event DictCacheKeysRequestedFound;
extern const Event DictCacheKeysExpired;
extern const Event DictCacheKeysNotFound;
extern const Event DictCacheKeysHit;
extern const Event DictCacheRequestTimeNs;
extern const Event DictCacheLockWriteNs;
extern const Event DictCacheLockReadNs;
}
namespace CurrentMetrics
{
extern const Metric DictCacheRequests;
}
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
extern const int TOO_SMALL_BUFFER_SIZE;
}
inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const
{
const auto hash = StringRefHash{}(key);
const auto idx = hash & size_overlap_mask;
return idx;
}
ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
const size_t size_)
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
, size{roundUpToPowerOfTwoOrZero(std::max(size_, size_t(max_collision_length)))}
, size_overlap_mask{this->size - 1}
, rnd_engine(randomSeed())
{
if (!this->source_ptr->supportsSelectiveLoad())
throw Exception{full_name + ": source cannot be used with ComplexKeyCacheDictionary", ErrorCodes::UNSUPPORTED_METHOD};
createAttributes();
}
ColumnPtr ComplexKeyCacheDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const
{
dict_struct.validateKeyTypes(key_types);
ColumnPtr result;
auto & attribute = getAttribute(attribute_name);
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto keys_size = key_columns.front()->size();
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ColumnProvider = DictionaryAttributeColumnProvider<AttributeType>;
const auto & null_value = std::get<AttributeType>(attribute.null_values);
DictionaryDefaultValueExtractor<AttributeType> default_value_extractor(null_value, default_values_column);
auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size);
if constexpr (std::is_same_v<AttributeType, String>)
{
auto * out = column.get();
getItemsString(attribute, key_columns, out, default_value_extractor);
}
else
{
auto & out = column->getData();
getItemsNumberImpl<AttributeType, AttributeType>(attribute, key_columns, out, default_value_extractor);
}
result = std::move(column);
};
callOnDictionaryAttributeType(attribute.type, type_call);
return result;
}
/// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag,
/// true false found and valid
/// false true not found (something outdated, maybe our cell)
/// false false not found (other id stored with valid data)
/// true true impossible
///
/// todo: split this func to two: find_for_get and find_for_set
ComplexKeyCacheDictionary::FindResult
ComplexKeyCacheDictionary::findCellIdx(const StringRef & key, const CellMetadata::time_point_t now, const size_t hash) const
{
auto pos = hash;
auto oldest_id = pos;
auto oldest_time = CellMetadata::time_point_t::max();
const auto stop = pos + max_collision_length;
for (; pos < stop; ++pos)
{
const auto cell_idx = pos & size_overlap_mask;
const auto & cell = cells[cell_idx];
if (cell.hash != hash || cell.key != key)
{
/// maybe we already found nearest expired cell
if (oldest_time > now && oldest_time > cell.expiresAt())
{
oldest_time = cell.expiresAt();
oldest_id = cell_idx;
}
continue;
}
if (cell.expiresAt() < now)
{
return {cell_idx, false, true};
}
return {cell_idx, true, false};
}
oldest_id &= size_overlap_mask;
return {oldest_id, false, false};
}
ColumnUInt8::Ptr ComplexKeyCacheDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{
dict_struct.validateKeyTypes(key_types);
const auto rows_num = key_columns.front()->size();
auto result = ColumnUInt8::create(rows_num);
auto& out = result->getData();
for (const auto row : ext::range(0, rows_num))
out[row] = false;
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<size_t>> outdated_keys;
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows_num);
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows_num))
{
const StringRef key = placeKeysInPool(row, key_columns, keys, *dict_struct.key, temporary_keys_pool);
keys_array[row] = key;
const auto find_result = findCellIdx(key, now);
const auto & cell_idx = find_result.cell_idx;
/** cell should be updated if either:
* 1. keys (or hash) do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (!find_result.valid)
{
outdated_keys[key].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell = cells[cell_idx];
out[row] = !cell.isDefault();
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows_num, std::memory_order_relaxed);
hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release);
if (outdated_keys.empty())
return result;
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(
std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getMapped().front(); });
/// request new values
update(
key_columns,
keys_array,
required_rows,
[&](const StringRef key, const auto)
{
for (const auto out_idx : outdated_keys[key])
out[out_idx] = true;
},
[&](const StringRef key, const auto)
{
for (const auto out_idx : outdated_keys[key])
out[out_idx] = false;
});
return result;
}
template <typename AttributeType, typename OutputType, typename DefaultValueExtractor>
void ComplexKeyCacheDictionary::getItemsNumberImpl(
Attribute & attribute,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultValueExtractor & default_value_extractor) const
{
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<size_t>> outdated_keys;
auto & attribute_array = std::get<ContainerPtrType<AttributeType>>(attribute.arrays);
const auto rows_num = key_columns.front()->size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows_num);
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows_num))
{
const StringRef key = placeKeysInPool(row, key_columns, keys, *dict_struct.key, temporary_keys_pool);
keys_array[row] = key;
const auto find_result = findCellIdx(key, now);
/** cell should be updated if either:
* 1. keys (or hash) do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (!find_result.valid)
{
outdated_keys[key].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
out[row] = cell.isDefault() ? default_value_extractor[row] : static_cast<OutputType>(attribute_array[cell_idx]);
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows_num, std::memory_order_relaxed);
hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release);
if (outdated_keys.empty())
return;
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair)
{
return pair.getMapped().front();
});
/// request new values
update(
key_columns,
keys_array,
required_rows,
[&](const StringRef key, const size_t cell_idx)
{
for (const auto row : outdated_keys[key])
out[row] = static_cast<OutputType>(attribute_array[cell_idx]);
},
[&](const StringRef key, const size_t)
{
for (const auto row : outdated_keys[key])
out[row] = default_value_extractor[row];
});
}
void ComplexKeyCacheDictionary::getItemsString(
Attribute & attribute,
const Columns & key_columns,
ColumnString * out,
DictionaryDefaultValueExtractor<String> & default_value_extractor) const
{
const auto rows_num = key_columns.front()->size();
/// save on some allocations
out->getOffsets().reserve(rows_num);
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
auto & attribute_array = std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto found_outdated_values = false;
/// perform optimistic version, fallback to pessimistic if failed
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto row : ext::range(0, rows_num))
{
const StringRef key = placeKeysInPool(row, key_columns, keys, *dict_struct.key, temporary_keys_pool);
SCOPE_EXIT(temporary_keys_pool.rollback(key.size));
const auto find_result = findCellIdx(key, now);
if (!find_result.valid)
{
found_outdated_values = true;
break;
}
else
{
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? default_value_extractor[row] : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
}
/// optimistic code completed successfully
if (!found_outdated_values)
{
query_count.fetch_add(rows_num, std::memory_order_relaxed);
hit_count.fetch_add(rows_num, std::memory_order_release);
return;
}
/// now onto the pessimistic one, discard possible partial results from the optimistic path
out->getChars().resize_assume_reserved(0);
out->getOffsets().resize_assume_reserved(0);
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<size_t>> outdated_keys;
/// we are going to store every string separately
MapType<StringRef> map;
PODArray<StringRef> keys_array(rows_num);
size_t total_length = 0;
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
for (const auto row : ext::range(0, rows_num))
{
const StringRef key = placeKeysInPool(row, key_columns, keys, *dict_struct.key, temporary_keys_pool);
keys_array[row] = key;
const auto find_result = findCellIdx(key, now);
if (!find_result.valid)
{
outdated_keys[key].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? default_value_extractor[row] : attribute_array[cell_idx];
if (!cell.isDefault())
map[key] = copyIntoArena(string_ref, temporary_keys_pool);
total_length += string_ref.size + 1;
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows_num, std::memory_order_relaxed);
hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release);
/// request new values
if (!outdated_keys.empty())
{
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair)
{
return pair.getMapped().front();
});
update(
key_columns,
keys_array,
required_rows,
[&](const StringRef key, const size_t cell_idx)
{
const StringRef attribute_value = attribute_array[cell_idx];
/// We must copy key and value to own memory, because it may be replaced with another
/// in next iterations of inner loop of update.
const StringRef copied_key = copyIntoArena(key, temporary_keys_pool);
const StringRef copied_value = copyIntoArena(attribute_value, temporary_keys_pool);
map[copied_key] = copied_value;
total_length += (attribute_value.size + 1) * outdated_keys[key].size();
},
[&](const StringRef key, const size_t)
{
for (const auto row : outdated_keys[key])
total_length += default_value_extractor[row].size + 1;
});
}
out->getChars().reserve(total_length);
for (const auto row : ext::range(0, ext::size(keys_array)))
{
const StringRef key = keys_array[row];
auto * const it = map.find(key);
const auto string_ref = it ? it->getMapped() : default_value_extractor[row];
out->insertData(string_ref.data, string_ref.size);
}
}
template <typename PresentKeyHandler, typename AbsentKeyHandler>
void ComplexKeyCacheDictionary::update(
const Columns & in_key_columns,
const PODArray<StringRef> & in_keys,
const std::vector<size_t> & in_requested_rows,
PresentKeyHandler && on_cell_updated,
AbsentKeyHandler && on_key_not_found) const
{
MapType<bool> remaining_keys{in_requested_rows.size()};
for (const auto row : in_requested_rows)
remaining_keys.insert({in_keys[row], false});
std::uniform_int_distribution<UInt64> distribution(dict_lifetime.min_sec, dict_lifetime.max_sec);
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
{
Stopwatch watch;
auto stream = source_ptr->loadKeys(in_key_columns, in_requested_rows);
stream->readPrefix();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();
const auto now = std::chrono::system_clock::now();
while (const auto block = stream->read())
{
/// cache column pointers
const auto key_columns = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
const auto attribute_columns = ext::map<Columns>(ext::range(0, attributes_size), [&](const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
const auto rows_num = block.rows();
for (const auto row : ext::range(0, rows_num))
{
auto key = allocKey(row, key_columns, keys);
const auto hash = StringRefHash{}(key);
const auto find_result = findCellIdx(key, now, hash);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *attribute_columns[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[row]);
}
/// if cell id is zero and zero does not map to this cell, then the cell is unused
if (cell.key == StringRef{} && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
/// handle memory allocated for old key
if (key == cell.key)
{
freeKey(key);
key = cell.key;
}
else
{
/// new key is different from the old one
if (cell.key.data)
freeKey(cell.key);
cell.key = key;
}
cell.hash = hash;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// inform caller
on_cell_updated(key, cell_idx);
/// mark corresponding id as found
remaining_keys[key] = true;
}
}
stream->readSuffix();
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, in_requested_rows.size());
ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed());
}
size_t found_num = 0;
size_t not_found_num = 0;
const auto now = std::chrono::system_clock::now();
/// Check which ids have not been found and require setting null_value
for (const auto & key_found_pair : remaining_keys)
{
if (key_found_pair.getMapped())
{
++found_num;
continue;
}
++not_found_num;
auto key = key_found_pair.getKey();
const auto hash = StringRefHash{}(key);
const auto find_result = findCellIdx(key, now, hash);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
/// Set null_value for each attribute
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.key == StringRef{} && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
if (key == cell.key)
key = cell.key;
else
{
if (cell.key.data)
freeKey(cell.key);
/// copy key from temporary pool
key = copyKey(key);
cell.key = key;
}
cell.hash = hash;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
cell.setDefault();
/// inform caller that the cell has not been found
on_key_not_found(key, cell_idx);
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num);
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num);
}
void ComplexKeyCacheDictionary::createAttributes()
{
const auto attributes_size = dict_struct.attributes.size();
attributes.reserve(attributes_size);
bytes_allocated += size * sizeof(CellMetadata);
bytes_allocated += attributes_size * sizeof(attributes.front());
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH};
}
}
ComplexKeyCacheDictionary::Attribute & ComplexKeyCacheDictionary::getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
return attributes[it->second];
}
void ComplexKeyCacheDictionary::setDefaultAttributeValue(Attribute & attribute, const size_t idx) const
{
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
if constexpr (std::is_same_v<AttributeType, String>)
{
const auto & null_value_ref = std::get<String>(attribute.null_values);
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
if (string_ref.data != null_value_ref.data())
{
if (string_ref.data)
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
string_ref = StringRef{null_value_ref};
}
}
else
{
std::get<ContainerPtrType<AttributeType>>(attribute.arrays)[idx] = std::get<AttributeType>(attribute.null_values);
}
};
callOnDictionaryAttributeType(attribute.type, type_call);
}
ComplexKeyCacheDictionary::Attribute
ComplexKeyCacheDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
Attribute attr{type, {}, {}};
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
if constexpr (std::is_same_v<AttributeType, String>)
{
attr.null_values = null_value.get<String>();
attr.arrays = std::make_unique<ContainerType<StringRef>>(size);
bytes_allocated += size * sizeof(StringRef);
if (!string_arena)
string_arena = std::make_unique<ArenaWithFreeLists>();
}
else
{
attr.null_values = AttributeType(null_value.get<NearestFieldType<AttributeType>>()); /* NOLINT */
attr.arrays = std::make_unique<ContainerType<AttributeType>>(size); /* NOLINT */
bytes_allocated += size * sizeof(AttributeType);
}
};
callOnDictionaryAttributeType(type, type_call);
return attr;
}
void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const size_t idx, const Field & value) const
{
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
if constexpr (std::is_same_v<AttributeType, String>)
{
const auto & string = value.get<String>();
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
const auto & null_value_ref = std::get<String>(attribute.null_values);
/// free memory unless it points to a null_value
if (string_ref.data && string_ref.data != null_value_ref.data())
string_arena->free(const_cast<char *>(string_ref.data), string_ref.size);
const auto str_size = string.size();
if (str_size != 0)
{
auto * str_ptr = string_arena->alloc(str_size);
std::copy(string.data(), string.data() + str_size, str_ptr);
string_ref = StringRef{str_ptr, str_size};
}
else
string_ref = {};
}
else
{
std::get<ContainerPtrType<AttributeType>>(attribute.arrays)[idx] = value.get<NearestFieldType<AttributeType>>();
}
};
callOnDictionaryAttributeType(attribute.type, type_call);
}
StringRef ComplexKeyCacheDictionary::allocKey(const size_t row, const Columns & key_columns, StringRefs & keys) const
{
if (key_size_is_fixed)
return placeKeysInFixedSizePool(row, key_columns);
return placeKeysInPool(row, key_columns, keys, *dict_struct.key, *keys_pool);
}
void ComplexKeyCacheDictionary::freeKey(const StringRef key) const
{
if (key_size_is_fixed)
fixed_size_keys_pool->free(const_cast<char *>(key.data));
else
keys_pool->free(const_cast<char *>(key.data), key.size);
}
template <typename Pool>
StringRef ComplexKeyCacheDictionary::placeKeysInPool(
const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector<DictionaryAttribute> & key_attributes, Pool & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
for (size_t j = 0; j < keys_size; ++j)
{
keys[j] = key_columns[j]->getDataAt(row);
sum_keys_size += keys[j].size;
if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString)
sum_keys_size += sizeof(size_t) + 1;
}
auto place = pool.alloc(sum_keys_size);
auto key_start = place;
for (size_t j = 0; j < keys_size; ++j)
{
if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString)
{
auto start = key_start;
auto key_size = keys[j].size + 1;
memcpy(key_start, &key_size, sizeof(size_t));
key_start += sizeof(size_t);
memcpy(key_start, keys[j].data, keys[j].size);
key_start += keys[j].size;
*key_start = '\0';
++key_start;
keys[j].data = start;
keys[j].size += sizeof(size_t) + 1;
}
else
{
memcpy(key_start, keys[j].data, keys[j].size);
keys[j].data = key_start;
key_start += keys[j].size;
}
}
return {place, sum_keys_size};
}
/// Explicit instantiations.
template StringRef ComplexKeyCacheDictionary::placeKeysInPool<Arena>(
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes,
Arena & pool);
template StringRef ComplexKeyCacheDictionary::placeKeysInPool<ArenaWithFreeLists>(
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes,
ArenaWithFreeLists & pool);
StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool(const size_t row, const Columns & key_columns) const
{
auto * res = fixed_size_keys_pool->alloc();
auto * place = res;
for (const auto & key_column : key_columns)
{
const StringRef key = key_column->getDataAt(row);
memcpy(place, key.data, key.size);
place += key.size;
}
return {res, key_size};
}
StringRef ComplexKeyCacheDictionary::copyIntoArena(StringRef src, Arena & arena)
{
char * allocated = arena.alloc(src.size);
memcpy(allocated, src.data, src.size);
return {allocated, src.size};
}
StringRef ComplexKeyCacheDictionary::copyKey(const StringRef key) const
{
auto * res = key_size_is_fixed ? fixed_size_keys_pool->alloc() : keys_pool->alloc(key.size);
memcpy(res, key.data, key.size);
return {res, key.size};
}
bool ComplexKeyCacheDictionary::isEmptyCell(const UInt64 idx) const
{
return (
cells[idx].key == StringRef{}
&& (idx != zero_cell_idx || cells[idx].data == ext::safe_bit_cast<CellMetadata::time_point_urep_t>(CellMetadata::time_point_t())));
}
BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
{
std::vector<StringRef> keys;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
for (auto idx : ext::range(0, cells.size()))
if (!isEmptyCell(idx) && !cells[idx].isDefault())
keys.push_back(cells[idx].key);
}
using BlockInputStreamType = DictionaryBlockInputStream<UInt64>;
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names);
}
void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
{
auto create_layout = [=](const std::string & full_name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};
const auto & layout_prefix = config_prefix + ".layout";
const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells");
if (size == 0)
throw Exception{full_name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
if (require_nonempty)
throw Exception{full_name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<ComplexKeyCacheDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("complex_key_cache", create_layout, true);
}
}

View File

@ -1,276 +0,0 @@
#pragma once
#include <atomic>
#include <chrono>
#include <map>
#include <shared_mutex>
#include <variant>
#include <vector>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <pcg_random.hpp>
#include <Common/ArenaWithFreeLists.h>
#include <Common/HashTable/HashMap.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/SmallObjectPool.h>
#include <common/StringRef.h>
#include <ext/bit_cast.h>
#include <ext/map.h>
#include <ext/range.h>
#include <ext/size.h>
#include <ext/scope_guard.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include <DataStreams/IBlockInputStream.h>
#include "DictionaryHelpers.h"
namespace ProfileEvents
{
extern const Event DictCacheKeysRequested;
extern const Event DictCacheKeysRequestedMiss;
extern const Event DictCacheKeysRequestedFound;
extern const Event DictCacheKeysExpired;
extern const Event DictCacheKeysNotFound;
extern const Event DictCacheKeysHit;
extern const Event DictCacheRequestTimeNs;
extern const Event DictCacheLockWriteNs;
extern const Event DictCacheLockReadNs;
}
namespace DB
{
class ComplexKeyCacheDictionary final : public IDictionaryBase
{
public:
ComplexKeyCacheDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
const size_t size_);
std::string getKeyDescription() const { return key_description; }
std::string getTypeName() const override { return "ComplexKeyCache"; }
size_t getBytesAllocated() const override
{
return bytes_allocated + (key_size_is_fixed ? fixed_size_keys_pool->size() : keys_pool->size())
+ (string_arena ? string_arena->size() : 0);
}
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override
{
return static_cast<double>(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed);
}
size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); }
double getLoadFactor() const override { return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size; }
bool supportUpdates() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyCacheDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, size);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
const DictionaryStructure & getStructure() const override { return dict_struct; }
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
ColumnPtr getColumn(
const std::string& attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const override;
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value>
using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value>
using ContainerType = Value[];
template <typename Value>
using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct CellMetadata final
{
using time_point_t = std::chrono::system_clock::time_point;
using time_point_rep_t = time_point_t::rep;
using time_point_urep_t = std::make_unsigned_t<time_point_rep_t>;
static constexpr UInt64 EXPIRES_AT_MASK = std::numeric_limits<time_point_rep_t>::max();
static constexpr UInt64 IS_DEFAULT_MASK = ~EXPIRES_AT_MASK;
StringRef key;
decltype(StringRefHash{}(key)) hash;
/// Stores both expiration time and `is_default` flag in the most significant bit
time_point_urep_t data;
/// Sets expiration time, resets `is_default` flag to false
time_point_t expiresAt() const { return ext::safe_bit_cast<time_point_t>(data & EXPIRES_AT_MASK); }
void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast<time_point_urep_t>(t); }
bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; }
void setDefault() { data |= IS_DEFAULT_MASK; }
};
struct Attribute final
{
AttributeUnderlyingType type;
std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
String>
null_values;
std::variant<
ContainerPtrType<UInt8>,
ContainerPtrType<UInt16>,
ContainerPtrType<UInt32>,
ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>,
ContainerPtrType<Int16>,
ContainerPtrType<Int32>,
ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>,
ContainerPtrType<Decimal64>,
ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>,
ContainerPtrType<Float64>,
ContainerPtrType<StringRef>>
arrays;
};
void createAttributes();
Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value);
template <typename AttributeType, typename OutputType, typename DefaultValueExtractor>
void getItemsNumberImpl(
Attribute & attribute,
const Columns & key_columns,
PaddedPODArray<OutputType> & out,
DefaultValueExtractor & default_value_extractor) const;
void getItemsString(
Attribute & attribute,
const Columns & key_columns,
ColumnString * out,
DictionaryDefaultValueExtractor<String> & default_value_extractor) const;
template <typename PresentKeyHandler, typename AbsentKeyHandler>
void update(
const Columns & in_key_columns,
const PODArray<StringRef> & in_keys,
const std::vector<size_t> & in_requested_rows,
PresentKeyHandler && on_cell_updated,
AbsentKeyHandler && on_key_not_found) const;
UInt64 getCellIdx(const StringRef key) const;
void setDefaultAttributeValue(Attribute & attribute, const size_t idx) const;
void setAttributeValue(Attribute & attribute, const size_t idx, const Field & value) const;
Attribute & getAttribute(const std::string & attribute_name) const;
StringRef allocKey(const size_t row, const Columns & key_columns, StringRefs & keys) const;
void freeKey(const StringRef key) const;
template <typename Arena>
static StringRef placeKeysInPool(
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes,
Arena & pool);
StringRef placeKeysInFixedSizePool(const size_t row, const Columns & key_columns) const;
static StringRef copyIntoArena(StringRef src, Arena & arena);
StringRef copyKey(const StringRef key) const;
struct FindResult
{
const size_t cell_idx;
const bool valid;
const bool outdated;
};
FindResult findCellIdx(const StringRef & key, const CellMetadata::time_point_t now, const size_t hash) const;
FindResult findCellIdx(const StringRef & key, const CellMetadata::time_point_t now) const
{
const auto hash = StringRefHash{}(key);
return findCellIdx(key, now, hash);
}
bool isEmptyCell(const UInt64 idx) const;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const std::string key_description{dict_struct.getKeyDescription()};
mutable std::shared_mutex rw_lock;
/// Actual size will be increased to match power of 2
const size_t size;
/// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111)
const size_t size_overlap_mask;
/// Max tries to find cell, overlapped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3
static constexpr size_t max_collision_length = 10;
const UInt64 zero_cell_idx{getCellIdx(StringRef{})};
std::map<std::string, size_t> attribute_index_by_name;
mutable std::vector<Attribute> attributes;
mutable std::vector<CellMetadata> cells{size};
const bool key_size_is_fixed{dict_struct.isKeySizeFixed()};
size_t key_size{key_size_is_fixed ? dict_struct.getKeySize() : 0};
std::unique_ptr<ArenaWithFreeLists> keys_pool = key_size_is_fixed ? nullptr : std::make_unique<ArenaWithFreeLists>();
std::unique_ptr<SmallObjectPool> fixed_size_keys_pool = key_size_is_fixed ? std::make_unique<SmallObjectPool>(key_size) : nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena;
mutable pcg64 rnd_engine;
mutable size_t bytes_allocated = 0;
mutable std::atomic<size_t> element_count{0};
mutable std::atomic<size_t> hit_count{0};
mutable std::atomic<size_t> query_count{0};
const std::chrono::time_point<std::chrono::system_clock> creation_time = std::chrono::system_clock::now();
};
}

View File

@ -1,403 +0,0 @@
#include "ComplexKeyDirectDictionary.h"
#include <IO/WriteHelpers.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#include <Core/Defines.h>
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
}
ComplexKeyDirectDictionary::ComplexKeyDirectDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_)
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, saved_block{std::move(saved_block_)}
{
if (!this->source_ptr->supportsSelectiveLoad())
throw Exception{full_name + ": source cannot be used with ComplexKeyDirectDictionary", ErrorCodes::UNSUPPORTED_METHOD};
createAttributes();
}
ColumnPtr ComplexKeyDirectDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const
{
dict_struct.validateKeyTypes(key_types);
ColumnPtr result;
const auto & attribute = getAttribute(attribute_name);
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto keys_size = key_columns.front()->size();
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to = nullptr;
if (attribute.is_nullable)
{
col_null_map_to = ColumnUInt8::create(keys_size, false);
vec_null_map_to = &col_null_map_to->getData();
}
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
using ColumnProvider = DictionaryAttributeColumnProvider<AttributeType>;
const auto attribute_null_value = std::get<ValueType>(attribute.null_values);
AttributeType null_value = static_cast<AttributeType>(attribute_null_value);
DictionaryDefaultValueExtractor<AttributeType> default_value_extractor(std::move(null_value), default_values_column);
auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size);
if constexpr (std::is_same_v<AttributeType, String>)
{
auto * out = column.get();
getItemsImpl<String, String>(
attribute,
key_columns,
[&](const size_t row, const String value, bool is_null)
{
if (attribute.is_nullable)
(*vec_null_map_to)[row] = is_null;
const auto ref = StringRef{value};
out->insertData(ref.data, ref.size);
},
default_value_extractor);
}
else
{
auto & out = column->getData();
getItemsImpl<AttributeType, AttributeType>(
attribute,
key_columns,
[&](const size_t row, const auto value, bool is_null)
{
if (attribute.is_nullable)
(*vec_null_map_to)[row] = is_null;
out[row] = value;
},
default_value_extractor);
}
result = std::move(column);
};
callOnDictionaryAttributeType(attribute.type, type_call);
if (attribute.is_nullable)
{
result = ColumnNullable::create(result, std::move(col_null_map_to));
}
return result;
}
ColumnUInt8::Ptr ComplexKeyDirectDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{
dict_struct.validateKeyTypes(key_types);
auto size = key_columns.front()->size();
auto result = ColumnUInt8::create(size);
auto& out = result->getData();
const auto rows = key_columns.front()->size();
const auto keys_size = dict_struct.key->size();
StringRefs keys_array(keys_size);
MapType<UInt8> has_key;
Arena temporary_keys_pool;
std::vector<size_t> to_load(rows);
PODArray<StringRef> keys(rows);
for (const auto row : ext::range(0, rows))
{
const StringRef key = placeKeysInPool(row, key_columns, keys_array, *dict_struct.key, temporary_keys_pool);
keys[row] = key;
has_key[key] = 0;
to_load[row] = row;
}
auto stream = source_ptr->loadKeys(key_columns, to_load);
stream->readPrefix();
while (const auto block = stream->read())
{
const auto columns = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
Arena pool;
StringRefs keys_temp(keys_size);
const auto columns_size = columns.front()->size();
for (const auto row_idx : ext::range(0, columns_size))
{
const StringRef key = placeKeysInPool(row_idx, columns, keys_temp, *dict_struct.key, pool);
if (has_key.has(key))
{
has_key[key] = 1;
}
}
}
stream->readSuffix();
for (const auto row : ext::range(0, rows))
{
out[row] = has_key[keys[row]];
}
query_count.fetch_add(rows, std::memory_order_relaxed);
return result;
}
void ComplexKeyDirectDictionary::createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attribute_name_by_index.emplace(attributes.size(), attribute.name);
attributes.push_back(createAttribute(attribute, attribute.null_value, attribute.name));
if (attribute.hierarchical)
throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH};
}
}
template <typename T>
void ComplexKeyDirectDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
attribute.null_values = T(null_value.get<NearestFieldType<T>>());
}
template <>
void ComplexKeyDirectDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
{
attribute.string_arena = std::make_unique<Arena>();
const String & string = null_value.get<String>();
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
}
ComplexKeyDirectDictionary::Attribute ComplexKeyDirectDictionary::createAttribute(
const DictionaryAttribute & attribute, const Field & null_value, const std::string & attr_name)
{
Attribute attr{attribute.underlying_type, attribute.is_nullable, {}, {}, attr_name};
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
createAttributeImpl<AttributeType>(attr, null_value);
};
callOnDictionaryAttributeType(attribute.underlying_type, type_call);
return attr;
}
template <typename Pool>
StringRef ComplexKeyDirectDictionary::placeKeysInPool(
const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector<DictionaryAttribute> & key_attributes, Pool & pool) const
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
for (size_t j = 0; j < keys_size; ++j)
{
keys[j] = key_columns[j]->getDataAt(row);
sum_keys_size += keys[j].size;
if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString)
sum_keys_size += sizeof(size_t) + 1;
}
auto place = pool.alloc(sum_keys_size);
auto key_start = place;
for (size_t j = 0; j < keys_size; ++j)
{
if (key_attributes[j].underlying_type == AttributeUnderlyingType::utString)
{
auto start = key_start;
auto key_size = keys[j].size + 1;
memcpy(key_start, &key_size, sizeof(size_t));
key_start += sizeof(size_t);
memcpy(key_start, keys[j].data, keys[j].size);
key_start += keys[j].size;
*key_start = '\0';
++key_start;
keys[j].data = start;
keys[j].size += sizeof(size_t) + 1;
}
else
{
memcpy(key_start, keys[j].data, keys[j].size);
keys[j].data = key_start;
key_start += keys[j].size;
}
}
return {place, sum_keys_size};
}
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void ComplexKeyDirectDictionary::getItemsImpl(
const Attribute & attribute,
const Columns & key_columns,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const
{
const auto rows = key_columns.front()->size();
const auto keys_size = dict_struct.key->size();
StringRefs keys_array(keys_size);
MapType<OutputType> value_by_key;
HashMapWithSavedHash<StringRef, bool, StringRefHash> value_is_null;
Arena temporary_keys_pool;
std::vector<size_t> to_load(rows);
PODArray<StringRef> keys(rows);
for (const auto row : ext::range(0, rows))
{
const StringRef key = placeKeysInPool(row, key_columns, keys_array, *dict_struct.key, temporary_keys_pool);
keys[row] = key;
value_by_key[key] = static_cast<AttributeType>(default_value_extractor[row]);
to_load[row] = row;
value_is_null[key] = false;
}
auto stream = source_ptr->loadKeys(key_columns, to_load);
const auto attributes_size = attributes.size();
stream->readPrefix();
while (const auto block = stream->read())
{
const auto columns = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
const auto attribute_columns = ext::map<Columns>(ext::range(0, attributes_size), [&](const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
for (const size_t attribute_idx : ext::range(0, attributes.size()))
{
if (attribute.name != attribute_name_by_index.at(attribute_idx))
{
continue;
}
const IColumn & attribute_column = *attribute_columns[attribute_idx];
Arena pool;
StringRefs keys_temp(keys_size);
const auto columns_size = columns.front()->size();
for (const auto row_idx : ext::range(0, columns_size))
{
const StringRef key = placeKeysInPool(row_idx, columns, keys_temp, *dict_struct.key, pool);
if (value_by_key.has(key))
{
auto value = attribute_column[row_idx];
if (value.isNull())
value_is_null[key] = true;
else
value_by_key[key] = static_cast<OutputType>(value.template get<NearestFieldType<AttributeType>>());
}
}
}
}
stream->readSuffix();
for (const auto row : ext::range(0, rows))
{
auto key = keys[row];
set_value(row, value_by_key[key], value_is_null[key]);
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
const ComplexKeyDirectDictionary::Attribute & ComplexKeyDirectDictionary::getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
return attributes[it->second];
}
BlockInputStreamPtr ComplexKeyDirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const
{
return source_ptr->loadAll();
}
void registerDictionaryComplexKeyDirect(DictionaryFactory & factory)
{
auto create_layout = [=](const std::string & full_name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_direct'", ErrorCodes::BAD_ARGUMENTS};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{full_name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max"))
throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<ComplexKeyDirectDictionary>(dict_id, dict_struct, std::move(source_ptr));
};
factory.registerLayout("complex_key_direct", create_layout, true);
}
}

View File

@ -1,147 +0,0 @@
#pragma once
#include <atomic>
#include <variant>
#include <vector>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Core/Block.h>
#include <Common/HashTable/HashMap.h>
#include <ext/range.h>
#include <ext/size.h>
#include <ext/map.h>
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include "DictionaryHelpers.h"
namespace DB
{
class ComplexKeyDirectDictionary final : public IDictionaryBase
{
public:
ComplexKeyDirectDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_ = nullptr);
std::string getTypeName() const override { return "ComplexKeyDirect"; }
size_t getBytesAllocated() const override { return 0; }
size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override { return 1.0; }
size_t getElementCount() const override { return 0; }
double getLoadFactor() const override { return 0; }
std::string getKeyDescription() const { return key_description; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyDirectDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
const DictionaryStructure & getStructure() const override { return dict_struct; }
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
ColumnPtr getColumn(
const std::string& attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const override;
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value>
using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
struct Attribute final
{
AttributeUnderlyingType type;
bool is_nullable;
std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
StringRef>
null_values;
std::unique_ptr<Arena> string_arena;
std::string name;
};
void createAttributes();
template <typename T>
void addAttributeSize(const Attribute & attribute);
template <typename T>
static void createAttributeImpl(Attribute & attribute, const Field & null_value);
static Attribute createAttribute(const DictionaryAttribute & attribute, const Field & null_value, const std::string & name);
template <typename Pool>
StringRef placeKeysInPool(
const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector<DictionaryAttribute> & key_attributes, Pool & pool) const;
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void getItemsImpl(
const Attribute & attribute,
const Columns & key_columns,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const;
template <typename T>
void setAttributeValueImpl(Attribute & attribute, const Key id, const T & value);
void setAttributeValue(Attribute & attribute, const Key id, const Field & value);
const Attribute & getAttribute(const std::string & attribute_name) const;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
std::map<std::string, size_t> attribute_index_by_name;
std::map<size_t, std::string> attribute_name_by_index;
std::vector<Attribute> attributes;
mutable std::atomic<size_t> query_count{0};
BlockPtr saved_block;
const std::string key_description{dict_struct.getKeyDescription()};
};
}

View File

@ -41,7 +41,7 @@ ColumnPtr ComplexKeyHashedDictionary::getColumn(
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const
const ColumnPtr & default_values_column) const
{
dict_struct.validateKeyTypes(key_types);
@ -563,7 +563,13 @@ std::vector<StringRef> ComplexKeyHashedDictionary::getKeys(const Attribute & att
BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
{
using BlockInputStreamType = DictionaryBlockInputStream<UInt64>;
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getKeys(), column_names);
auto vector_keys = getKeys();
PaddedPODArray<StringRef> keys;
keys.reserve(vector_keys.size());
keys.assign(vector_keys.begin(), vector_keys.end());
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names);
}
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)

View File

@ -67,7 +67,7 @@ public:
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const override;
const ColumnPtr & default_values_column) const override;
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;

View File

@ -21,7 +21,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
/// TODO: Remove this class
/* BlockInputStream implementation for external dictionaries
* read() returns blocks consisting of the in-memory contents of the dictionaries
*/
@ -30,12 +30,15 @@ class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, UInt64 max_block_size, PaddedPODArray<Key> && ids, const Names & column_names);
std::shared_ptr<const IDictionaryBase> dictionary,
UInt64 max_block_size,
PaddedPODArray<Key> && ids,
const Names & column_names);
DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary,
UInt64 max_block_size,
const std::vector<StringRef> & keys,
const PaddedPODArray<StringRef> & keys,
const Names & column_names);
using GetColumnsFunction = std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & attributes)>;
@ -55,7 +58,7 @@ public:
String getName() const override { return "Dictionary"; }
protected:
Block getBlock(size_t start, size_t size) const override;
Block getBlock(size_t start, size_t length) const override;
private:
Block
@ -64,7 +67,7 @@ private:
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const;
void fillKeyColumns(
const std::vector<StringRef> & keys,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
@ -105,7 +108,7 @@ template <typename Key>
DictionaryBlockInputStream<Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary_,
UInt64 max_block_size_,
const std::vector<StringRef> & keys,
const PaddedPODArray<StringRef> & keys,
const Names & column_names_)
: DictionaryBlockInputStreamBase(keys.size(), max_block_size_)
, dictionary(dictionary_)
@ -260,7 +263,7 @@ ColumnPtr DictionaryBlockInputStream<Key>::getColumnFromIds(const PaddedPODArray
template <typename Key>
void DictionaryBlockInputStream<Key>::fillKeyColumns(
const std::vector<StringRef> & keys,
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
@ -275,7 +278,7 @@ void DictionaryBlockInputStream<Key>::fillKeyColumns(
for (auto idx : ext::range(start, size))
{
const auto & key = keys[idx];
auto ptr = key.data;
const auto *ptr = key.data;
for (auto & column : columns)
ptr = column->deserializeAndInsertFromArena(ptr);
}

View File

@ -1,11 +1,13 @@
#pragma once
#include <Common/Arena.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesDecimal.h>
#include "DictionaryStructure.h"
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/DictionaryStructure.h>
namespace DB
{
@ -13,6 +15,190 @@ namespace DB
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
}
/** Simple helper for getting default.
* Initialized with default value and default values column.
* If default values column is not null default value is taken from column.
* If default value is null default value is taken from initializer.
*/
class DefaultValueProvider final
{
public:
explicit DefaultValueProvider(Field default_value_, ColumnPtr default_values_column_ = nullptr)
: default_value(std::move(default_value_))
, default_values_column(default_values_column_)
{
}
inline bool isConstant() const { return default_values_column == nullptr; }
Field getDefaultValue(size_t row) const
{
if (default_values_column)
return (*default_values_column)[row];
return default_value;
}
private:
Field default_value;
ColumnPtr default_values_column;
};
/** Support class for dictionary storages.
The main idea is that during fetch we create all columns, but fill only columns that client requested.
We need to create other columns during fetch, because in case of serialized storage we can skip
unnecessary columns serialized in cache with skipSerializedInArena method.
When result is fetched from the storage client of storage can filterOnlyNecessaryColumns
and get only columns that match attributes_names_to_fetch.
*/
class DictionaryStorageFetchRequest
{
public:
DictionaryStorageFetchRequest(const DictionaryStructure & structure, const Strings & attributes_names_to_fetch, Columns attributes_default_values_columns)
: attributes_to_fetch_names_set(attributes_names_to_fetch.begin(), attributes_names_to_fetch.end())
, attributes_to_fetch_filter(structure.attributes.size(), false)
{
assert(attributes_default_values_columns.size() == attributes_names_to_fetch.size());
if (attributes_to_fetch_names_set.size() != attributes_names_to_fetch.size())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Attribute names to fetch should be unique");
size_t attributes_size = structure.attributes.size();
dictionary_attributes_types.reserve(attributes_size);
attributes_default_value_providers.reserve(attributes_to_fetch_names_set.size());
size_t default_values_column_index = 0;
for (size_t i = 0; i < attributes_size; ++i)
{
const auto & dictionary_attribute = structure.attributes[i];
const auto & name = dictionary_attribute.name;
const auto & type = dictionary_attribute.type;
dictionary_attributes_types.emplace_back(type);
if (attributes_to_fetch_names_set.find(name) != attributes_to_fetch_names_set.end())
{
attributes_to_fetch_filter[i] = true;
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value, attributes_default_values_columns[default_values_column_index]);
++default_values_column_index;
}
else
attributes_default_value_providers.emplace_back(dictionary_attribute.null_value);
}
}
DictionaryStorageFetchRequest() = default;
/// Check requested attributes size
ALWAYS_INLINE size_t attributesSize() const
{
return dictionary_attributes_types.size();
}
/// Check if attribute with attribute_name was requested to fetch
ALWAYS_INLINE bool containsAttribute(const String & attribute_name) const
{
return attributes_to_fetch_names_set.find(attribute_name) != attributes_to_fetch_names_set.end();
}
/// Check if attribute with attribute_index should be filled during fetch
ALWAYS_INLINE bool shouldFillResultColumnWithIndex(size_t attribute_index) const
{
return attributes_to_fetch_filter[attribute_index];
}
const DataTypePtr & dataTypeAtIndex(size_t attribute_index) const
{
return dictionary_attributes_types[attribute_index];
}
const DefaultValueProvider & defaultValueProviderAtIndex(size_t attribute_index) const
{
return attributes_default_value_providers[attribute_index];
}
/// Create columns for each of dictionary attributes
MutableColumns makeAttributesResultColumns() const
{
MutableColumns result;
result.reserve(dictionary_attributes_types.size());
for (const auto & type : dictionary_attributes_types)
result.emplace_back(type->createColumn());
return result;
}
Columns makeAttributesResultColumnsNonMutable() const
{
Columns result;
result.reserve(dictionary_attributes_types.size());
for (const auto & type : dictionary_attributes_types)
result.emplace_back(type->createColumn());
return result;
}
/// Filter only requested columns
Columns filterRequestedColumns(MutableColumns & fetched_mutable_columns) const
{
Columns result;
result.reserve(dictionary_attributes_types.size());
for (size_t fetch_request_index = 0; fetch_request_index < dictionary_attributes_types.size(); ++fetch_request_index)
if (shouldFillResultColumnWithIndex(fetch_request_index))
result.emplace_back(std::move(fetched_mutable_columns[fetch_request_index]));
return result;
}
private:
std::unordered_set<String> attributes_to_fetch_names_set;
std::vector<bool> attributes_to_fetch_filter;
std::vector<DefaultValueProvider> attributes_default_value_providers;
DataTypes dictionary_attributes_types;
};
static inline void insertDefaultValuesIntoColumns(
MutableColumns & columns,
const DictionaryStorageFetchRequest & fetch_request,
size_t row_index)
{
size_t columns_size = columns.size();
for (size_t column_index = 0; column_index < columns_size; ++column_index)
{
const auto & column = columns[column_index];
const auto & default_value_provider = fetch_request.defaultValueProviderAtIndex(column_index);
if (fetch_request.shouldFillResultColumnWithIndex(column_index))
column->insert(default_value_provider.getDefaultValue(row_index));
}
}
/// Deserialize column value and insert it in columns.
/// Skip unnecessary columns that were not requested from deserialization.
static inline void deserializeAndInsertIntoColumns(
MutableColumns & columns,
const DictionaryStorageFetchRequest & fetch_request,
const char * place_for_serialized_columns)
{
size_t columns_size = columns.size();
for (size_t column_index = 0; column_index < columns_size; ++column_index)
{
const auto & column = columns[column_index];
if (fetch_request.shouldFillResultColumnWithIndex(column_index))
place_for_serialized_columns = column->deserializeAndInsertFromArena(place_for_serialized_columns);
else
place_for_serialized_columns = column->skipSerializedInArena(place_for_serialized_columns);
}
}
/**
@ -69,7 +255,7 @@ class DictionaryDefaultValueExtractor
public:
using DefaultValueType = DictionaryValueType<DictionaryAttributeType>;
DictionaryDefaultValueExtractor(DictionaryAttributeType attribute_default_value, ColumnPtr default_values_column_ = nullptr)
explicit DictionaryDefaultValueExtractor(DictionaryAttributeType attribute_default_value, ColumnPtr default_values_column_ = nullptr)
: default_value(std::move(attribute_default_value))
{
if (default_values_column_ == nullptr)
@ -109,6 +295,76 @@ private:
bool use_default_value_from_column = false;
};
template <DictionaryKeyType key_type>
class DictionaryKeysExtractor
{
public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
explicit DictionaryKeysExtractor(const Columns & key_columns, Arena & existing_arena)
{
assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple)
keys = getColumnVectorData(key_columns.front());
else
keys = deserializeKeyColumnsInArena(key_columns, existing_arena);
}
const PaddedPODArray<KeyType> & getKeys() const
{
return keys;
}
private:
static PaddedPODArray<UInt64> getColumnVectorData(const ColumnPtr column)
{
PaddedPODArray<UInt64> result;
auto full_column = column->convertToFullColumnIfConst();
const auto *vector_col = checkAndGetColumn<ColumnVector<UInt64>>(full_column.get());
if (!vector_col)
throw Exception{ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"};
result.assign(vector_col->getData());
return result;
}
static PaddedPODArray<StringRef> deserializeKeyColumnsInArena(const Columns & key_columns, Arena & temporary_arena)
{
size_t keys_size = key_columns.front()->size();
PaddedPODArray<StringRef> result;
result.reserve(keys_size);
PaddedPODArray<StringRef> temporary_column_data(key_columns.size());
for (size_t key_index = 0; key_index < keys_size; ++key_index)
{
size_t allocated_size_for_columns = 0;
const char * block_start = nullptr;
for (size_t column_index = 0; column_index < key_columns.size(); ++column_index)
{
const auto & column = key_columns[column_index];
temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_arena, block_start);
allocated_size_for_columns += temporary_column_data[column_index].size;
}
result.push_back(StringRef{block_start, allocated_size_for_columns});
}
return result;
}
PaddedPODArray<KeyType> keys;
};
/**
* Returns ColumnVector data as PaddedPodArray.

View File

@ -147,7 +147,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
id.emplace(config, structure_prefix + ".id");
else if (has_key)
{
key.emplace(getAttributes(config, structure_prefix + ".key", false, false));
key.emplace(getAttributes(config, structure_prefix + ".key", true));
if (key->empty())
throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS};
}
@ -196,7 +196,13 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
has_expressions = true;
}
attributes = getAttributes(config, structure_prefix);
attributes = getAttributes(config, structure_prefix, false);
for (size_t i = 0; i < attributes.size(); ++i)
{
const auto & attribute_name = attributes[i].name;
attribute_name_to_index[attribute_name] = i;
}
if (attributes.empty())
throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS};
@ -223,24 +229,25 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
}
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const String & attribute_name) const
const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string & attribute_name) const
{
auto find_iter
= std::find_if(attributes.begin(), attributes.end(), [&](const auto & attribute) { return attribute.name == attribute_name; });
if (find_iter != attributes.end())
return *find_iter;
auto it = attribute_name_to_index.find(attribute_name);
if (key && access_to_key_from_attributes)
if (it == attribute_name_to_index.end())
{
find_iter = std::find_if(key->begin(), key->end(), [&](const auto & attribute) { return attribute.name == attribute_name; });
if (find_iter != key->end())
return *find_iter;
if (!access_to_key_from_attributes)
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
for (const auto & key_attribute : *key)
if (key_attribute.name == attribute_name)
return key_attribute;
}
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
size_t attribute_index = it->second;
return attributes[attribute_index];
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const String & attribute_name, const DataTypePtr & type) const
const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string & attribute_name, const DataTypePtr & type) const
{
const auto & attribute = getAttribute(attribute_name);
@ -251,6 +258,14 @@ const DictionaryAttribute & DictionaryStructure::getAttribute(const String & att
return attribute;
}
size_t DictionaryStructure::getKeysSize() const
{
if (id)
return 1;
else
return key->size();
}
std::string DictionaryStructure::getKeyDescription() const
{
if (id)
@ -329,9 +344,12 @@ static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & k
std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed,
const bool allow_null_values)
bool complex_key_attributes)
{
/// If we request complex key attributes they does not support hierarchy and does not allow null values
const bool hierarchy_allowed = !complex_key_attributes;
const bool allow_null_values = !complex_key_attributes;
Poco::Util::AbstractConfiguration::Keys config_elems;
config.keys(config_prefix, config_elems);
auto has_hierarchy = false;
@ -358,7 +376,6 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
if ((range_min && name == range_min->name) || (range_max && name == range_max->name))
continue;
const auto type_string = config.getString(prefix + "type");
const auto initial_type = DataTypeFactory::instance().get(type_string);
auto type = initial_type;

View File

@ -1,17 +1,18 @@
#pragma once
#include <Core/Field.h>
#include <DataTypes/IDataType.h>
#include <IO/ReadBufferFromString.h>
#include <Interpreters/IExternalLoadable.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <map>
#include <optional>
#include <string>
#include <vector>
#include <Poco/Util/AbstractConfiguration.h>
#include <Core/Field.h>
#include <IO/ReadBufferFromString.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/IExternalLoadable.h>
namespace DB
{
@ -45,6 +46,7 @@ using DictionaryLifetime = ExternalLoadableLifetime;
/** Holds the description of a single dictionary attribute:
* - name, used for lookup into dictionary and source;
* - type, used in conjunction with DataTypeFactory and getAttributeUnderlyingTypeByname;
* - nested_type, contains nested type of complex type like Nullable, Array
* - null_value, used as a default value for non-existent entries in the dictionary,
* decimal representation for numeric attributes;
* - hierarchical, whether this attribute defines a hierarchy;
@ -147,6 +149,7 @@ struct DictionaryStructure final
std::optional<DictionarySpecialAttribute> id;
std::optional<std::vector<DictionaryAttribute>> key;
std::vector<DictionaryAttribute> attributes;
std::unordered_map<std::string, size_t> attribute_name_to_index;
std::optional<DictionaryTypedSpecialAttribute> range_min;
std::optional<DictionaryTypedSpecialAttribute> range_max;
bool has_expressions = false;
@ -155,8 +158,11 @@ struct DictionaryStructure final
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
void validateKeyTypes(const DataTypes & key_types) const;
const DictionaryAttribute & getAttribute(const String & attribute_name) const;
const DictionaryAttribute & getAttribute(const String & attribute_name, const DataTypePtr & type) const;
const DictionaryAttribute & getAttribute(const std::string & attribute_name) const;
const DictionaryAttribute & getAttribute(const std::string & attribute_name, const DataTypePtr & type) const;
size_t getKeysSize() const;
std::string getKeyDescription() const;
bool isKeySizeFixed() const;
size_t getKeySize() const;
@ -167,8 +173,7 @@ private:
std::vector<DictionaryAttribute> getAttributes(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed = true,
const bool allow_null_values = true);
bool complex_key_attributes);
};
}

View File

@ -6,19 +6,36 @@
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/HashMap.h>
#include <Interpreters/AggregationCommon.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD;
extern const int BAD_ARGUMENTS;
}
namespace
{
DirectDictionary::DirectDictionary(
inline UInt64 getAt(const PaddedPODArray<UInt64> & arr, const size_t idx)
{
return arr[idx];
}
inline UInt64 getAt(const UInt64 & value, const size_t)
{
return value;
}
}
template <DictionaryKeyType dictionary_key_type>
DirectDictionary<dictionary_key_type>::DirectDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
@ -28,36 +45,37 @@ DirectDictionary::DirectDictionary(
, source_ptr{std::move(source_ptr_)}
, saved_block{std::move(saved_block_)}
{
if (!this->source_ptr->supportsSelectiveLoad())
if (!source_ptr->supportsSelectiveLoad())
throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD};
createAttributes();
setup();
}
void DirectDictionary::toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const
template <DictionaryKeyType dictionary_key_type>
void DirectDictionary<dictionary_key_type>::toParent(const PaddedPODArray<Key> & ids [[maybe_unused]], PaddedPODArray<Key> & out [[maybe_unused]]) const
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
DictionaryDefaultValueExtractor<UInt64> extractor(null_value);
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
{
const auto & attribute_name = hierarchical_attribute->name;
getItemsImpl<UInt64, UInt64>(
*hierarchical_attribute,
ids,
[&](const size_t row, const UInt64 value, bool) { out[row] = value; },
extractor);
auto result_type = std::make_shared<DataTypeUInt64>();
auto input_column = result_type->createColumn();
auto & input_column_typed = assert_cast<ColumnVector<UInt64> &>(*input_column);
auto & data = input_column_typed.getData();
data.insert(ids.begin(), ids.end());
auto column = getColumn({attribute_name}, result_type, {std::move(input_column)}, {result_type}, {nullptr});
const auto & result_column_typed = assert_cast<const ColumnVector<UInt64> &>(*column);
const auto & result_data = result_column_typed.getData();
out.assign(result_data);
}
else
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Hierarchy is not supported for complex key DirectDictionary");
}
static inline DirectDictionary::Key getAt(const PaddedPODArray<DirectDictionary::Key> & arr, const size_t idx)
{
return arr[idx];
}
static inline DirectDictionary::Key getAt(const DirectDictionary::Key & value, const size_t)
{
return value;
}
DirectDictionary::Key DirectDictionary::getValueOrNullByKey(const Key & to_find) const
template <DictionaryKeyType dictionary_key_type>
UInt64 DirectDictionary<dictionary_key_type>::getValueOrNullByKey(const Key & to_find) const
{
std::vector<Key> required_key = {to_find};
@ -65,12 +83,13 @@ DirectDictionary::Key DirectDictionary::getValueOrNullByKey(const Key & to_find)
stream->readPrefix();
bool is_found = false;
Key result = std::get<Key>(hierarchical_attribute->null_values);
UInt64 result = hierarchical_attribute->null_value.template get<UInt64>();
while (const auto block = stream->read())
{
const IColumn & id_column = *block.safeGetByPosition(0).column;
for (const size_t attribute_idx : ext::range(0, attributes.size()))
for (const size_t attribute_idx : ext::range(0, dict_struct.attributes.size()))
{
if (is_found)
break;
@ -96,10 +115,11 @@ DirectDictionary::Key DirectDictionary::getValueOrNullByKey(const Key & to_find)
return result;
}
template <DictionaryKeyType dictionary_key_type>
template <typename ChildType, typename AncestorType>
void DirectDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
void DirectDictionary<dictionary_key_type>::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
const auto null_value = hierarchical_attribute->null_value.template get<UInt64>();
const auto rows = out.size();
for (const auto row : ext::range(0, rows))
@ -116,315 +136,295 @@ void DirectDictionary::isInImpl(const ChildType & child_ids, const AncestorType
query_count.fetch_add(rows, std::memory_order_relaxed);
}
void DirectDictionary::isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
template <DictionaryKeyType dictionary_key_type>
void DirectDictionary<dictionary_key_type>::isInVectorVector(
const PaddedPODArray<UInt64> & child_ids, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_ids, out);
}
void DirectDictionary::isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const
template <DictionaryKeyType dictionary_key_type>
void DirectDictionary<dictionary_key_type>::isInVectorConstant(const PaddedPODArray<UInt64> & child_ids, const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_id, out);
}
void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
template <DictionaryKeyType dictionary_key_type>
void DirectDictionary<dictionary_key_type>::isInConstantVector(const UInt64 child_id, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_id, ancestor_ids, out);
}
ColumnPtr DirectDictionary::getColumn(
template <DictionaryKeyType dictionary_key_type>
ColumnPtr DirectDictionary<dictionary_key_type>::getColumn(
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_values_column) const
const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column) const
{
ColumnPtr result;
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
dict_struct.validateKeyTypes(key_types);
PaddedPODArray<Key> backup_storage;
const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage);
Arena complex_key_arena;
const auto & attribute = getAttribute(attribute_name);
const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type);
DefaultValueProvider default_value_provider(attribute.null_value, default_values_column);
auto keys_size = ids.size();
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_key_arena);
const auto & requested_keys = extractor.getKeys();
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to = nullptr;
if (attribute.is_nullable)
{
col_null_map_to = ColumnUInt8::create(keys_size, false);
vec_null_map_to = &col_null_map_to->getData();
}
HashMap<KeyType, size_t> key_to_fetched_index;
key_to_fetched_index.reserve(requested_keys.size());
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto fetched_from_storage = attribute.type->createColumn();
size_t fetched_key_index = 0;
size_t requested_attribute_index = attribute_index_by_name.find(attribute_name)->second;
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
Columns block_key_columns;
size_t dictionary_keys_size = dict_struct.getKeysNames().size();
block_key_columns.reserve(dictionary_keys_size);
using ValueType = DictionaryValueType<AttributeType>;
using ColumnProvider = DictionaryAttributeColumnProvider<AttributeType>;
BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys);
const auto attribute_null_value = std::get<ValueType>(attribute.null_values);
AttributeType null_value = static_cast<AttributeType>(attribute_null_value);
DictionaryDefaultValueExtractor<AttributeType> default_value_extractor(std::move(null_value), default_values_column);
auto column = ColumnProvider::getColumn(dictionary_attribute, keys_size);
if constexpr (std::is_same_v<AttributeType, String>)
{
auto * out = column.get();
getItemsImpl<String, String>(
attribute,
ids,
[&](const size_t row, const String value, bool is_null)
{
if (attribute.is_nullable)
(*vec_null_map_to)[row] = is_null;
const auto ref = StringRef{value};
out->insertData(ref.data, ref.size);
},
default_value_extractor);
}
else
{
auto & out = column->getData();
getItemsImpl<AttributeType, AttributeType>(
attribute,
ids,
[&](const size_t row, const auto value, bool is_null)
{
if (attribute.is_nullable)
(*vec_null_map_to)[row] = is_null;
out[row] = value;
},
default_value_extractor);
}
result = std::move(column);
};
callOnDictionaryAttributeType(attribute.type, type_call);
if (attribute.is_nullable)
{
result = ColumnNullable::create(result, std::move(col_null_map_to));
}
return result;
}
ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & key_columns, const DataTypes &) const
{
PaddedPODArray<Key> backup_storage;
const auto& ids = getColumnVectorData(this, key_columns.front(), backup_storage);
auto result = ColumnUInt8::create(ext::size(ids));
auto& out = result->getData();
const auto rows = ext::size(ids);
HashMap<Key, UInt8> has_key;
for (const auto row : ext::range(0, rows))
has_key[ids[row]] = 0;
std::vector<Key> to_load;
to_load.reserve(has_key.size());
for (auto it = has_key.begin(); it != has_key.end(); ++it)
to_load.emplace_back(static_cast<Key>(it->getKey()));
auto stream = source_ptr->loadIds(to_load);
stream->readPrefix();
while (const auto block = stream->read())
{
const IColumn & id_column = *block.safeGetByPosition(0).column;
auto block_columns = block.getColumns();
for (const auto row_idx : ext::range(0, id_column.size()))
/// Split into keys columns and attribute columns
for (size_t i = 0; i < dictionary_keys_size; ++i)
{
const auto key = id_column[row_idx].get<UInt64>();
has_key[key] = 1;
block_key_columns.emplace_back(*block_columns.begin());
block_columns.erase(block_columns.begin());
}
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, complex_key_arena);
const auto & block_keys = block_keys_extractor.getKeys();
size_t block_keys_size = block_keys.size();
const auto & block_column = block.safeGetByPosition(dictionary_keys_size + requested_attribute_index).column;
fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys_size);
for (size_t block_key_index = 0; block_key_index < block_keys_size; ++block_key_index)
{
const auto & block_key = block_keys[block_key_index];
key_to_fetched_index[block_key] = fetched_key_index;
++fetched_key_index;
}
block_key_columns.clear();
}
stream->readSuffix();
for (const auto row : ext::range(0, rows))
out[row] = has_key[ids[row]];
Field value_to_insert;
query_count.fetch_add(rows, std::memory_order_relaxed);
size_t requested_keys_size = requested_keys.size();
auto result = fetched_from_storage->cloneEmpty();
result->reserve(requested_keys_size);
for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index)
{
const auto requested_key = requested_keys[requested_key_index];
const auto * it = key_to_fetched_index.find(requested_key);
if (it)
fetched_from_storage->get(it->getMapped(), value_to_insert);
else
value_to_insert = default_value_provider.getDefaultValue(requested_key_index);
result->insert(value_to_insert);
}
query_count.fetch_add(requested_keys_size, std::memory_order_relaxed);
return result;
}
void DirectDictionary::createAttributes()
template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types [[maybe_unused]]) const
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
dict_struct.validateKeyTypes(key_types);
for (const auto & attribute : dict_struct.attributes)
Arena complex_key_arena;
DictionaryKeysExtractor<dictionary_key_type> requested_keys_extractor(key_columns, complex_key_arena);
const auto & requested_keys = requested_keys_extractor.getKeys();
size_t requested_keys_size = requested_keys.size();
HashMap<KeyType, size_t> requested_key_to_index;
requested_key_to_index.reserve(requested_keys_size);
for (size_t i = 0; i < requested_keys.size(); ++i)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attribute_name_by_index.emplace(attributes.size(), attribute.name);
attributes.push_back(createAttribute(attribute, attribute.null_value, attribute.name));
auto requested_key = requested_keys[i];
requested_key_to_index[requested_key] = i;
}
auto result = ColumnUInt8::create(requested_keys_size, false);
auto & result_data = result->getData();
Columns block_key_columns;
size_t dictionary_keys_size = dict_struct.getKeysNames().size();
block_key_columns.reserve(dictionary_keys_size);
BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys);
stream->readPrefix();
while (const auto block = stream->read())
{
auto block_columns = block.getColumns();
/// Split into keys columns and attribute columns
for (size_t i = 0; i < dictionary_keys_size; ++i)
{
block_key_columns.emplace_back(*block_columns.begin());
block_columns.erase(block_columns.begin());
}
DictionaryKeysExtractor<dictionary_key_type> block_keys_extractor(block_key_columns, complex_key_arena);
const auto & block_keys = block_keys_extractor.getKeys();
for (const auto & block_key : block_keys)
{
const auto * it = requested_key_to_index.find(block_key);
assert(it);
size_t result_data_found_index = it->getMapped();
result_data[result_data_found_index] = true;
}
block_key_columns.clear();
}
stream->readSuffix();
query_count.fetch_add(requested_keys_size, std::memory_order_relaxed);
return result;
}
template <DictionaryKeyType dictionary_key_type>
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
const Columns & key_columns [[maybe_unused]],
const PaddedPODArray<KeyType> & requested_keys [[maybe_unused]]) const
{
size_t requested_keys_size = requested_keys.size();
BlockInputStreamPtr stream;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
{
std::vector<UInt64> ids;
ids.reserve(requested_keys_size);
for (auto key : requested_keys)
ids.emplace_back(key);
stream = source_ptr->loadIds(ids);
}
else
{
std::vector<size_t> requested_rows;
requested_rows.reserve(requested_keys_size);
for (size_t i = 0; i < requested_keys_size; ++i)
requested_rows.emplace_back(i);
stream = source_ptr->loadKeys(key_columns, requested_rows);
}
return stream;
}
template <DictionaryKeyType dictionary_key_type>
void DirectDictionary<dictionary_key_type>::setup()
{
/// TODO: Move this to DictionaryStructure
size_t dictionary_attributes_size = dict_struct.attributes.size();
for (size_t i = 0; i < dictionary_attributes_size; ++i)
{
const auto & attribute = dict_struct.attributes[i];
attribute_index_by_name[attribute.name] = i;
attribute_name_by_index[i] = attribute.name;
if (attribute.hierarchical)
{
hierarchical_attribute = &attributes.back();
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): hierarchical attributes are not supported for complex key direct dictionary",
full_name);
if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64)
hierarchical_attribute = &attribute;
if (attribute.underlying_type != AttributeUnderlyingType::utUInt64)
throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH};
}
}
}
template <typename T>
void DirectDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)
{
attribute.null_values = T(null_value.get<NearestFieldType<T>>());
}
template <>
void DirectDictionary::createAttributeImpl<String>(Attribute & attribute, const Field & null_value)
{
attribute.string_arena = std::make_unique<Arena>();
const String & string = null_value.get<String>();
const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
attribute.null_values.emplace<StringRef>(string_in_arena, string.size());
}
DirectDictionary::Attribute DirectDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value, const std::string & attr_name)
{
Attribute attr{attribute.underlying_type, attribute.is_nullable, {}, {}, attr_name};
auto type_call = [&](const auto &dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
createAttributeImpl<AttributeType>(attr, null_value);
};
callOnDictionaryAttributeType(attribute.underlying_type, type_call);
return attr;
}
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void DirectDictionary::getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const
{
const auto rows = ext::size(ids);
HashMap<Key, OutputType> value_by_key;
HashSet<Key> value_is_null;
for (const auto row : ext::range(0, rows))
{
auto key = ids[row];
value_by_key[key] = static_cast<AttributeType>(default_value_extractor[row]);
}
std::vector<Key> to_load;
to_load.reserve(value_by_key.size());
for (auto it = value_by_key.begin(); it != value_by_key.end(); ++it)
to_load.emplace_back(static_cast<Key>(it->getKey()));
auto stream = source_ptr->loadIds(to_load);
stream->readPrefix();
const auto it = attribute_index_by_name.find(attribute.name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute.name + "'", ErrorCodes::BAD_ARGUMENTS};
auto attribute_index = it->second;
while (const auto block = stream->read())
{
const IColumn & id_column = *block.safeGetByPosition(0).column;
const IColumn & attribute_column = *block.safeGetByPosition(attribute_index + 1).column;
for (const auto row_idx : ext::range(0, id_column.size()))
{
const auto key = id_column[row_idx].get<UInt64>();
if (value_by_key.find(key) != value_by_key.end())
{
auto value = attribute_column[row_idx];
if (value.isNull())
value_is_null.insert(key);
else
value_by_key[key] = static_cast<OutputType>(value.get<NearestFieldType<AttributeType>>());
}
}
}
stream->readSuffix();
for (const auto row : ext::range(0, rows))
{
auto key = ids[row];
set_value(row, value_by_key[key], value_is_null.find(key) != nullptr);
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
const DirectDictionary::Attribute & DirectDictionary::getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
return attributes[it->second];
}
BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const
template <DictionaryKeyType dictionary_key_type>
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const
{
return source_ptr->loadAll();
}
void registerDictionaryDirect(DictionaryFactory & factory)
namespace
{
auto create_layout = [=](const std::string & full_name,
template <DictionaryKeyType dictionary_key_type>
DictionaryPtr createDirectDictionary(
const std::string & full_name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr)
{
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct";
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'direct'", ErrorCodes::UNSUPPORTED_METHOD};
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"'key' is not supported for dictionary of layout '({})'",
layout_name);
}
else
{
if (dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"'id' is not supported for dictionary of layout '({})'",
layout_name);
}
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{full_name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): elements .structure.range_min and .structure.range_max should be defined only " \
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
full_name);
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max"))
throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS};
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"'lifetime' parameter is redundant for the dictionary' of layout '({})'",
layout_name);
return std::make_unique<DirectDictionary<dictionary_key_type>>(dict_id, dict_struct, std::move(source_ptr));
}
}
return std::make_unique<DirectDictionary>(dict_id, dict_struct, std::move(source_ptr));
};
factory.registerLayout("direct", create_layout, false);
template class DirectDictionary<DictionaryKeyType::simple>;
template class DirectDictionary<DictionaryKeyType::complex>;
void registerDictionaryDirect(DictionaryFactory & factory)
{
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::simple>, false);
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::complex>, true);
}

View File

@ -18,16 +18,31 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
template <DictionaryKeyType dictionary_key_type>
class DirectDictionary final : public IDictionary
{
public:
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by direct dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
DirectDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_ = nullptr);
std::string getTypeName() const override { return "Direct"; }
std::string getTypeName() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return "Direct";
else
return "ComplexKeyDirect";
}
size_t getBytesAllocated() const override { return 0; }
@ -52,82 +67,45 @@ public:
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
auto it = attribute_index_by_name.find(attribute_name);
if (it == attribute_index_by_name.end())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"({}): no attribute with name ({}) in dictionary",
full_name,
attribute_name);
return dict_struct.attributes[it->second].injective;
}
bool hasHierarchy() const override { return hierarchical_attribute; }
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
void toParent(const PaddedPODArray<UInt64> & ids, PaddedPODArray<UInt64> & out) const override;
void isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
const PaddedPODArray<UInt64> & child_ids, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<UInt64> & child_ids, const UInt64 ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const UInt64 child_id, const PaddedPODArray<UInt64> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
ColumnPtr getColumn(
const std::string& attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_values_column) const override;
const ColumnPtr & default_values_column) const override;
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
struct Attribute final
{
AttributeUnderlyingType type;
bool is_nullable;
std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
StringRef>
null_values;
std::unique_ptr<Arena> string_arena;
std::string name;
};
void setup();
void createAttributes();
BlockInputStreamPtr getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray<KeyType> & requested_keys) const;
template <typename T>
void addAttributeSize(const Attribute & attribute);
template <typename T>
static void createAttributeImpl(Attribute & attribute, const Field & null_value);
static Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value, const std::string & name);
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
void getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const;
template <typename T>
void setAttributeValueImpl(Attribute & attribute, const Key id, const T & value);
void setAttributeValue(Attribute & attribute, const Key id, const Field & value);
const Attribute & getAttribute(const std::string & attribute_name) const;
Key getValueOrNullByKey(const Key & to_find) const;
UInt64 getValueOrNullByKey(const UInt64 & to_find) const;
template <typename ChildType, typename AncestorType>
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
@ -136,14 +114,17 @@ private:
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
std::map<std::string, size_t> attribute_index_by_name;
std::map<size_t, std::string> attribute_name_by_index;
std::vector<Attribute> attributes;
const Attribute * hierarchical_attribute = nullptr;
std::unordered_map<std::string, size_t> attribute_index_by_name;
std::unordered_map<size_t, std::string> attribute_name_by_index;
const DictionaryAttribute * hierarchical_attribute = nullptr;
mutable std::atomic<size_t> query_count{0};
BlockPtr saved_block;
};
extern template class DirectDictionary<DictionaryKeyType::simple>;
extern template class DirectDictionary<DictionaryKeyType::complex>;
}

Some files were not shown because too many files have changed in this diff Show More