mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge remote-tracking branch 'origin/master' into fixes-clang-12-cmake-3.19
This commit is contained in:
commit
abbf4b01c2
@ -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());
|
||||
|
||||
|
1
contrib/CMakeLists.txt
vendored
1
contrib/CMakeLists.txt
vendored
@ -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
2
contrib/NuRaft
vendored
@ -1 +1 @@
|
||||
Subproject commit 9a0d78de4b90546368d954b6434f0e9a823e8d80
|
||||
Subproject commit ff9049bcc8ea6a02276ccdc8629d764e9e5de853
|
18
contrib/abseil-cpp-cmake/CMakeLists.txt
Normal file
18
contrib/abseil-cpp-cmake/CMakeLists.txt
Normal 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})
|
@ -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
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit fbaaba4a02e29987b8c584747a496c79528f125f
|
||||
Subproject commit c55b91f394efa9c238c33957682501681ef9b716
|
@ -151,6 +151,7 @@ function clone_submodules
|
||||
cd "$FASTTEST_SOURCE"
|
||||
|
||||
SUBMODULES_TO_UPDATE=(
|
||||
contrib/abseil-cpp
|
||||
contrib/antlr4-runtime
|
||||
contrib/boost
|
||||
contrib/zlib-ng
|
||||
|
@ -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 ||:
|
||||
|
@ -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). -->
|
||||
|
@ -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).
|
||||
|
||||
|
@ -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"))
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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(); }
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
{
|
||||
}
|
||||
|
@ -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 (...)
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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) \
|
||||
|
@ -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 {};
|
||||
|
@ -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>;
|
||||
|
@ -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)...);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
namespace Coordination
|
||||
{
|
||||
|
||||
|
||||
void write(size_t x, WriteBuffer & out)
|
||||
{
|
||||
x = __builtin_bswap64(x);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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();
|
||||
|
||||
|
363
src/Coordination/NuKeeperSnapshotManager.cpp
Normal file
363
src/Coordination/NuKeeperSnapshotManager.cpp
Normal 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);
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
85
src/Coordination/NuKeeperSnapshotManager.h
Normal file
85
src/Coordination/NuKeeperSnapshotManager.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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 (...)
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
197
src/Coordination/SnapshotableHashTable.h
Normal file
197
src/Coordination/SnapshotableHashTable.h
Normal 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(); }
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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>
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
@ -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
@ -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>;
|
||||
|
||||
}
|
||||
|
418
src/Dictionaries/CacheDictionaryStorage.h
Normal file
418
src/Dictionaries/CacheDictionaryStorage.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
162
src/Dictionaries/CacheDictionaryUpdateQueue.cpp
Normal file
162
src/Dictionaries/CacheDictionaryUpdateQueue.cpp
Normal 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>;
|
||||
|
||||
}
|
172
src/Dictionaries/CacheDictionaryUpdateQueue.h
Normal file
172
src/Dictionaries/CacheDictionaryUpdateQueue.h
Normal 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>;
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -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();
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -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()};
|
||||
};
|
||||
|
||||
}
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user